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 2020/03/30 10:16:04 UTC

[GitHub] [incubator-doris] worker24h opened a new pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

worker24h opened a new pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230
 
 
   Doris support load json-data by RoutineLoad or StreamLoad.  Ref: https://github.com/apache/incubator-doris/issues/3124
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r400086167
 
 

 ##########
 File path: docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/ROUTINE LOAD.md
 ##########
 @@ -130,15 +130,23 @@ under the License.
             采样窗口内,允许的最大错误行数。必须大于等于0。默认是 0,即不允许有错误行。
             采样窗口为 max_batch_rows * 10。即如果在采样窗口内,错误行数大于 max_error_number,则会导致例行作业被暂停,需要人工介入检查数据质量问题。
             被 where 条件过滤掉的行不算错误行。
-        
+
         4. strict_mode
 
             是否开启严格模式,默认为开启。如果开启后,非空原始数据的列类型变换如果结果为 NULL,则会被过滤。指定方式为 "strict_mode" = "true"
 
-        5. timezone
-            
+        5. `timezone`
+
             指定导入作业所使用的时区。默认为使用 Session 的 timezone 参数。该参数会影响所有导入涉及的和时区有关的函数结果。
 
+        6. `format`
 
 Review comment:
   ```suggestion
           6. format
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r400085305
 
 

 ##########
 File path: docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/ROUTINE LOAD.md
 ##########
 @@ -130,15 +130,23 @@ under the License.
             采样窗口内,允许的最大错误行数。必须大于等于0。默认是 0,即不允许有错误行。
             采样窗口为 max_batch_rows * 10。即如果在采样窗口内,错误行数大于 max_error_number,则会导致例行作业被暂停,需要人工介入检查数据质量问题。
             被 where 条件过滤掉的行不算错误行。
-        
+
         4. strict_mode
 
             是否开启严格模式,默认为开启。如果开启后,非空原始数据的列类型变换如果结果为 NULL,则会被过滤。指定方式为 "strict_mode" = "true"
 
-        5. timezone
-            
+        5. `timezone`
 
 Review comment:
   ```suggestion
           5. timezone
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r410011861
 
 

 ##########
 File path: be/src/exec/json_scanner.cpp
 ##########
 @@ -0,0 +1,485 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+         _jsonValues(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::isEnd() {
+    return _jsonValues->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::getNext() {
+    if (isEnd()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonPathDoc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+                _parseJsonPathFlag = -1;// failed, has none object
+            } else {
+                _parseJsonPathFlag = 1;// success
+            }
+        } else {
+            _parseJsonPathFlag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parseJsonPathFlag = parseJsonPathFromFile(fileMgr, jsonpath_file);
+    } else {
+        _parseJsonPathFlag = 0;
+    }
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parseJsonPathFromFile(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parseJsonPathFromFile Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonPathDoc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parseJsonDoc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr; //
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_jsonDoc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::getDataByJsonPath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonPathDoc["jsonpath"];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* jsonValues = JsonFunctions::get_json_object_simple(value, &_jsonDoc);
+        if (jsonValues == NULL) {
+            return -1;
+        }
+        if (jsonValues->IsArray()) {
+            max_lines = max_lines < jsonValues->Size() ? jsonValues->Size() : max_lines;
+        } else {
+            max_lines = max_lines > 1 ? max_lines : 1;
+        }
+        jmap.emplace(column, jsonValues);
+    }
+    return max_lines;
+}
+
+void JsonReader::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {
+    tuple->set_not_null(slot_desc->null_indicator_offset());
+    void* slot = tuple->get_slot(slot_desc->tuple_offset());
+    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(str_slot->ptr, value, len);
+    str_slot->len = len;
+    return;
+}
+
+Status JsonReader::writeDataToTuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool) {
+    const char *str_value = NULL;
+    uint8_t tmp_buf[128] = {0};
+    int32_t wbytes = 0;
+    switch (value->GetType()) {
+        case rapidjson::Type::kStringType:
+            str_value = value->GetString();
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)str_value, strlen(str_value));
+            break;
+        case rapidjson::Type::kNumberType:
+            if (value->IsUint()) {
+                wbytes = sprintf((char*)tmp_buf, "%u", value->GetUint());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt()) {
+                wbytes = sprintf((char*)tmp_buf, "%d", value->GetInt());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsUint64()) {
+                wbytes = sprintf((char*)tmp_buf, "%lu", value->GetUint64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt64()) {
+                wbytes = sprintf((char*)tmp_buf, "%ld", value->GetInt64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else {
+                wbytes = sprintf((char*)tmp_buf, "%f", value->GetDouble());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            }
+            break;
+        case rapidjson::Type::kFalseType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"false", 5);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"0", 1);
+            break;
+        case rapidjson::Type::kTrueType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"true", 4);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"1", 1);
+            break;
+        default:
+            std::stringstream str_error;
+            str_error << "Invalid JsonType " << value->GetType() << ", Column Name `" << desc->col_name() << "`.";
+            LOG(WARNING) << str_error.str();
+            return Status::RuntimeError(str_error.str());
+    }
+    return Status::OK();
+}
+
+/**
+ * handle input a simple json
+ * For example:
+ *  1. {"doris_data": [{"colunm1":"value1", "colunm2":10}, {"colunm1":"value2", "colunm2":30}]}
+ *  2. {"colunm1":"value1", "colunm2":10}
+ */
+Status JsonReader::handleSimpleJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof) {
+    if (_next_line >= _total_lines) {
+        // generic document
+        RETURN_IF_ERROR(parseJsonDoc(eof));
+        if (*eof) {// handle over all data
+            return Status::OK();
+        }
+        if (_jsonDoc.HasMember("doris_data") && _jsonDoc["doris_data"].IsArray() ) {
+            _total_lines = _jsonDoc["doris_data"].Size();
+        } else {
+            _total_lines = 1;
+        }
+        _next_line = 0;
+    }
+
+    if (_jsonDoc.HasMember("doris_data") && _jsonDoc["doris_data"].IsArray()) {//handle case 1
+        rapidjson::Value& valueArray = _jsonDoc["doris_data"];
+        rapidjson::Value& objectValue = valueArray[_next_line++];// json object
+        for (auto v : slot_descs) {
+            if (objectValue.HasMember(v->col_name().c_str())) {
+                rapidjson::Value& value = objectValue[v->col_name().c_str()];
+                writeDataToTuple(&value, v, tuple, tuple_pool);
+            } else {
+                if (v->is_nullable()) {
+                    tuple->set_null(v->null_indicator_offset());
+                } else  {
+                    std::stringstream str_error;
+                    str_error << "The column `" << v->col_name() << "` is not nullable, but it's not found in jsondata.";
+                    LOG(WARNING) << str_error.str();
+                    return Status::RuntimeError(str_error.str());
+                }
+            }
+        }
+    } else {// handle case 2
+        int nullcount = 0;
+        for (auto v : slot_descs) {
+            if (_jsonDoc.HasMember(v->col_name().c_str())) {
+                rapidjson::Value& value = _jsonDoc[v->col_name().c_str()];
+                writeDataToTuple(&value, v, tuple, tuple_pool);
 
 Review comment:
   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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r410710996
 
 

 ##########
 File path: be/src/exec/json_scanner.cpp
 ##########
 @@ -0,0 +1,485 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+         _jsonValues(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::isEnd() {
+    return _jsonValues->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::getNext() {
+    if (isEnd()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    //parse jsonpath
+    if (!jsonpath.empty()) {
 
 Review comment:
   It's better to create a `init()` method for JsonReader, like:
   ```
   JsonReader* reader = new JsonReader();
   Status st = reader->init();
   if (!st.ok()) {
       // handle the error
   }
   ```
   
   And `_parseJsonPathFlag` is only used for simple or complex json flag

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r409998491
 
 

 ##########
 File path: be/src/exec/json_scanner.cpp
 ##########
 @@ -0,0 +1,485 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+         _jsonValues(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::isEnd() {
+    return _jsonValues->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::getNext() {
+    if (isEnd()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonPathDoc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+                _parseJsonPathFlag = -1;// failed, has none object
+            } else {
+                _parseJsonPathFlag = 1;// success
+            }
+        } else {
+            _parseJsonPathFlag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parseJsonPathFlag = parseJsonPathFromFile(fileMgr, jsonpath_file);
+    } else {
+        _parseJsonPathFlag = 0;
+    }
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parseJsonPathFromFile(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parseJsonPathFromFile Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonPathDoc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parseJsonDoc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr; //
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_jsonDoc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::getDataByJsonPath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonPathDoc["jsonpath"];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* jsonValues = JsonFunctions::get_json_object_simple(value, &_jsonDoc);
+        if (jsonValues == NULL) {
+            return -1;
+        }
+        if (jsonValues->IsArray()) {
+            max_lines = max_lines < jsonValues->Size() ? jsonValues->Size() : max_lines;
+        } else {
+            max_lines = max_lines > 1 ? max_lines : 1;
+        }
+        jmap.emplace(column, jsonValues);
+    }
+    return max_lines;
+}
+
+void JsonReader::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {
+    tuple->set_not_null(slot_desc->null_indicator_offset());
+    void* slot = tuple->get_slot(slot_desc->tuple_offset());
+    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(str_slot->ptr, value, len);
+    str_slot->len = len;
+    return;
+}
+
+Status JsonReader::writeDataToTuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool) {
+    const char *str_value = NULL;
+    uint8_t tmp_buf[128] = {0};
+    int32_t wbytes = 0;
+    switch (value->GetType()) {
+        case rapidjson::Type::kStringType:
+            str_value = value->GetString();
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)str_value, strlen(str_value));
+            break;
+        case rapidjson::Type::kNumberType:
+            if (value->IsUint()) {
+                wbytes = sprintf((char*)tmp_buf, "%u", value->GetUint());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt()) {
+                wbytes = sprintf((char*)tmp_buf, "%d", value->GetInt());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsUint64()) {
+                wbytes = sprintf((char*)tmp_buf, "%lu", value->GetUint64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt64()) {
+                wbytes = sprintf((char*)tmp_buf, "%ld", value->GetInt64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else {
+                wbytes = sprintf((char*)tmp_buf, "%f", value->GetDouble());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            }
+            break;
+        case rapidjson::Type::kFalseType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"false", 5);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"0", 1);
+            break;
+        case rapidjson::Type::kTrueType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"true", 4);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"1", 1);
+            break;
+        default:
+            std::stringstream str_error;
+            str_error << "Invalid JsonType " << value->GetType() << ", Column Name `" << desc->col_name() << "`.";
+            LOG(WARNING) << str_error.str();
+            return Status::RuntimeError(str_error.str());
+    }
+    return Status::OK();
+}
+
+/**
+ * handle input a simple json
+ * For example:
+ *  1. {"doris_data": [{"colunm1":"value1", "colunm2":10}, {"colunm1":"value2", "colunm2":30}]}
+ *  2. {"colunm1":"value1", "colunm2":10}
+ */
+Status JsonReader::handleSimpleJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof) {
+    if (_next_line >= _total_lines) {
+        // generic document
+        RETURN_IF_ERROR(parseJsonDoc(eof));
+        if (*eof) {// handle over all data
+            return Status::OK();
+        }
+        if (_jsonDoc.HasMember("doris_data") && _jsonDoc["doris_data"].IsArray() ) {
+            _total_lines = _jsonDoc["doris_data"].Size();
+        } else {
+            _total_lines = 1;
+        }
+        _next_line = 0;
+    }
+
+    if (_jsonDoc.HasMember("doris_data") && _jsonDoc["doris_data"].IsArray()) {//handle case 1
+        rapidjson::Value& valueArray = _jsonDoc["doris_data"];
+        rapidjson::Value& objectValue = valueArray[_next_line++];// json object
+        for (auto v : slot_descs) {
+            if (objectValue.HasMember(v->col_name().c_str())) {
+                rapidjson::Value& value = objectValue[v->col_name().c_str()];
+                writeDataToTuple(&value, v, tuple, tuple_pool);
+            } else {
+                if (v->is_nullable()) {
+                    tuple->set_null(v->null_indicator_offset());
+                } else  {
+                    std::stringstream str_error;
+                    str_error << "The column `" << v->col_name() << "` is not nullable, but it's not found in jsondata.";
+                    LOG(WARNING) << str_error.str();
+                    return Status::RuntimeError(str_error.str());
+                }
+            }
+        }
+    } else {// handle case 2
+        int nullcount = 0;
+        for (auto v : slot_descs) {
+            if (_jsonDoc.HasMember(v->col_name().c_str())) {
+                rapidjson::Value& value = _jsonDoc[v->col_name().c_str()];
+                writeDataToTuple(&value, v, tuple, tuple_pool);
+            } else {
+                if (v->is_nullable()) {
+                    nullcount++;
+                    tuple->set_null(v->null_indicator_offset());
+                } else  {
+                    std::stringstream str_error;
+                    str_error << "The column `" << v->col_name() << "` is not nullable, but it's not found in jsondata.";
+                    LOG(WARNING) << str_error.str();
+                    return Status::RuntimeError(str_error.str());
+                }
+            }
+        }
+        if (nullcount == slot_descs.size()) {// All fields is null, then it's judged a failure
+            return Status::RuntimeError("All column names were not found in the json data.");
+        }
+        _next_line = 1;//only one row, so set _next_line = 1
+    }
+    return Status::OK();
+}
+
+Status JsonReader::handleComplexJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof) {
+    if (_next_line >= _total_lines) {
+        RETURN_IF_ERROR(parseJsonDoc(eof));
+        if (*eof) {
+            return Status::OK();
+        }
+        _total_lines = getDataByJsonPath();
+        if (_total_lines == -1) {
+            return Status::InternalError("Parse json data is failed.");
+        } else if (_total_lines == 0) {
+            *eof = true;
+            return Status::OK();
+        }
+        _next_line = 0;
+    }
+
+    std::map<std::string, JsonDataInternal>::iterator it_map;
+    for (auto v : slot_descs) {
+        it_map = jmap.find(v->col_name());
+        if (it_map == jmap.end()) {
+            return Status::RuntimeError("The column name of table is not foud in jsonpath.");
+        }
+        rapidjson::Value::ConstValueIterator value = it_map->second.getNext();
+        if (value == nullptr) {
+            if (v->is_nullable()) {
+                tuple->set_null(v->null_indicator_offset());
+            } else  {
+                std::stringstream str_error;
+                str_error << "The column `" << it_map->first << "` is not nullable, but it's not found in jsondata.";
+                LOG(WARNING) << str_error.str();
+                return Status::RuntimeError(str_error.str());
+            }
+        } else {
+            writeDataToTuple(value, v, tuple, tuple_pool);
 
 Review comment:
   return status

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r410707220
 
 

 ##########
 File path: be/src/exec/json_scanner.h
 ##########
 @@ -0,0 +1,140 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to
+    Status open() override;
+
+    // Get next tuple
+    Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) override;
+
+    // Close this scanner
+    void close() override;
+private:
+    Status open_next_reader();
+private:
+    //const TBrokerScanRangeParams& _params;
+    const std::vector<TBrokerRangeDesc>& _ranges;
+    const std::vector<TNetworkAddress>& _broker_addresses;
+
+    std::string _jsonpath;
+    std::string _jsonpath_file;
+    // std::shared_ptr<rapidjson::Document> _jsonPathDoc;
+
+    // used to hold current StreamLoadPipe
+    std::shared_ptr<StreamLoadPipe> _stream_load_pipe;
+    // Reader
+    JsonReader* _cur_file_reader;
+    int _next_range;
+    bool _cur_file_eof; // is read over?
+    bool _scanner_eof;
+};
+
+
+class JsonDataInternal {
+public:
+    JsonDataInternal(rapidjson::Value* v);
+    ~JsonDataInternal();
+    rapidjson::Value::ConstValueIterator getNext();
+
+private:
+    bool isEnd();
+
+private:
+    rapidjson::Value* _jsonValues;
+    rapidjson::Value::ConstValueIterator  _iterator;
+};
+
+class JsonReader {
+public:
+    JsonReader(SmallFileMgr *fileMgr, RuntimeProfile* profile, FileReader* file_reader,
+            std::string& jsonpath, std::string& jsonpath_file);
+    ~JsonReader();
+    Status read(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof);
+
+private:
+    void fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len);
+    size_t getDataByJsonPath();
+    int parseJsonPathFromFile(SmallFileMgr *smallFileMgr, std::string& fileinfo);
+    Status parseJsonDoc(bool *eof);
+    Status handleSimpleJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof);
+    Status handleComplexJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof);
+    Status writeDataToTuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool);
 
 Review comment:
   CodeStyle, in C++, it should be `write_data_to_tuple`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r400089614
 
 

 ##########
 File path: gensrc/thrift/BackendService.thrift
 ##########
 @@ -54,11 +55,12 @@ struct TRoutineLoadTask {
     6: optional string db
     7: optional string tbl
     8: optional string label
-    9: optional i64 max_interval_s
-    10: optional i64 max_batch_rows
-    11: optional i64 max_batch_size
-    12: optional TKafkaLoadInfo kafka_load_info
-    13: optional PaloInternalService.TExecPlanFragmentParams params
+    9: optional PlanNodes.TFileFormatType format
 
 Review comment:
   Need field in thrift should put at end

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r410713273
 
 

 ##########
 File path: be/src/exec/json_scanner.cpp
 ##########
 @@ -0,0 +1,485 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+         _jsonValues(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::isEnd() {
+    return _jsonValues->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::getNext() {
+    if (isEnd()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonPathDoc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+                _parseJsonPathFlag = -1;// failed, has none object
+            } else {
+                _parseJsonPathFlag = 1;// success
+            }
+        } else {
+            _parseJsonPathFlag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parseJsonPathFlag = parseJsonPathFromFile(fileMgr, jsonpath_file);
+    } else {
+        _parseJsonPathFlag = 0;
+    }
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parseJsonPathFromFile(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parseJsonPathFromFile Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonPathDoc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parseJsonDoc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr; //
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_jsonDoc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::getDataByJsonPath() {
 
 Review comment:
   In c language code style, you should define this method as:
   ```
   Status JsonReader::get_data_by_jsonpath(size_t* max_lines) {
   
   }
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r410712486
 
 

 ##########
 File path: be/src/exec/json_scanner.cpp
 ##########
 @@ -0,0 +1,485 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+         _jsonValues(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::isEnd() {
+    return _jsonValues->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::getNext() {
+    if (isEnd()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonPathDoc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+                _parseJsonPathFlag = -1;// failed, has none object
+            } else {
+                _parseJsonPathFlag = 1;// success
+            }
+        } else {
+            _parseJsonPathFlag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parseJsonPathFlag = parseJsonPathFromFile(fileMgr, jsonpath_file);
+    } else {
+        _parseJsonPathFlag = 0;
+    }
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parseJsonPathFromFile(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parseJsonPathFromFile Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonPathDoc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parseJsonDoc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr; //
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_jsonDoc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::getDataByJsonPath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonPathDoc["jsonpath"];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* jsonValues = JsonFunctions::get_json_object_simple(value, &_jsonDoc);
+        if (jsonValues == NULL) {
+            return -1;
+        }
+        if (jsonValues->IsArray()) {
+            max_lines = max_lines < jsonValues->Size() ? jsonValues->Size() : max_lines;
+        } else {
+            max_lines = max_lines > 1 ? max_lines : 1;
+        }
+        jmap.emplace(column, jsonValues);
+    }
+    return max_lines;
+}
+
+void JsonReader::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {
+    tuple->set_not_null(slot_desc->null_indicator_offset());
+    void* slot = tuple->get_slot(slot_desc->tuple_offset());
+    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(str_slot->ptr, value, len);
+    str_slot->len = len;
+    return;
+}
+
+Status JsonReader::writeDataToTuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool) {
+    const char *str_value = NULL;
+    uint8_t tmp_buf[128] = {0};
+    int32_t wbytes = 0;
+    switch (value->GetType()) {
+        case rapidjson::Type::kStringType:
+            str_value = value->GetString();
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)str_value, strlen(str_value));
+            break;
+        case rapidjson::Type::kNumberType:
+            if (value->IsUint()) {
+                wbytes = sprintf((char*)tmp_buf, "%u", value->GetUint());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt()) {
+                wbytes = sprintf((char*)tmp_buf, "%d", value->GetInt());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsUint64()) {
+                wbytes = sprintf((char*)tmp_buf, "%lu", value->GetUint64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt64()) {
+                wbytes = sprintf((char*)tmp_buf, "%ld", value->GetInt64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else {
+                wbytes = sprintf((char*)tmp_buf, "%f", value->GetDouble());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            }
+            break;
+        case rapidjson::Type::kFalseType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"false", 5);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"0", 1);
+            break;
+        case rapidjson::Type::kTrueType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"true", 4);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"1", 1);
+            break;
+        default:
+            std::stringstream str_error;
+            str_error << "Invalid JsonType " << value->GetType() << ", Column Name `" << desc->col_name() << "`.";
+            LOG(WARNING) << str_error.str();
+            return Status::RuntimeError(str_error.str());
+    }
+    return Status::OK();
+}
+
+/**
+ * handle input a simple json
+ * For example:
+ *  1. {"doris_data": [{"colunm1":"value1", "colunm2":10}, {"colunm1":"value2", "colunm2":30}]}
+ *  2. {"colunm1":"value1", "colunm2":10}
+ */
+Status JsonReader::handleSimpleJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof) {
+    if (_next_line >= _total_lines) {
+        // generic document
+        RETURN_IF_ERROR(parseJsonDoc(eof));
+        if (*eof) {// handle over all data
+            return Status::OK();
+        }
+        if (_jsonDoc.HasMember("doris_data") && _jsonDoc["doris_data"].IsArray() ) {
+            _total_lines = _jsonDoc["doris_data"].Size();
+        } else {
+            _total_lines = 1;
+        }
+        _next_line = 0;
+    }
+
+    if (_jsonDoc.HasMember("doris_data") && _jsonDoc["doris_data"].IsArray()) {//handle case 1
+        rapidjson::Value& valueArray = _jsonDoc["doris_data"];
+        rapidjson::Value& objectValue = valueArray[_next_line++];// json object
+        for (auto v : slot_descs) {
+            if (objectValue.HasMember(v->col_name().c_str())) {
+                rapidjson::Value& value = objectValue[v->col_name().c_str()];
+                writeDataToTuple(&value, v, tuple, tuple_pool);
+            } else {
+                if (v->is_nullable()) {
+                    tuple->set_null(v->null_indicator_offset());
+                } else  {
+                    std::stringstream str_error;
+                    str_error << "The column `" << v->col_name() << "` is not nullable, but it's not found in jsondata.";
+                    LOG(WARNING) << str_error.str();
+                    return Status::RuntimeError(str_error.str());
 
 Review comment:
   Here you directly return an error, and the entire load job will be failed.
   But sometimes user will set the `max_filter_ratio` to try to skip some error line.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r400083364
 
 

 ##########
 File path: gensrc/thrift/FrontendService.thrift
 ##########
 @@ -130,7 +130,7 @@ struct TSessionState {
   2: required string user
 
   // The user who this session belongs to.
-  3: required i64 connection_id
+  3: required i64 connection_id544
 
 Review comment:
   ```suggestion
     3: required i64 connection_id544
   ```
   spelling mistake?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r409999972
 
 

 ##########
 File path: be/src/exec/json_scanner.h
 ##########
 @@ -0,0 +1,140 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to
+    Status open() override;
+
+    // Get next tuple
+    Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) override;
+
+    // Close this scanner
+    void close() override;
+private:
+    Status open_next_reader();
+private:
+    //const TBrokerScanRangeParams& _params;
+    const std::vector<TBrokerRangeDesc>& _ranges;
+    const std::vector<TNetworkAddress>& _broker_addresses;
+
+    std::string _jsonpath;
+    std::string _jsonpath_file;
+    // std::shared_ptr<rapidjson::Document> _jsonPathDoc;
+
+    // used to hold current StreamLoadPipe
+    std::shared_ptr<StreamLoadPipe> _stream_load_pipe;
+    // Reader
+    JsonReader* _cur_file_reader;
+    int _next_range;
+    bool _cur_file_eof; // is read over?
+    bool _scanner_eof;
+};
+
+
+class JsonDataInternal {
+public:
+    JsonDataInternal(rapidjson::Value* v);
+    ~JsonDataInternal();
+    rapidjson::Value::ConstValueIterator getNext();
+
+private:
+    bool isEnd();
+
+private:
+    rapidjson::Value* _jsonValues;
 
 Review comment:
   Variable names should be kept in lower case with '_'

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] imay commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r400685057
 
 

 ##########
 File path: be/src/exec/json_scanner.h
 ##########
 @@ -0,0 +1,140 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to
+    virtual Status open();
+
+    // Get next tuple
+    virtual Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof);
+
+    // Close this scanner
+    virtual void close();
 
 Review comment:
   ```suggestion
       void close() override;
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] imay commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r400685330
 
 

 ##########
 File path: be/src/exec/json_scanner.cpp
 ##########
 @@ -0,0 +1,479 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(NULL),
 
 Review comment:
   ```suggestion
                             _cur_file_reader(nullptr),
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r410711587
 
 

 ##########
 File path: be/src/exec/json_scanner.cpp
 ##########
 @@ -0,0 +1,485 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+         _jsonValues(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::isEnd() {
+    return _jsonValues->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::getNext() {
+    if (isEnd()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonPathDoc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+                _parseJsonPathFlag = -1;// failed, has none object
+            } else {
+                _parseJsonPathFlag = 1;// success
+            }
+        } else {
+            _parseJsonPathFlag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parseJsonPathFlag = parseJsonPathFromFile(fileMgr, jsonpath_file);
+    } else {
+        _parseJsonPathFlag = 0;
+    }
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parseJsonPathFromFile(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parseJsonPathFromFile Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonPathDoc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parseJsonDoc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr; //
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_jsonDoc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::getDataByJsonPath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonPathDoc["jsonpath"];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* jsonValues = JsonFunctions::get_json_object_simple(value, &_jsonDoc);
+        if (jsonValues == NULL) {
+            return -1;
+        }
+        if (jsonValues->IsArray()) {
+            max_lines = max_lines < jsonValues->Size() ? jsonValues->Size() : max_lines;
+        } else {
+            max_lines = max_lines > 1 ? max_lines : 1;
+        }
+        jmap.emplace(column, jsonValues);
+    }
+    return max_lines;
+}
+
+void JsonReader::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {
+    tuple->set_not_null(slot_desc->null_indicator_offset());
+    void* slot = tuple->get_slot(slot_desc->tuple_offset());
+    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(str_slot->ptr, value, len);
+    str_slot->len = len;
+    return;
+}
+
+Status JsonReader::writeDataToTuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool) {
+    const char *str_value = NULL;
+    uint8_t tmp_buf[128] = {0};
+    int32_t wbytes = 0;
+    switch (value->GetType()) {
+        case rapidjson::Type::kStringType:
+            str_value = value->GetString();
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)str_value, strlen(str_value));
+            break;
+        case rapidjson::Type::kNumberType:
+            if (value->IsUint()) {
+                wbytes = sprintf((char*)tmp_buf, "%u", value->GetUint());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt()) {
+                wbytes = sprintf((char*)tmp_buf, "%d", value->GetInt());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsUint64()) {
+                wbytes = sprintf((char*)tmp_buf, "%lu", value->GetUint64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt64()) {
+                wbytes = sprintf((char*)tmp_buf, "%ld", value->GetInt64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else {
+                wbytes = sprintf((char*)tmp_buf, "%f", value->GetDouble());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            }
+            break;
+        case rapidjson::Type::kFalseType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"false", 5);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"0", 1);
+            break;
+        case rapidjson::Type::kTrueType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"true", 4);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"1", 1);
+            break;
+        default:
+            std::stringstream str_error;
+            str_error << "Invalid JsonType " << value->GetType() << ", Column Name `" << desc->col_name() << "`.";
+            LOG(WARNING) << str_error.str();
+            return Status::RuntimeError(str_error.str());
+    }
+    return Status::OK();
+}
+
+/**
+ * handle input a simple json
+ * For example:
+ *  1. {"doris_data": [{"colunm1":"value1", "colunm2":10}, {"colunm1":"value2", "colunm2":30}]}
+ *  2. {"colunm1":"value1", "colunm2":10}
+ */
+Status JsonReader::handleSimpleJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof) {
+    if (_next_line >= _total_lines) {
+        // generic document
+        RETURN_IF_ERROR(parseJsonDoc(eof));
+        if (*eof) {// handle over all data
+            return Status::OK();
+        }
+        if (_jsonDoc.HasMember("doris_data") && _jsonDoc["doris_data"].IsArray() ) {
 
 Review comment:
   `doris_data` should be defined somewhere as a static member.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] imay commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r400684949
 
 

 ##########
 File path: be/src/exec/json_scanner.h
 ##########
 @@ -0,0 +1,140 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to
+    virtual Status open();
 
 Review comment:
   ```suggestion
       Status open() override;
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r410010003
 
 

 ##########
 File path: be/src/exec/json_scanner.h
 ##########
 @@ -0,0 +1,140 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to
+    Status open() override;
+
+    // Get next tuple
+    Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) override;
+
+    // Close this scanner
+    void close() override;
+private:
+    Status open_next_reader();
+private:
+    //const TBrokerScanRangeParams& _params;
+    const std::vector<TBrokerRangeDesc>& _ranges;
+    const std::vector<TNetworkAddress>& _broker_addresses;
+
+    std::string _jsonpath;
+    std::string _jsonpath_file;
+    // std::shared_ptr<rapidjson::Document> _jsonPathDoc;
+
+    // used to hold current StreamLoadPipe
+    std::shared_ptr<StreamLoadPipe> _stream_load_pipe;
+    // Reader
+    JsonReader* _cur_file_reader;
+    int _next_range;
+    bool _cur_file_eof; // is read over?
+    bool _scanner_eof;
+};
+
+
+class JsonDataInternal {
+public:
+    JsonDataInternal(rapidjson::Value* v);
+    ~JsonDataInternal();
+    rapidjson::Value::ConstValueIterator getNext();
+
+private:
+    bool isEnd();
+
+private:
+    rapidjson::Value* _jsonValues;
 
 Review comment:
   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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r410713975
 
 

 ##########
 File path: be/src/exec/json_scanner.cpp
 ##########
 @@ -0,0 +1,485 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+         _jsonValues(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::isEnd() {
+    return _jsonValues->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::getNext() {
+    if (isEnd()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonPathDoc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+                _parseJsonPathFlag = -1;// failed, has none object
+            } else {
+                _parseJsonPathFlag = 1;// success
+            }
+        } else {
+            _parseJsonPathFlag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parseJsonPathFlag = parseJsonPathFromFile(fileMgr, jsonpath_file);
+    } else {
+        _parseJsonPathFlag = 0;
+    }
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parseJsonPathFromFile(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parseJsonPathFromFile Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonPathDoc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parseJsonDoc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr; //
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_jsonDoc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::getDataByJsonPath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonPathDoc["jsonpath"];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* jsonValues = JsonFunctions::get_json_object_simple(value, &_jsonDoc);
+        if (jsonValues == NULL) {
+            return -1;
+        }
+        if (jsonValues->IsArray()) {
+            max_lines = max_lines < jsonValues->Size() ? jsonValues->Size() : max_lines;
 
 Review comment:
   use `std::max`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] imay commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r400685133
 
 

 ##########
 File path: be/src/exec/json_scanner.h
 ##########
 @@ -0,0 +1,140 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to
+    virtual Status open();
+
+    // Get next tuple
+    virtual Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof);
 
 Review comment:
   ```suggestion
       Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) override;
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r410011876
 
 

 ##########
 File path: be/src/exec/json_scanner.cpp
 ##########
 @@ -0,0 +1,485 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+         _jsonValues(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::isEnd() {
+    return _jsonValues->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::getNext() {
+    if (isEnd()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonPathDoc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+                _parseJsonPathFlag = -1;// failed, has none object
+            } else {
+                _parseJsonPathFlag = 1;// success
+            }
+        } else {
+            _parseJsonPathFlag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parseJsonPathFlag = parseJsonPathFromFile(fileMgr, jsonpath_file);
+    } else {
+        _parseJsonPathFlag = 0;
+    }
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parseJsonPathFromFile(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parseJsonPathFromFile Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonPathDoc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parseJsonDoc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr; //
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_jsonDoc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::getDataByJsonPath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonPathDoc["jsonpath"];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* jsonValues = JsonFunctions::get_json_object_simple(value, &_jsonDoc);
+        if (jsonValues == NULL) {
+            return -1;
+        }
+        if (jsonValues->IsArray()) {
+            max_lines = max_lines < jsonValues->Size() ? jsonValues->Size() : max_lines;
+        } else {
+            max_lines = max_lines > 1 ? max_lines : 1;
+        }
+        jmap.emplace(column, jsonValues);
+    }
+    return max_lines;
+}
+
+void JsonReader::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {
+    tuple->set_not_null(slot_desc->null_indicator_offset());
+    void* slot = tuple->get_slot(slot_desc->tuple_offset());
+    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(str_slot->ptr, value, len);
+    str_slot->len = len;
+    return;
+}
+
+Status JsonReader::writeDataToTuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool) {
+    const char *str_value = NULL;
+    uint8_t tmp_buf[128] = {0};
+    int32_t wbytes = 0;
+    switch (value->GetType()) {
+        case rapidjson::Type::kStringType:
+            str_value = value->GetString();
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)str_value, strlen(str_value));
+            break;
+        case rapidjson::Type::kNumberType:
+            if (value->IsUint()) {
+                wbytes = sprintf((char*)tmp_buf, "%u", value->GetUint());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt()) {
+                wbytes = sprintf((char*)tmp_buf, "%d", value->GetInt());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsUint64()) {
+                wbytes = sprintf((char*)tmp_buf, "%lu", value->GetUint64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt64()) {
+                wbytes = sprintf((char*)tmp_buf, "%ld", value->GetInt64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else {
+                wbytes = sprintf((char*)tmp_buf, "%f", value->GetDouble());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            }
+            break;
+        case rapidjson::Type::kFalseType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"false", 5);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"0", 1);
+            break;
+        case rapidjson::Type::kTrueType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"true", 4);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"1", 1);
+            break;
+        default:
+            std::stringstream str_error;
+            str_error << "Invalid JsonType " << value->GetType() << ", Column Name `" << desc->col_name() << "`.";
+            LOG(WARNING) << str_error.str();
+            return Status::RuntimeError(str_error.str());
+    }
+    return Status::OK();
+}
+
+/**
+ * handle input a simple json
+ * For example:
+ *  1. {"doris_data": [{"colunm1":"value1", "colunm2":10}, {"colunm1":"value2", "colunm2":30}]}
+ *  2. {"colunm1":"value1", "colunm2":10}
+ */
+Status JsonReader::handleSimpleJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof) {
+    if (_next_line >= _total_lines) {
+        // generic document
+        RETURN_IF_ERROR(parseJsonDoc(eof));
+        if (*eof) {// handle over all data
+            return Status::OK();
+        }
+        if (_jsonDoc.HasMember("doris_data") && _jsonDoc["doris_data"].IsArray() ) {
+            _total_lines = _jsonDoc["doris_data"].Size();
+        } else {
+            _total_lines = 1;
+        }
+        _next_line = 0;
+    }
+
+    if (_jsonDoc.HasMember("doris_data") && _jsonDoc["doris_data"].IsArray()) {//handle case 1
+        rapidjson::Value& valueArray = _jsonDoc["doris_data"];
+        rapidjson::Value& objectValue = valueArray[_next_line++];// json object
+        for (auto v : slot_descs) {
+            if (objectValue.HasMember(v->col_name().c_str())) {
+                rapidjson::Value& value = objectValue[v->col_name().c_str()];
+                writeDataToTuple(&value, v, tuple, tuple_pool);
 
 Review comment:
   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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r409999635
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java
 ##########
 @@ -270,8 +281,20 @@ protected void setOptional(CreateRoutineLoadStmt stmt) throws UserException {
         if (stmt.getMaxBatchSize() != -1) {
             this.maxBatchSizeBytes = stmt.getMaxBatchSize();
         }
+        if (stmt.getFormat().equals("json")) {
 
 Review comment:
   stmt.getFormat() may null in csv format

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] Youngwb commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
Youngwb commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r410143344
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java
 ##########
 @@ -270,8 +281,20 @@ protected void setOptional(CreateRoutineLoadStmt stmt) throws UserException {
         if (stmt.getMaxBatchSize() != -1) {
             this.maxBatchSizeBytes = stmt.getMaxBatchSize();
         }
+        if (stmt.getFormat().equals("json")) {
 
 Review comment:
   I think format to null is not handled in CreateRoutineLoadStmt

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r409998644
 
 

 ##########
 File path: be/src/exec/json_scanner.cpp
 ##########
 @@ -0,0 +1,485 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+         _jsonValues(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::isEnd() {
+    return _jsonValues->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::getNext() {
+    if (isEnd()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonPathDoc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+                _parseJsonPathFlag = -1;// failed, has none object
+            } else {
+                _parseJsonPathFlag = 1;// success
+            }
+        } else {
+            _parseJsonPathFlag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parseJsonPathFlag = parseJsonPathFromFile(fileMgr, jsonpath_file);
+    } else {
+        _parseJsonPathFlag = 0;
+    }
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parseJsonPathFromFile(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parseJsonPathFromFile Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonPathDoc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parseJsonDoc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr; //
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_jsonDoc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::getDataByJsonPath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonPathDoc["jsonpath"];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* jsonValues = JsonFunctions::get_json_object_simple(value, &_jsonDoc);
+        if (jsonValues == NULL) {
+            return -1;
+        }
+        if (jsonValues->IsArray()) {
+            max_lines = max_lines < jsonValues->Size() ? jsonValues->Size() : max_lines;
+        } else {
+            max_lines = max_lines > 1 ? max_lines : 1;
+        }
+        jmap.emplace(column, jsonValues);
+    }
+    return max_lines;
+}
+
+void JsonReader::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {
+    tuple->set_not_null(slot_desc->null_indicator_offset());
+    void* slot = tuple->get_slot(slot_desc->tuple_offset());
+    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(str_slot->ptr, value, len);
+    str_slot->len = len;
+    return;
+}
+
+Status JsonReader::writeDataToTuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool) {
+    const char *str_value = NULL;
+    uint8_t tmp_buf[128] = {0};
+    int32_t wbytes = 0;
+    switch (value->GetType()) {
+        case rapidjson::Type::kStringType:
+            str_value = value->GetString();
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)str_value, strlen(str_value));
+            break;
+        case rapidjson::Type::kNumberType:
+            if (value->IsUint()) {
+                wbytes = sprintf((char*)tmp_buf, "%u", value->GetUint());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt()) {
+                wbytes = sprintf((char*)tmp_buf, "%d", value->GetInt());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsUint64()) {
+                wbytes = sprintf((char*)tmp_buf, "%lu", value->GetUint64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt64()) {
+                wbytes = sprintf((char*)tmp_buf, "%ld", value->GetInt64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else {
+                wbytes = sprintf((char*)tmp_buf, "%f", value->GetDouble());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            }
+            break;
+        case rapidjson::Type::kFalseType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"false", 5);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"0", 1);
+            break;
+        case rapidjson::Type::kTrueType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"true", 4);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"1", 1);
+            break;
+        default:
 
 Review comment:
   kNullType is need 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r403421506
 
 

 ##########
 File path: be/test/exprs/json_function_test.cpp
 ##########
 @@ -186,10 +186,57 @@ TEST_F(JsonFunctionTest, special_char)
     ASSERT_EQ(std::string(res3->GetString()), "v1");
 }
 
+TEST_F(JsonFunctionTest, json_path1)
+{
+    std::string json_raw_data("{\"jsonpath\":[{\"key\":\"colname1\",\"type\":\"string\",\"value\":\"$.store.book.category\"},{\"key\":\"colname2\",\"type\":\"string\",\"value\":\"$.store.book.author\"},{\"key\":\"colname3\",\"type\":\"string\",\"value\":\"$.store.book.title\"},{\"key\":\"colname4\",\"type\":\"float\",\"value\":\"$.store.book.price\"}],\"store\":{\"book\":[{\"category\":\"reference\",\"author\":\"NigelRees\",\"title\":\"SayingsoftheCentury\",\"price\":8.95},{\"category\":\"fiction\",\"author\":\"EvelynWaugh\",\"title\":\"SwordofHonour\",\"price\":12.99}],\"bicycle\":{\"color\":\"red\",\"price\":19.95}},\"expensive\":10}");
+    rapidjson::Document jsonDoc;
+    ASSERT_FALSE(jsonDoc.Parse(json_raw_data.c_str()).HasParseError());
+    if (!jsonDoc.HasMember("jsonpath") || !jsonDoc["jsonpath"].IsArray()) {
+        std::cout << jsonDoc.HasMember("jsonpath");
+        std::cout << jsonDoc["jsonpath"].IsArray();
+        ASSERT_TRUE(false);
+    }
+    const rapidjson::Value& arr = jsonDoc["jsonpath"];
+    enum JsonFunctionType type;
+    rapidjson::Document jsonDoc2;
+    for (int i = 0; i < arr.Size(); i++) {
+        const rapidjson::Value& path = arr[i];
+        if (!path["type"].IsString() || !path["key"].IsString() || !path["value"].IsString()) {
+            ASSERT_TRUE(false);
+        }
+        const char *c_type = path["type"].GetString();
+        if (!strcmp(c_type, "integer")) {
+            type = JSON_FUN_INT;
+        } else if (!strcmp(c_type, "float")) {
+            type = JSON_FUN_DOUBLE;
+        } else if (!strcmp(c_type, "string")) {
+            type = JSON_FUN_STRING;
+        } else
+        {
+            ASSERT_TRUE(false);
+        }
+        rapidjson::Value* res3 = JsonFunctions::get_json_object(nullptr, json_raw_data, path["value"].GetString(), type, &jsonDoc2);
+        rapidjson::Value& value = *res3;
+        if (value.IsArray()) {
+            std::cout << "Array:" << path["value"].GetString() << std::endl;
+
+            for (int i = 0; i < res3->Size(); i++) {
+                std::cout << (res3 + i)->GetString()  << " ";
+            }
+            std::cout << std::endl;
+        }
+    }
+}
+
 }
 
 int main(int argc, char** argv) {
-    std::string conffile = std::string(getenv("DORIS_HOME")) + "/conf/be.conf";
+    std::string home(getenv("DORIS_HOME"));
+    if (home.empty()) {
+        home = ".";
+    }
+    std::string conffile = home + "/conf/be.conf";
+    std::cout<<conffile<<std::endl;
 
 Review comment:
   I deleted it

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r400089614
 
 

 ##########
 File path: gensrc/thrift/BackendService.thrift
 ##########
 @@ -54,11 +55,12 @@ struct TRoutineLoadTask {
     6: optional string db
     7: optional string tbl
     8: optional string label
-    9: optional i64 max_interval_s
-    10: optional i64 max_batch_rows
-    11: optional i64 max_batch_size
-    12: optional TKafkaLoadInfo kafka_load_info
-    13: optional PaloInternalService.TExecPlanFragmentParams params
+    9: optional PlanNodes.TFileFormatType format
 
 Review comment:
   New field in thrift should put at end

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r410713063
 
 

 ##########
 File path: be/src/exec/json_scanner.h
 ##########
 @@ -0,0 +1,140 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to
+    Status open() override;
+
+    // Get next tuple
+    Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) override;
+
+    // Close this scanner
+    void close() override;
+private:
+    Status open_next_reader();
+private:
+    //const TBrokerScanRangeParams& _params;
+    const std::vector<TBrokerRangeDesc>& _ranges;
+    const std::vector<TNetworkAddress>& _broker_addresses;
+
+    std::string _jsonpath;
+    std::string _jsonpath_file;
+    // std::shared_ptr<rapidjson::Document> _jsonPathDoc;
+
+    // used to hold current StreamLoadPipe
+    std::shared_ptr<StreamLoadPipe> _stream_load_pipe;
+    // Reader
+    JsonReader* _cur_file_reader;
+    int _next_range;
+    bool _cur_file_eof; // is read over?
+    bool _scanner_eof;
+};
+
+
+class JsonDataInternal {
+public:
+    JsonDataInternal(rapidjson::Value* v);
+    ~JsonDataInternal();
+    rapidjson::Value::ConstValueIterator getNext();
+
+private:
+    bool isEnd();
+
+private:
+    rapidjson::Value* _jsonValues;
+    rapidjson::Value::ConstValueIterator  _iterator;
+};
+
+class JsonReader {
+public:
+    JsonReader(SmallFileMgr *fileMgr, RuntimeProfile* profile, FileReader* file_reader,
+            std::string& jsonpath, std::string& jsonpath_file);
+    ~JsonReader();
+    Status read(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof);
+
+private:
+    void fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len);
+    size_t getDataByJsonPath();
+    int parseJsonPathFromFile(SmallFileMgr *smallFileMgr, std::string& fileinfo);
+    Status parseJsonDoc(bool *eof);
+    Status handleSimpleJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof);
+    Status handleComplexJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof);
+    Status writeDataToTuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool);
+    void close();
+
+private:
+    int _next_line;
+    int _total_lines;
+    RuntimeProfile* _profile;
+    FileReader*_file_reader;
+    bool _closed;
+    /**
+     * _parseJsonPathFlag == 1, jsonpath is valid
+     * _parseJsonPathFlag == 0, jsonpath is empty, default
+     * _parseJsonPathFlag == -1, jsonpath parse is error, it will return ERROR
+     */
+    int _parseJsonPathFlag;
+    RuntimeProfile::Counter* _bytes_read_counter;
+    RuntimeProfile::Counter* _read_timer;
+    rapidjson::Document _jsonPathDoc;
+    rapidjson::Document _jsonDoc;
+    std::map<std::string, JsonDataInternal> jmap;
 
 Review comment:
   ```suggestion
       std::unorderd_map<std::string, JsonDataInternal> _jmap;
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r409999207
 
 

 ##########
 File path: be/src/exec/json_scanner.cpp
 ##########
 @@ -0,0 +1,485 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+         _jsonValues(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::isEnd() {
+    return _jsonValues->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::getNext() {
+    if (isEnd()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonPathDoc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+                _parseJsonPathFlag = -1;// failed, has none object
+            } else {
+                _parseJsonPathFlag = 1;// success
+            }
+        } else {
+            _parseJsonPathFlag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parseJsonPathFlag = parseJsonPathFromFile(fileMgr, jsonpath_file);
+    } else {
+        _parseJsonPathFlag = 0;
+    }
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parseJsonPathFromFile(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parseJsonPathFromFile Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonPathDoc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parseJsonDoc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr; //
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_jsonDoc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::getDataByJsonPath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonPathDoc["jsonpath"];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* jsonValues = JsonFunctions::get_json_object_simple(value, &_jsonDoc);
+        if (jsonValues == NULL) {
+            return -1;
+        }
+        if (jsonValues->IsArray()) {
+            max_lines = max_lines < jsonValues->Size() ? jsonValues->Size() : max_lines;
+        } else {
+            max_lines = max_lines > 1 ? max_lines : 1;
+        }
+        jmap.emplace(column, jsonValues);
+    }
+    return max_lines;
+}
+
+void JsonReader::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {
+    tuple->set_not_null(slot_desc->null_indicator_offset());
+    void* slot = tuple->get_slot(slot_desc->tuple_offset());
+    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(str_slot->ptr, value, len);
+    str_slot->len = len;
+    return;
+}
+
+Status JsonReader::writeDataToTuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool) {
+    const char *str_value = NULL;
+    uint8_t tmp_buf[128] = {0};
+    int32_t wbytes = 0;
+    switch (value->GetType()) {
+        case rapidjson::Type::kStringType:
+            str_value = value->GetString();
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)str_value, strlen(str_value));
+            break;
+        case rapidjson::Type::kNumberType:
+            if (value->IsUint()) {
+                wbytes = sprintf((char*)tmp_buf, "%u", value->GetUint());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt()) {
+                wbytes = sprintf((char*)tmp_buf, "%d", value->GetInt());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsUint64()) {
+                wbytes = sprintf((char*)tmp_buf, "%lu", value->GetUint64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt64()) {
+                wbytes = sprintf((char*)tmp_buf, "%ld", value->GetInt64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else {
+                wbytes = sprintf((char*)tmp_buf, "%f", value->GetDouble());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            }
+            break;
+        case rapidjson::Type::kFalseType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"false", 5);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"0", 1);
+            break;
+        case rapidjson::Type::kTrueType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"true", 4);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"1", 1);
+            break;
+        default:
+            std::stringstream str_error;
+            str_error << "Invalid JsonType " << value->GetType() << ", Column Name `" << desc->col_name() << "`.";
+            LOG(WARNING) << str_error.str();
+            return Status::RuntimeError(str_error.str());
+    }
+    return Status::OK();
+}
+
+/**
+ * handle input a simple json
+ * For example:
+ *  1. {"doris_data": [{"colunm1":"value1", "colunm2":10}, {"colunm1":"value2", "colunm2":30}]}
+ *  2. {"colunm1":"value1", "colunm2":10}
+ */
+Status JsonReader::handleSimpleJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof) {
+    if (_next_line >= _total_lines) {
+        // generic document
+        RETURN_IF_ERROR(parseJsonDoc(eof));
+        if (*eof) {// handle over all data
+            return Status::OK();
+        }
+        if (_jsonDoc.HasMember("doris_data") && _jsonDoc["doris_data"].IsArray() ) {
+            _total_lines = _jsonDoc["doris_data"].Size();
+        } else {
+            _total_lines = 1;
+        }
+        _next_line = 0;
+    }
+
+    if (_jsonDoc.HasMember("doris_data") && _jsonDoc["doris_data"].IsArray()) {//handle case 1
+        rapidjson::Value& valueArray = _jsonDoc["doris_data"];
+        rapidjson::Value& objectValue = valueArray[_next_line++];// json object
+        for (auto v : slot_descs) {
+            if (objectValue.HasMember(v->col_name().c_str())) {
+                rapidjson::Value& value = objectValue[v->col_name().c_str()];
+                writeDataToTuple(&value, v, tuple, tuple_pool);
+            } else {
 
 Review comment:
   If the column in the load column is not in the json data, you should report an error

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r403421443
 
 

 ##########
 File path: docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/ROUTINE LOAD.md
 ##########
 @@ -130,15 +130,23 @@ under the License.
             采样窗口内,允许的最大错误行数。必须大于等于0。默认是 0,即不允许有错误行。
             采样窗口为 max_batch_rows * 10。即如果在采样窗口内,错误行数大于 max_error_number,则会导致例行作业被暂停,需要人工介入检查数据质量问题。
             被 where 条件过滤掉的行不算错误行。
-        
+
         4. strict_mode
 
             是否开启严格模式,默认为开启。如果开启后,非空原始数据的列类型变换如果结果为 NULL,则会被过滤。指定方式为 "strict_mode" = "true"
 
-        5. timezone
-            
+        5. `timezone`
+
             指定导入作业所使用的时区。默认为使用 Session 的 timezone 参数。该参数会影响所有导入涉及的和时区有关的函数结果。
 
+        6. `format`
 
 Review comment:
   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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] Youngwb commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
Youngwb commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r410142198
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java
 ##########
 @@ -270,8 +281,20 @@ protected void setOptional(CreateRoutineLoadStmt stmt) throws UserException {
         if (stmt.getMaxBatchSize() != -1) {
             this.maxBatchSizeBytes = stmt.getMaxBatchSize();
         }
+        if (stmt.getFormat().equals("json")) {
 
 Review comment:
   ![image](https://user-images.githubusercontent.com/9495145/79560947-195deb00-80db-11ea-8ce4-98841e15b72f.png)
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r410710574
 
 

 ##########
 File path: docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/ROUTINE LOAD.md
 ##########
 @@ -301,6 +309,84 @@ under the License.
             "property.client.id" = "my_client_id"
         );
 
+    4. 为 example_db 的 example_tbl 创建一个名为 test1 的 Kafka 例行导入任务,导入的简单json数据。
+        1)数据样例, doris_data为固定关键字
+        {
+            "doris_data":[
+                {"category":"a9jadhx","author":"test","price":895},
+                {"category":"axdfa1","author":"EvelynWaugh","price":1299}
+            ]
+        }
+        2) 创建任务,可不设置jsonpath或者jsonpath_file
+        CREATE ROUTINE LOAD example_db.test1 ON example_tbl
+        COLUMNS(category, author, price)
+        PROPERTIES
+        (
+            "desired_concurrent_number"="3",
+            "max_batch_interval" = "20",
+            "max_batch_rows" = "300000",
+            "max_batch_size" = "209715200",
+            "strict_mode" = "false",
+            "format" = "json"
+        )
+        FROM KAFKA
+        (
+            "kafka_broker_list" = "broker1:9092,broker2:9092,broker3:9092",
+            "kafka_topic" = "my_topic",
+            "kafka_partitions" = "0,1,2",
+            "kafka_offsets" = "0,0,0"
+        );
+
+    5. 通过jsonpath参数,为 example_db 的 example_tbl 创建一个名为 test1 的 Kafka 例行导入任务,导入的数据格式为json。
+
+        CREATE ROUTINE LOAD example_db.test1 ON example_tbl
+        COLUMNS(category, author, price)
+        PROPERTIES
+        (
+            "desired_concurrent_number"="3",
+            "max_batch_interval" = "20",
+            "max_batch_rows" = "300000",
+            "max_batch_size" = "209715200",
+            "strict_mode" = "false",
+            "format" = "json",
+            "jsonpath" = "{\"jsonpath\":[{\"column\":\"category\",\"value\":\"$.store.book.category\"},{\"column\":\"author\",\"value\":\"$.store.book.author\"},,{\"column\":\"price\",\"value\":\"$.store.book.price\"}]}"
+        )
+        FROM KAFKA
+        (
+            "kafka_broker_list" = "broker1:9092,broker2:9092,broker3:9092",
+            "kafka_topic" = "my_topic",
+            "kafka_partitions" = "0,1,2",
+            "kafka_offsets" = "0,0,0"
+        );
+
+    6. 通过jsonpath_file参数, 为 example_db 的 example_tbl 创建一个名为 test1 的 Kafka 例行导入任务,导入的数据格式为json。
+        1)通过create file导入jsonpath文件到Doris集群中,然后通过show file查看文件id以及md5, 例如:
+        mysql> show file;
+        +-------+--------------------------+---------+-----------------+----------+-----------+----------------------------------+
+        | Id    | DbName                   | Catalog | FileName        | FileSize | IsContent | MD5                              |
+        +-------+--------------------------+---------+-----------------+----------+-----------+----------------------------------+
+        | 43017 | default_cluster:mediavad | kafka   | myjsonpath.json | 215      | true      | 3f3ab257be8a422e0044abe5ed51d410 |
+        +-------+--------------------------+---------+-----------------+----------+-----------+----------------------------------+
+        2)创建导入任务
+        CREATE ROUTINE LOAD example_db.test1 ON example_tbl
+        COLUMNS(category, author, price)
+        PROPERTIES
+        (
+            "desired_concurrent_number"="3",
+            "max_batch_interval" = "20",
+            "max_batch_rows" = "300000",
+            "max_batch_size" = "209715200",
+            "strict_mode" = "false",
+            "format" = "json",
+            "jsonpath_file" = "43017:3f3ab257be8a422e0044abe5ed51d410"
 
 Review comment:
   should be:
   ```suggestion
               "jsonpath_file" = "FILE:myjsonpath.json"
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r410010257
 
 

 ##########
 File path: fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java
 ##########
 @@ -270,8 +281,20 @@ protected void setOptional(CreateRoutineLoadStmt stmt) throws UserException {
         if (stmt.getMaxBatchSize() != -1) {
             this.maxBatchSizeBytes = stmt.getMaxBatchSize();
         }
+        if (stmt.getFormat().equals("json")) {
 
 Review comment:
   No, the class field `format` is initial "" in CreateRoutineLoadStmt.java, then it is safely

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r403421449
 
 

 ##########
 File path: docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/ROUTINE LOAD.md
 ##########
 @@ -130,15 +130,23 @@ under the License.
             采样窗口内,允许的最大错误行数。必须大于等于0。默认是 0,即不允许有错误行。
             采样窗口为 max_batch_rows * 10。即如果在采样窗口内,错误行数大于 max_error_number,则会导致例行作业被暂停,需要人工介入检查数据质量问题。
             被 where 条件过滤掉的行不算错误行。
-        
+
         4. strict_mode
 
             是否开启严格模式,默认为开启。如果开启后,非空原始数据的列类型变换如果结果为 NULL,则会被过滤。指定方式为 "strict_mode" = "true"
 
-        5. timezone
-            
+        5. `timezone`
+
             指定导入作业所使用的时区。默认为使用 Session 的 timezone 参数。该参数会影响所有导入涉及的和时区有关的函数结果。
 
+        6. `format`
+
+            指定导入数据格式,默认是csv,支持json格式。
+
+        7. `jsonpath`、`jsonpath_file`
 
 Review comment:
   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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r403421441
 
 

 ##########
 File path: docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/ROUTINE LOAD.md
 ##########
 @@ -130,15 +130,23 @@ under the License.
             采样窗口内,允许的最大错误行数。必须大于等于0。默认是 0,即不允许有错误行。
             采样窗口为 max_batch_rows * 10。即如果在采样窗口内,错误行数大于 max_error_number,则会导致例行作业被暂停,需要人工介入检查数据质量问题。
             被 where 条件过滤掉的行不算错误行。
-        
+
         4. strict_mode
 
             是否开启严格模式,默认为开启。如果开启后,非空原始数据的列类型变换如果结果为 NULL,则会被过滤。指定方式为 "strict_mode" = "true"
 
-        5. timezone
-            
+        5. `timezone`
 
 Review comment:
   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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r400086294
 
 

 ##########
 File path: docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/ROUTINE LOAD.md
 ##########
 @@ -130,15 +130,23 @@ under the License.
             采样窗口内,允许的最大错误行数。必须大于等于0。默认是 0,即不允许有错误行。
             采样窗口为 max_batch_rows * 10。即如果在采样窗口内,错误行数大于 max_error_number,则会导致例行作业被暂停,需要人工介入检查数据质量问题。
             被 where 条件过滤掉的行不算错误行。
-        
+
         4. strict_mode
 
             是否开启严格模式,默认为开启。如果开启后,非空原始数据的列类型变换如果结果为 NULL,则会被过滤。指定方式为 "strict_mode" = "true"
 
-        5. timezone
-            
+        5. `timezone`
+
             指定导入作业所使用的时区。默认为使用 Session 的 timezone 参数。该参数会影响所有导入涉及的和时区有关的函数结果。
 
+        6. `format`
+
+            指定导入数据格式,默认是csv,支持json格式。
+
+        7. `jsonpath`、`jsonpath_file`
 
 Review comment:
   ```suggestion
           7. jsonpath、jsonpath_file
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r410010995
 
 

 ##########
 File path: be/src/exec/json_scanner.cpp
 ##########
 @@ -0,0 +1,485 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+         _jsonValues(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::isEnd() {
+    return _jsonValues->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::getNext() {
+    if (isEnd()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonPathDoc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+                _parseJsonPathFlag = -1;// failed, has none object
+            } else {
+                _parseJsonPathFlag = 1;// success
+            }
+        } else {
+            _parseJsonPathFlag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parseJsonPathFlag = parseJsonPathFromFile(fileMgr, jsonpath_file);
+    } else {
+        _parseJsonPathFlag = 0;
+    }
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parseJsonPathFromFile(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parseJsonPathFromFile Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonPathDoc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parseJsonDoc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr; //
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_jsonDoc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::getDataByJsonPath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonPathDoc["jsonpath"];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* jsonValues = JsonFunctions::get_json_object_simple(value, &_jsonDoc);
+        if (jsonValues == NULL) {
+            return -1;
+        }
+        if (jsonValues->IsArray()) {
+            max_lines = max_lines < jsonValues->Size() ? jsonValues->Size() : max_lines;
+        } else {
+            max_lines = max_lines > 1 ? max_lines : 1;
+        }
+        jmap.emplace(column, jsonValues);
+    }
+    return max_lines;
+}
+
+void JsonReader::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {
+    tuple->set_not_null(slot_desc->null_indicator_offset());
+    void* slot = tuple->get_slot(slot_desc->tuple_offset());
+    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(str_slot->ptr, value, len);
+    str_slot->len = len;
+    return;
+}
+
+Status JsonReader::writeDataToTuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool) {
+    const char *str_value = NULL;
+    uint8_t tmp_buf[128] = {0};
+    int32_t wbytes = 0;
+    switch (value->GetType()) {
+        case rapidjson::Type::kStringType:
+            str_value = value->GetString();
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)str_value, strlen(str_value));
+            break;
+        case rapidjson::Type::kNumberType:
+            if (value->IsUint()) {
+                wbytes = sprintf((char*)tmp_buf, "%u", value->GetUint());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt()) {
+                wbytes = sprintf((char*)tmp_buf, "%d", value->GetInt());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsUint64()) {
+                wbytes = sprintf((char*)tmp_buf, "%lu", value->GetUint64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt64()) {
+                wbytes = sprintf((char*)tmp_buf, "%ld", value->GetInt64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else {
+                wbytes = sprintf((char*)tmp_buf, "%f", value->GetDouble());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            }
+            break;
+        case rapidjson::Type::kFalseType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"false", 5);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"0", 1);
+            break;
+        case rapidjson::Type::kTrueType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"true", 4);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"1", 1);
+            break;
+        default:
+            std::stringstream str_error;
+            str_error << "Invalid JsonType " << value->GetType() << ", Column Name `" << desc->col_name() << "`.";
+            LOG(WARNING) << str_error.str();
+            return Status::RuntimeError(str_error.str());
+    }
+    return Status::OK();
+}
+
+/**
+ * handle input a simple json
+ * For example:
+ *  1. {"doris_data": [{"colunm1":"value1", "colunm2":10}, {"colunm1":"value2", "colunm2":30}]}
+ *  2. {"colunm1":"value1", "colunm2":10}
+ */
+Status JsonReader::handleSimpleJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof) {
+    if (_next_line >= _total_lines) {
+        // generic document
+        RETURN_IF_ERROR(parseJsonDoc(eof));
+        if (*eof) {// handle over all data
+            return Status::OK();
+        }
+        if (_jsonDoc.HasMember("doris_data") && _jsonDoc["doris_data"].IsArray() ) {
+            _total_lines = _jsonDoc["doris_data"].Size();
+        } else {
+            _total_lines = 1;
+        }
+        _next_line = 0;
+    }
+
+    if (_jsonDoc.HasMember("doris_data") && _jsonDoc["doris_data"].IsArray()) {//handle case 1
+        rapidjson::Value& valueArray = _jsonDoc["doris_data"];
+        rapidjson::Value& objectValue = valueArray[_next_line++];// json object
+        for (auto v : slot_descs) {
+            if (objectValue.HasMember(v->col_name().c_str())) {
+                rapidjson::Value& value = objectValue[v->col_name().c_str()];
+                writeDataToTuple(&value, v, tuple, tuple_pool);
+            } else {
 
 Review comment:
   The logic is in `else` switch

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] imay commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r400686128
 
 

 ##########
 File path: be/src/exec/json_scanner.cpp
 ##########
 @@ -0,0 +1,479 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(NULL),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+            _cur_file_reader = nullptr;
+        } else {
+            delete _cur_file_reader;
+            _cur_file_reader = nullptr;
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+            _cur_file_reader = nullptr;
+        } else {
+            delete _cur_file_reader;
+            _cur_file_reader = nullptr;
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+         _jsonValues(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::isEnd() {
+    return _jsonValues->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::getNext() {
+    if (isEnd()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _profile(profile),
+            _file_reader(file_reader) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonPathDoc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+                _parseJsonPathFlag = -1;// failed, has none object
+            } else {
+                _parseJsonPathFlag = 1;// success
+            }
+        } else {
+            _parseJsonPathFlag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
 
 Review comment:
   why not parse jsonpath in frontend, and pass the json path to backend?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r403421428
 
 

 ##########
 File path: gensrc/thrift/FrontendService.thrift
 ##########
 @@ -130,7 +130,7 @@ struct TSessionState {
   2: required string user
 
   // The user who this session belongs to.
-  3: required i64 connection_id
+  3: required i64 connection_id544
 
 Review comment:
   Yes, i changed it

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r403421574
 
 

 ##########
 File path: be/src/exec/json_scanner.h
 ##########
 @@ -0,0 +1,140 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to
+    virtual Status open();
+
+    // Get next tuple
+    virtual Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof);
+
+    // Close this scanner
+    virtual void close();
 
 Review comment:
   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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r403421758
 
 

 ##########
 File path: be/src/exec/json_scanner.cpp
 ##########
 @@ -0,0 +1,479 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(NULL),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+            _cur_file_reader = nullptr;
+        } else {
+            delete _cur_file_reader;
+            _cur_file_reader = nullptr;
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+            _cur_file_reader = nullptr;
+        } else {
+            delete _cur_file_reader;
+            _cur_file_reader = nullptr;
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+         _jsonValues(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::isEnd() {
+    return _jsonValues->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::getNext() {
+    if (isEnd()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _profile(profile),
+            _file_reader(file_reader) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonPathDoc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+                _parseJsonPathFlag = -1;// failed, has none object
+            } else {
+                _parseJsonPathFlag = 1;// success
+            }
+        } else {
+            _parseJsonPathFlag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
 
 Review comment:
   I think paese jsonpath in backend, it's easy

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r403421549
 
 

 ##########
 File path: be/src/exec/broker_reader.h
 ##########
 @@ -52,6 +52,7 @@ class BrokerReader : public FileReader {
     // Read 
     virtual Status read(uint8_t* buf, size_t* buf_len, bool* eof) override;
     virtual Status readat(int64_t position, int64_t nbytes, int64_t* bytes_read, void* out) override;
+    virtual Status read(uint8_t** buf, size_t *length) override;
 
 Review comment:
   It can read a total message.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r403421556
 
 

 ##########
 File path: be/src/exec/json_scanner.h
 ##########
 @@ -0,0 +1,140 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to
+    virtual Status open();
 
 Review comment:
   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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] imay commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
imay commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r400682774
 
 

 ##########
 File path: be/src/exec/broker_reader.h
 ##########
 @@ -52,6 +52,7 @@ class BrokerReader : public FileReader {
     // Read 
     virtual Status read(uint8_t* buf, size_t* buf_len, bool* eof) override;
     virtual Status readat(int64_t position, int64_t nbytes, int64_t* bytes_read, void* out) override;
+    virtual Status read(uint8_t** buf, size_t *length) override;
 
 Review comment:
   What's the reason for this new interface? Why not use the already existed interface?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r400087109
 
 

 ##########
 File path: be/test/exprs/json_function_test.cpp
 ##########
 @@ -186,10 +186,57 @@ TEST_F(JsonFunctionTest, special_char)
     ASSERT_EQ(std::string(res3->GetString()), "v1");
 }
 
+TEST_F(JsonFunctionTest, json_path1)
+{
+    std::string json_raw_data("{\"jsonpath\":[{\"key\":\"colname1\",\"type\":\"string\",\"value\":\"$.store.book.category\"},{\"key\":\"colname2\",\"type\":\"string\",\"value\":\"$.store.book.author\"},{\"key\":\"colname3\",\"type\":\"string\",\"value\":\"$.store.book.title\"},{\"key\":\"colname4\",\"type\":\"float\",\"value\":\"$.store.book.price\"}],\"store\":{\"book\":[{\"category\":\"reference\",\"author\":\"NigelRees\",\"title\":\"SayingsoftheCentury\",\"price\":8.95},{\"category\":\"fiction\",\"author\":\"EvelynWaugh\",\"title\":\"SwordofHonour\",\"price\":12.99}],\"bicycle\":{\"color\":\"red\",\"price\":19.95}},\"expensive\":10}");
+    rapidjson::Document jsonDoc;
+    ASSERT_FALSE(jsonDoc.Parse(json_raw_data.c_str()).HasParseError());
+    if (!jsonDoc.HasMember("jsonpath") || !jsonDoc["jsonpath"].IsArray()) {
+        std::cout << jsonDoc.HasMember("jsonpath");
+        std::cout << jsonDoc["jsonpath"].IsArray();
+        ASSERT_TRUE(false);
+    }
+    const rapidjson::Value& arr = jsonDoc["jsonpath"];
+    enum JsonFunctionType type;
+    rapidjson::Document jsonDoc2;
+    for (int i = 0; i < arr.Size(); i++) {
+        const rapidjson::Value& path = arr[i];
+        if (!path["type"].IsString() || !path["key"].IsString() || !path["value"].IsString()) {
+            ASSERT_TRUE(false);
+        }
+        const char *c_type = path["type"].GetString();
+        if (!strcmp(c_type, "integer")) {
+            type = JSON_FUN_INT;
+        } else if (!strcmp(c_type, "float")) {
+            type = JSON_FUN_DOUBLE;
+        } else if (!strcmp(c_type, "string")) {
+            type = JSON_FUN_STRING;
+        } else
+        {
+            ASSERT_TRUE(false);
+        }
+        rapidjson::Value* res3 = JsonFunctions::get_json_object(nullptr, json_raw_data, path["value"].GetString(), type, &jsonDoc2);
+        rapidjson::Value& value = *res3;
+        if (value.IsArray()) {
+            std::cout << "Array:" << path["value"].GetString() << std::endl;
+
+            for (int i = 0; i < res3->Size(); i++) {
+                std::cout << (res3 + i)->GetString()  << " ";
+            }
+            std::cout << std::endl;
+        }
+    }
+}
+
 }
 
 int main(int argc, char** argv) {
-    std::string conffile = std::string(getenv("DORIS_HOME")) + "/conf/be.conf";
+    std::string home(getenv("DORIS_HOME"));
+    if (home.empty()) {
+        home = ".";
+    }
+    std::string conffile = home + "/conf/be.conf";
+    std::cout<<conffile<<std::endl;
 
 Review comment:
   Do we need cout?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r410703515
 
 

 ##########
 File path: be/src/exprs/json_functions.cpp
 ##########
 @@ -107,75 +107,28 @@ DoubleVal JsonFunctions::get_json_double(
     }
 }
 
-rapidjson::Value* JsonFunctions::get_json_object(
-        FunctionContext* context,
-        const std::string& json_string,
-        const std::string& path_string,
-        const JsonFunctionType& fntype,
-        rapidjson::Document* document) {
-
-    // split path by ".", and escape quota by "\"
-    // eg:
-    //    '$.text#abc.xyz'  ->  [$, text#abc, xyz]
-    //    '$."text.abc".xyz'  ->  [$, text.abc, xyz]
-    //    '$."text.abc"[1].xyz'  ->  [$, text.abc[1], xyz]
-    std::vector<JsonPath>* parsed_paths;
-    std::vector<JsonPath> tmp_parsed_paths;
-#ifndef BE_TEST
-    parsed_paths = reinterpret_cast<std::vector<JsonPath>*>(context->get_function_state(FunctionContext::FRAGMENT_LOCAL));
-    if (parsed_paths == nullptr) {
-        boost::tokenizer<boost::escaped_list_separator<char> > tok(path_string, boost::escaped_list_separator<char>("\\", ".", "\""));
-        std::vector<std::string> paths(tok.begin(), tok.end());
-        get_parsed_paths(paths, &tmp_parsed_paths);
-        parsed_paths = &tmp_parsed_paths;
-    }
-#else
-    boost::tokenizer<boost::escaped_list_separator<char> > tok(path_string, boost::escaped_list_separator<char>("\\", ".", "\""));
-    std::vector<std::string> paths(tok.begin(), tok.end());
-    get_parsed_paths(paths, &tmp_parsed_paths);
-    parsed_paths = &tmp_parsed_paths;
-#endif
-
-    VLOG(10) << "first parsed path: " << (*parsed_paths)[0].debug_string();
-
-    if (!(*parsed_paths)[0].is_valid) {
-        return document;
-    }
-
-    if (UNLIKELY((*parsed_paths).size() == 1)) {
-        if (fntype == JSON_FUN_STRING) {
-            document->SetString(json_string.c_str(), document->GetAllocator());
-        } else {
-            return document;
-        }
-    }
-
-    //rapidjson::Document document;
-    document->Parse(json_string.c_str());
-    if (UNLIKELY(document->HasParseError())) {
-        VLOG(1) << "Error at offset " << document->GetErrorOffset()
-            << ": " << GetParseError_En(document->GetParseError());
-        document->SetNull();
-        return document;
-    }
 
+rapidjson::Value* JsonFunctions::match_value(std::vector<JsonPath>& parsed_paths, rapidjson::Document* document) {
+    //bool is_arr_set_by_last = false;
 
 Review comment:
   Remove the unused variables

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r409998385
 
 

 ##########
 File path: be/src/exec/json_scanner.cpp
 ##########
 @@ -0,0 +1,485 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+         _jsonValues(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::isEnd() {
+    return _jsonValues->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::getNext() {
+    if (isEnd()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonPathDoc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+                _parseJsonPathFlag = -1;// failed, has none object
+            } else {
+                _parseJsonPathFlag = 1;// success
+            }
+        } else {
+            _parseJsonPathFlag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parseJsonPathFlag = parseJsonPathFromFile(fileMgr, jsonpath_file);
+    } else {
+        _parseJsonPathFlag = 0;
+    }
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parseJsonPathFromFile(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parseJsonPathFromFile Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonPathDoc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parseJsonDoc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr; //
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_jsonDoc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::getDataByJsonPath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonPathDoc["jsonpath"];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* jsonValues = JsonFunctions::get_json_object_simple(value, &_jsonDoc);
+        if (jsonValues == NULL) {
+            return -1;
+        }
+        if (jsonValues->IsArray()) {
+            max_lines = max_lines < jsonValues->Size() ? jsonValues->Size() : max_lines;
+        } else {
+            max_lines = max_lines > 1 ? max_lines : 1;
+        }
+        jmap.emplace(column, jsonValues);
+    }
+    return max_lines;
+}
+
+void JsonReader::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {
+    tuple->set_not_null(slot_desc->null_indicator_offset());
+    void* slot = tuple->get_slot(slot_desc->tuple_offset());
+    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(str_slot->ptr, value, len);
+    str_slot->len = len;
+    return;
+}
+
+Status JsonReader::writeDataToTuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool) {
+    const char *str_value = NULL;
+    uint8_t tmp_buf[128] = {0};
+    int32_t wbytes = 0;
+    switch (value->GetType()) {
+        case rapidjson::Type::kStringType:
+            str_value = value->GetString();
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)str_value, strlen(str_value));
+            break;
+        case rapidjson::Type::kNumberType:
+            if (value->IsUint()) {
+                wbytes = sprintf((char*)tmp_buf, "%u", value->GetUint());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt()) {
+                wbytes = sprintf((char*)tmp_buf, "%d", value->GetInt());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsUint64()) {
+                wbytes = sprintf((char*)tmp_buf, "%lu", value->GetUint64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt64()) {
+                wbytes = sprintf((char*)tmp_buf, "%ld", value->GetInt64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else {
+                wbytes = sprintf((char*)tmp_buf, "%f", value->GetDouble());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            }
+            break;
+        case rapidjson::Type::kFalseType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"false", 5);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"0", 1);
+            break;
+        case rapidjson::Type::kTrueType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"true", 4);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"1", 1);
+            break;
+        default:
+            std::stringstream str_error;
+            str_error << "Invalid JsonType " << value->GetType() << ", Column Name `" << desc->col_name() << "`.";
+            LOG(WARNING) << str_error.str();
+            return Status::RuntimeError(str_error.str());
+    }
+    return Status::OK();
+}
+
+/**
+ * handle input a simple json
+ * For example:
+ *  1. {"doris_data": [{"colunm1":"value1", "colunm2":10}, {"colunm1":"value2", "colunm2":30}]}
+ *  2. {"colunm1":"value1", "colunm2":10}
+ */
+Status JsonReader::handleSimpleJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof) {
+    if (_next_line >= _total_lines) {
+        // generic document
+        RETURN_IF_ERROR(parseJsonDoc(eof));
+        if (*eof) {// handle over all data
+            return Status::OK();
+        }
+        if (_jsonDoc.HasMember("doris_data") && _jsonDoc["doris_data"].IsArray() ) {
+            _total_lines = _jsonDoc["doris_data"].Size();
+        } else {
+            _total_lines = 1;
+        }
+        _next_line = 0;
+    }
+
+    if (_jsonDoc.HasMember("doris_data") && _jsonDoc["doris_data"].IsArray()) {//handle case 1
+        rapidjson::Value& valueArray = _jsonDoc["doris_data"];
+        rapidjson::Value& objectValue = valueArray[_next_line++];// json object
+        for (auto v : slot_descs) {
+            if (objectValue.HasMember(v->col_name().c_str())) {
+                rapidjson::Value& value = objectValue[v->col_name().c_str()];
+                writeDataToTuple(&value, v, tuple, tuple_pool);
+            } else {
+                if (v->is_nullable()) {
+                    tuple->set_null(v->null_indicator_offset());
+                } else  {
+                    std::stringstream str_error;
+                    str_error << "The column `" << v->col_name() << "` is not nullable, but it's not found in jsondata.";
+                    LOG(WARNING) << str_error.str();
+                    return Status::RuntimeError(str_error.str());
+                }
+            }
+        }
+    } else {// handle case 2
+        int nullcount = 0;
+        for (auto v : slot_descs) {
+            if (_jsonDoc.HasMember(v->col_name().c_str())) {
+                rapidjson::Value& value = _jsonDoc[v->col_name().c_str()];
+                writeDataToTuple(&value, v, tuple, tuple_pool);
 
 Review comment:
   judge return status

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r403421635
 
 

 ##########
 File path: be/src/exec/json_scanner.cpp
 ##########
 @@ -0,0 +1,479 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(NULL),
 
 Review comment:
   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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r410011852
 
 

 ##########
 File path: be/src/exec/json_scanner.cpp
 ##########
 @@ -0,0 +1,485 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+         _jsonValues(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::isEnd() {
+    return _jsonValues->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::getNext() {
+    if (isEnd()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonPathDoc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+                _parseJsonPathFlag = -1;// failed, has none object
+            } else {
+                _parseJsonPathFlag = 1;// success
+            }
+        } else {
+            _parseJsonPathFlag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parseJsonPathFlag = parseJsonPathFromFile(fileMgr, jsonpath_file);
+    } else {
+        _parseJsonPathFlag = 0;
+    }
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parseJsonPathFromFile(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parseJsonPathFromFile Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonPathDoc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parseJsonDoc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr; //
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_jsonDoc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::getDataByJsonPath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonPathDoc["jsonpath"];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* jsonValues = JsonFunctions::get_json_object_simple(value, &_jsonDoc);
+        if (jsonValues == NULL) {
+            return -1;
+        }
+        if (jsonValues->IsArray()) {
+            max_lines = max_lines < jsonValues->Size() ? jsonValues->Size() : max_lines;
+        } else {
+            max_lines = max_lines > 1 ? max_lines : 1;
+        }
+        jmap.emplace(column, jsonValues);
+    }
+    return max_lines;
+}
+
+void JsonReader::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {
+    tuple->set_not_null(slot_desc->null_indicator_offset());
+    void* slot = tuple->get_slot(slot_desc->tuple_offset());
+    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(str_slot->ptr, value, len);
+    str_slot->len = len;
+    return;
+}
+
+Status JsonReader::writeDataToTuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool) {
+    const char *str_value = NULL;
+    uint8_t tmp_buf[128] = {0};
+    int32_t wbytes = 0;
+    switch (value->GetType()) {
+        case rapidjson::Type::kStringType:
+            str_value = value->GetString();
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)str_value, strlen(str_value));
+            break;
+        case rapidjson::Type::kNumberType:
+            if (value->IsUint()) {
+                wbytes = sprintf((char*)tmp_buf, "%u", value->GetUint());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt()) {
+                wbytes = sprintf((char*)tmp_buf, "%d", value->GetInt());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsUint64()) {
+                wbytes = sprintf((char*)tmp_buf, "%lu", value->GetUint64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt64()) {
+                wbytes = sprintf((char*)tmp_buf, "%ld", value->GetInt64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else {
+                wbytes = sprintf((char*)tmp_buf, "%f", value->GetDouble());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            }
+            break;
+        case rapidjson::Type::kFalseType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"false", 5);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"0", 1);
+            break;
+        case rapidjson::Type::kTrueType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"true", 4);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"1", 1);
+            break;
+        default:
+            std::stringstream str_error;
+            str_error << "Invalid JsonType " << value->GetType() << ", Column Name `" << desc->col_name() << "`.";
+            LOG(WARNING) << str_error.str();
+            return Status::RuntimeError(str_error.str());
+    }
+    return Status::OK();
+}
+
+/**
+ * handle input a simple json
+ * For example:
+ *  1. {"doris_data": [{"colunm1":"value1", "colunm2":10}, {"colunm1":"value2", "colunm2":30}]}
+ *  2. {"colunm1":"value1", "colunm2":10}
+ */
+Status JsonReader::handleSimpleJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof) {
+    if (_next_line >= _total_lines) {
+        // generic document
+        RETURN_IF_ERROR(parseJsonDoc(eof));
+        if (*eof) {// handle over all data
+            return Status::OK();
+        }
+        if (_jsonDoc.HasMember("doris_data") && _jsonDoc["doris_data"].IsArray() ) {
+            _total_lines = _jsonDoc["doris_data"].Size();
+        } else {
+            _total_lines = 1;
+        }
+        _next_line = 0;
+    }
+
+    if (_jsonDoc.HasMember("doris_data") && _jsonDoc["doris_data"].IsArray()) {//handle case 1
+        rapidjson::Value& valueArray = _jsonDoc["doris_data"];
+        rapidjson::Value& objectValue = valueArray[_next_line++];// json object
+        for (auto v : slot_descs) {
+            if (objectValue.HasMember(v->col_name().c_str())) {
+                rapidjson::Value& value = objectValue[v->col_name().c_str()];
+                writeDataToTuple(&value, v, tuple, tuple_pool);
+            } else {
+                if (v->is_nullable()) {
+                    tuple->set_null(v->null_indicator_offset());
+                } else  {
+                    std::stringstream str_error;
+                    str_error << "The column `" << v->col_name() << "` is not nullable, but it's not found in jsondata.";
+                    LOG(WARNING) << str_error.str();
+                    return Status::RuntimeError(str_error.str());
+                }
+            }
+        }
+    } else {// handle case 2
+        int nullcount = 0;
+        for (auto v : slot_descs) {
+            if (_jsonDoc.HasMember(v->col_name().c_str())) {
+                rapidjson::Value& value = _jsonDoc[v->col_name().c_str()];
+                writeDataToTuple(&value, v, tuple, tuple_pool);
+            } else {
+                if (v->is_nullable()) {
+                    nullcount++;
+                    tuple->set_null(v->null_indicator_offset());
+                } else  {
+                    std::stringstream str_error;
+                    str_error << "The column `" << v->col_name() << "` is not nullable, but it's not found in jsondata.";
+                    LOG(WARNING) << str_error.str();
+                    return Status::RuntimeError(str_error.str());
+                }
+            }
+        }
+        if (nullcount == slot_descs.size()) {// All fields is null, then it's judged a failure
+            return Status::RuntimeError("All column names were not found in the json data.");
+        }
+        _next_line = 1;//only one row, so set _next_line = 1
+    }
+    return Status::OK();
+}
+
+Status JsonReader::handleComplexJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof) {
+    if (_next_line >= _total_lines) {
+        RETURN_IF_ERROR(parseJsonDoc(eof));
+        if (*eof) {
+            return Status::OK();
+        }
+        _total_lines = getDataByJsonPath();
+        if (_total_lines == -1) {
+            return Status::InternalError("Parse json data is failed.");
+        } else if (_total_lines == 0) {
+            *eof = true;
+            return Status::OK();
+        }
+        _next_line = 0;
+    }
+
+    std::map<std::string, JsonDataInternal>::iterator it_map;
+    for (auto v : slot_descs) {
+        it_map = jmap.find(v->col_name());
+        if (it_map == jmap.end()) {
+            return Status::RuntimeError("The column name of table is not foud in jsonpath.");
+        }
+        rapidjson::Value::ConstValueIterator value = it_map->second.getNext();
+        if (value == nullptr) {
+            if (v->is_nullable()) {
+                tuple->set_null(v->null_indicator_offset());
+            } else  {
+                std::stringstream str_error;
+                str_error << "The column `" << it_map->first << "` is not nullable, but it's not found in jsondata.";
+                LOG(WARNING) << str_error.str();
+                return Status::RuntimeError(str_error.str());
+            }
+        } else {
+            writeDataToTuple(value, v, tuple, tuple_pool);
 
 Review comment:
   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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
HangyuanLiu commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r409998346
 
 

 ##########
 File path: be/src/exec/json_scanner.cpp
 ##########
 @@ -0,0 +1,485 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+         _jsonValues(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::isEnd() {
+    return _jsonValues->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::getNext() {
+    if (isEnd()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonPathDoc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+                _parseJsonPathFlag = -1;// failed, has none object
+            } else {
+                _parseJsonPathFlag = 1;// success
+            }
+        } else {
+            _parseJsonPathFlag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parseJsonPathFlag = parseJsonPathFromFile(fileMgr, jsonpath_file);
+    } else {
+        _parseJsonPathFlag = 0;
+    }
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parseJsonPathFromFile(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parseJsonPathFromFile Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonPathDoc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parseJsonDoc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr; //
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_jsonDoc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::getDataByJsonPath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonPathDoc["jsonpath"];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* jsonValues = JsonFunctions::get_json_object_simple(value, &_jsonDoc);
+        if (jsonValues == NULL) {
+            return -1;
+        }
+        if (jsonValues->IsArray()) {
+            max_lines = max_lines < jsonValues->Size() ? jsonValues->Size() : max_lines;
+        } else {
+            max_lines = max_lines > 1 ? max_lines : 1;
+        }
+        jmap.emplace(column, jsonValues);
+    }
+    return max_lines;
+}
+
+void JsonReader::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {
+    tuple->set_not_null(slot_desc->null_indicator_offset());
+    void* slot = tuple->get_slot(slot_desc->tuple_offset());
+    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(str_slot->ptr, value, len);
+    str_slot->len = len;
+    return;
+}
+
+Status JsonReader::writeDataToTuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool) {
+    const char *str_value = NULL;
+    uint8_t tmp_buf[128] = {0};
+    int32_t wbytes = 0;
+    switch (value->GetType()) {
+        case rapidjson::Type::kStringType:
+            str_value = value->GetString();
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)str_value, strlen(str_value));
+            break;
+        case rapidjson::Type::kNumberType:
+            if (value->IsUint()) {
+                wbytes = sprintf((char*)tmp_buf, "%u", value->GetUint());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt()) {
+                wbytes = sprintf((char*)tmp_buf, "%d", value->GetInt());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsUint64()) {
+                wbytes = sprintf((char*)tmp_buf, "%lu", value->GetUint64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt64()) {
+                wbytes = sprintf((char*)tmp_buf, "%ld", value->GetInt64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else {
+                wbytes = sprintf((char*)tmp_buf, "%f", value->GetDouble());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            }
+            break;
+        case rapidjson::Type::kFalseType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"false", 5);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"0", 1);
+            break;
+        case rapidjson::Type::kTrueType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"true", 4);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"1", 1);
+            break;
+        default:
+            std::stringstream str_error;
+            str_error << "Invalid JsonType " << value->GetType() << ", Column Name `" << desc->col_name() << "`.";
+            LOG(WARNING) << str_error.str();
+            return Status::RuntimeError(str_error.str());
+    }
+    return Status::OK();
+}
+
+/**
+ * handle input a simple json
+ * For example:
+ *  1. {"doris_data": [{"colunm1":"value1", "colunm2":10}, {"colunm1":"value2", "colunm2":30}]}
+ *  2. {"colunm1":"value1", "colunm2":10}
+ */
+Status JsonReader::handleSimpleJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof) {
+    if (_next_line >= _total_lines) {
+        // generic document
+        RETURN_IF_ERROR(parseJsonDoc(eof));
+        if (*eof) {// handle over all data
+            return Status::OK();
+        }
+        if (_jsonDoc.HasMember("doris_data") && _jsonDoc["doris_data"].IsArray() ) {
+            _total_lines = _jsonDoc["doris_data"].Size();
+        } else {
+            _total_lines = 1;
+        }
+        _next_line = 0;
+    }
+
+    if (_jsonDoc.HasMember("doris_data") && _jsonDoc["doris_data"].IsArray()) {//handle case 1
+        rapidjson::Value& valueArray = _jsonDoc["doris_data"];
+        rapidjson::Value& objectValue = valueArray[_next_line++];// json object
+        for (auto v : slot_descs) {
+            if (objectValue.HasMember(v->col_name().c_str())) {
+                rapidjson::Value& value = objectValue[v->col_name().c_str()];
+                writeDataToTuple(&value, v, tuple, tuple_pool);
 
 Review comment:
   judge return status

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r410713892
 
 

 ##########
 File path: be/src/exprs/json_functions.cpp
 ##########
 @@ -209,31 +162,138 @@ rapidjson::Value* JsonFunctions::get_json_object(
             } else if (root->IsObject()){
                 if (!root->HasMember(col.c_str())) {
                     root->SetNull();
+                    continue;
                 } else {
                     root = &((*root)[col.c_str()]);
                 }
             } else {
                 // root is not a nested type, return NULL
                 root->SetNull();
+                continue;
             }
         }
 
         if (UNLIKELY(index != -1)) {
             // judge the rapidjson:Value, which base the top's result,
             // if not array return NULL;else get the index value from the array
             if (root->IsArray()) {
-                if (root->IsNull() || index >= root->Size()) {
+                if (root->IsNull()) {
+                    root->SetNull();
+                    continue;
+                } else if (index == -2) {
+                    // [*]
+                    array_obj = static_cast<rapidjson::Value*>(
+                            document->GetAllocator().Malloc(sizeof(rapidjson::Value)));
+                    array_obj->SetArray();
+
+                    for (int j = 0; j < root->Size(); j++) {
+                        rapidjson::Value v;
+                        v.CopyFrom((*root)[j], document->GetAllocator());
+                        array_obj->PushBack(v, document->GetAllocator());
+                    }
+                    root = array_obj;
+                    // is_arr_set_by_last = true;
+                } else if (index >= root->Size()) {
                     root->SetNull();
+                    continue;
                 } else {
                     root = &((*root)[index]);
                 }
             } else {
                 root->SetNull();
+                continue;
             }
         }
-     }
+    }
+    return root;
+}
+
+rapidjson::Value* JsonFunctions::get_json_object(
+        FunctionContext* context,
+        const std::string& json_string,
+        const std::string& path_string,
+        const JsonFunctionType& fntype,
+        rapidjson::Document* document) {
+
+    // split path by ".", and escape quota by "\"
+    // eg:
+    //    '$.text#abc.xyz'  ->  [$, text#abc, xyz]
+    //    '$."text.abc".xyz'  ->  [$, text.abc, xyz]
+    //    '$."text.abc"[1].xyz'  ->  [$, text.abc[1], xyz]
+    std::vector<JsonPath>* parsed_paths;
+    std::vector<JsonPath> tmp_parsed_paths;
+#ifndef BE_TEST
+    parsed_paths = reinterpret_cast<std::vector<JsonPath>*>(context->get_function_state(FunctionContext::FRAGMENT_LOCAL));
+    if (parsed_paths == nullptr) {
+        boost::tokenizer<boost::escaped_list_separator<char> > tok(path_string, boost::escaped_list_separator<char>("\\", ".", "\""));
+        std::vector<std::string> paths(tok.begin(), tok.end());
+        get_parsed_paths(paths, &tmp_parsed_paths);
+        parsed_paths = &tmp_parsed_paths;
+    }
+#else
+    boost::tokenizer<boost::escaped_list_separator<char> > tok(path_string, boost::escaped_list_separator<char>("\\", ".", "\""));
+    std::vector<std::string> paths(tok.begin(), tok.end());
+    get_parsed_paths(paths, &tmp_parsed_paths);
+    parsed_paths = &tmp_parsed_paths;
+#endif
+
+    VLOG(10) << "first parsed path: " << (*parsed_paths)[0].debug_string();
+
+    if (!(*parsed_paths)[0].is_valid) {
+        return document;
+    }
+
+    if (UNLIKELY((*parsed_paths).size() == 1)) {
+        if (fntype == JSON_FUN_STRING) {
+            document->SetString(json_string.c_str(), document->GetAllocator());
+        } else {
+            return document;
+        }
+    }
 
-     return root;
+    //rapidjson::Document document;
+    document->Parse(json_string.c_str());
+    if (UNLIKELY(document->HasParseError())) {
+        VLOG(1) << "Error at offset " << document->GetErrorOffset()
+            << ": " << GetParseError_En(document->GetParseError());
+        document->SetNull();
+        return document;
+    }
+    return match_value(*parsed_paths, document);
+}
+
+
+rapidjson::Value* JsonFunctions::get_json_object_simple (
 
 Review comment:
   ```suggestion
   rapidjson::Value* JsonFunctions::get_json_object_from_parsed_json(
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad
URL: https://github.com/apache/incubator-doris/pull/3230#discussion_r403421593
 
 

 ##########
 File path: be/src/exec/json_scanner.h
 ##########
 @@ -0,0 +1,140 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to
+    virtual Status open();
+
+    // Get next tuple
+    virtual Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof);
 
 Review comment:
   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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [incubator-doris] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

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



##########
File path: be/src/exprs/json_functions.cpp
##########
@@ -209,31 +162,138 @@ rapidjson::Value* JsonFunctions::get_json_object(
             } else if (root->IsObject()){
                 if (!root->HasMember(col.c_str())) {
                     root->SetNull();
+                    continue;
                 } else {
                     root = &((*root)[col.c_str()]);
                 }
             } else {
                 // root is not a nested type, return NULL
                 root->SetNull();
+                continue;
             }
         }
 
         if (UNLIKELY(index != -1)) {
             // judge the rapidjson:Value, which base the top's result,
             // if not array return NULL;else get the index value from the array
             if (root->IsArray()) {
-                if (root->IsNull() || index >= root->Size()) {
+                if (root->IsNull()) {
+                    root->SetNull();
+                    continue;
+                } else if (index == -2) {
+                    // [*]
+                    array_obj = static_cast<rapidjson::Value*>(
+                            document->GetAllocator().Malloc(sizeof(rapidjson::Value)));
+                    array_obj->SetArray();
+
+                    for (int j = 0; j < root->Size(); j++) {
+                        rapidjson::Value v;
+                        v.CopyFrom((*root)[j], document->GetAllocator());
+                        array_obj->PushBack(v, document->GetAllocator());
+                    }
+                    root = array_obj;
+                    // is_arr_set_by_last = true;
+                } else if (index >= root->Size()) {
                     root->SetNull();
+                    continue;
                 } else {
                     root = &((*root)[index]);
                 }
             } else {
                 root->SetNull();
+                continue;
             }
         }
-     }
+    }
+    return root;
+}
+
+rapidjson::Value* JsonFunctions::get_json_object(
+        FunctionContext* context,
+        const std::string& json_string,
+        const std::string& path_string,
+        const JsonFunctionType& fntype,
+        rapidjson::Document* document) {
+
+    // split path by ".", and escape quota by "\"
+    // eg:
+    //    '$.text#abc.xyz'  ->  [$, text#abc, xyz]
+    //    '$."text.abc".xyz'  ->  [$, text.abc, xyz]
+    //    '$."text.abc"[1].xyz'  ->  [$, text.abc[1], xyz]
+    std::vector<JsonPath>* parsed_paths;
+    std::vector<JsonPath> tmp_parsed_paths;
+#ifndef BE_TEST
+    parsed_paths = reinterpret_cast<std::vector<JsonPath>*>(context->get_function_state(FunctionContext::FRAGMENT_LOCAL));
+    if (parsed_paths == nullptr) {
+        boost::tokenizer<boost::escaped_list_separator<char> > tok(path_string, boost::escaped_list_separator<char>("\\", ".", "\""));
+        std::vector<std::string> paths(tok.begin(), tok.end());
+        get_parsed_paths(paths, &tmp_parsed_paths);
+        parsed_paths = &tmp_parsed_paths;
+    }
+#else
+    boost::tokenizer<boost::escaped_list_separator<char> > tok(path_string, boost::escaped_list_separator<char>("\\", ".", "\""));
+    std::vector<std::string> paths(tok.begin(), tok.end());
+    get_parsed_paths(paths, &tmp_parsed_paths);
+    parsed_paths = &tmp_parsed_paths;
+#endif
+
+    VLOG(10) << "first parsed path: " << (*parsed_paths)[0].debug_string();
+
+    if (!(*parsed_paths)[0].is_valid) {
+        return document;
+    }
+
+    if (UNLIKELY((*parsed_paths).size() == 1)) {
+        if (fntype == JSON_FUN_STRING) {
+            document->SetString(json_string.c_str(), document->GetAllocator());
+        } else {
+            return document;
+        }
+    }
 
-     return root;
+    //rapidjson::Document document;
+    document->Parse(json_string.c_str());
+    if (UNLIKELY(document->HasParseError())) {
+        VLOG(1) << "Error at offset " << document->GetErrorOffset()
+            << ": " << GetParseError_En(document->GetParseError());
+        document->SetNull();
+        return document;
+    }
+    return match_value(*parsed_paths, document);
+}
+
+
+rapidjson::Value* JsonFunctions::get_json_object_simple (

Review comment:
       ok

##########
File path: be/src/exec/json_scanner.cpp
##########
@@ -0,0 +1,485 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+         _jsonValues(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::isEnd() {
+    return _jsonValues->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::getNext() {
+    if (isEnd()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonPathDoc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+                _parseJsonPathFlag = -1;// failed, has none object
+            } else {
+                _parseJsonPathFlag = 1;// success
+            }
+        } else {
+            _parseJsonPathFlag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parseJsonPathFlag = parseJsonPathFromFile(fileMgr, jsonpath_file);
+    } else {
+        _parseJsonPathFlag = 0;
+    }
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parseJsonPathFromFile(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parseJsonPathFromFile Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonPathDoc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parseJsonDoc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr; //
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_jsonDoc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::getDataByJsonPath() {

Review comment:
       ok

##########
File path: be/src/exec/json_scanner.h
##########
@@ -0,0 +1,140 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to
+    Status open() override;
+
+    // Get next tuple
+    Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) override;
+
+    // Close this scanner
+    void close() override;
+private:
+    Status open_next_reader();
+private:
+    //const TBrokerScanRangeParams& _params;
+    const std::vector<TBrokerRangeDesc>& _ranges;
+    const std::vector<TNetworkAddress>& _broker_addresses;
+
+    std::string _jsonpath;
+    std::string _jsonpath_file;
+    // std::shared_ptr<rapidjson::Document> _jsonPathDoc;
+
+    // used to hold current StreamLoadPipe
+    std::shared_ptr<StreamLoadPipe> _stream_load_pipe;
+    // Reader
+    JsonReader* _cur_file_reader;
+    int _next_range;
+    bool _cur_file_eof; // is read over?
+    bool _scanner_eof;
+};
+
+
+class JsonDataInternal {
+public:
+    JsonDataInternal(rapidjson::Value* v);
+    ~JsonDataInternal();
+    rapidjson::Value::ConstValueIterator getNext();
+
+private:
+    bool isEnd();
+
+private:
+    rapidjson::Value* _jsonValues;
+    rapidjson::Value::ConstValueIterator  _iterator;
+};
+
+class JsonReader {
+public:
+    JsonReader(SmallFileMgr *fileMgr, RuntimeProfile* profile, FileReader* file_reader,
+            std::string& jsonpath, std::string& jsonpath_file);
+    ~JsonReader();
+    Status read(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof);
+
+private:
+    void fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len);
+    size_t getDataByJsonPath();
+    int parseJsonPathFromFile(SmallFileMgr *smallFileMgr, std::string& fileinfo);
+    Status parseJsonDoc(bool *eof);
+    Status handleSimpleJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof);
+    Status handleComplexJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof);
+    Status writeDataToTuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool);
+    void close();
+
+private:
+    int _next_line;
+    int _total_lines;
+    RuntimeProfile* _profile;
+    FileReader*_file_reader;
+    bool _closed;
+    /**
+     * _parseJsonPathFlag == 1, jsonpath is valid
+     * _parseJsonPathFlag == 0, jsonpath is empty, default
+     * _parseJsonPathFlag == -1, jsonpath parse is error, it will return ERROR
+     */
+    int _parseJsonPathFlag;
+    RuntimeProfile::Counter* _bytes_read_counter;
+    RuntimeProfile::Counter* _read_timer;
+    rapidjson::Document _jsonPathDoc;
+    rapidjson::Document _jsonDoc;
+    std::map<std::string, JsonDataInternal> jmap;

Review comment:
       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.

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 issue #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

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


   > > I have just review the code related to stream load. I will review the routine load part later.
   > > And I found a json lib which support full function of jsonpath:
   > > `https://github.com/danielaparker/jsoncons`
   > > It is very active project, and base on its benchmark, it's performance is good as rapidjson, except for the floating data type read time:
   > > https://github.com/danielaparker/json_benchmarks/blob/master/report/performance.md
   > > https://github.com/danielaparker/json_benchmarks/blob/master/report/performance_fp.md
   > > Maybe we can introduce this lib later.
   > 
   > @morningman AFAIK, the simdjson is most fast json parse in the word. https://github.com/simdjson/simdjson
   
   That is really impressive, we may replace the rapidjson with simdjson some later.
   But what I concern about is the support of json path. Seems that there is few C++ library which can fully support json path.


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

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 #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

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



##########
File path: gensrc/thrift/BackendService.thrift
##########
@@ -54,11 +55,12 @@ struct TRoutineLoadTask {
     6: optional string db
     7: optional string tbl
     8: optional string label
-    9: optional i64 max_interval_s
-    10: optional i64 max_batch_rows
-    11: optional i64 max_batch_size
-    12: optional TKafkaLoadInfo kafka_load_info
-    13: optional PaloInternalService.TExecPlanFragmentParams params
+    9: optional PlanNodes.TFileFormatType format

Review comment:
       Put new field at the end of the struct




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

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] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

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



##########
File path: be/src/exec/json_scanner.cpp
##########
@@ -0,0 +1,485 @@
+// 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 "exec/json_scanner.h"
+
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+         _jsonValues(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::isEnd() {
+    return _jsonValues->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::getNext() {
+    if (isEnd()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonPathDoc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+                _parseJsonPathFlag = -1;// failed, has none object
+            } else {
+                _parseJsonPathFlag = 1;// success
+            }
+        } else {
+            _parseJsonPathFlag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parseJsonPathFlag = parseJsonPathFromFile(fileMgr, jsonpath_file);
+    } else {
+        _parseJsonPathFlag = 0;
+    }
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parseJsonPathFromFile(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parseJsonPathFromFile Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonPathDoc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonPathDoc.HasMember("jsonpath") || !_jsonPathDoc["jsonpath"].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parseJsonDoc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr; //
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_jsonDoc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::getDataByJsonPath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonPathDoc["jsonpath"];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* jsonValues = JsonFunctions::get_json_object_simple(value, &_jsonDoc);
+        if (jsonValues == NULL) {
+            return -1;
+        }
+        if (jsonValues->IsArray()) {
+            max_lines = max_lines < jsonValues->Size() ? jsonValues->Size() : max_lines;

Review comment:
       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.

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 #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

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



##########
File path: be/src/exec/broker_reader.h
##########
@@ -52,6 +52,7 @@ class BrokerReader : public FileReader {
     // Read 
     virtual Status read(uint8_t* buf, size_t* buf_len, bool* eof) override;
     virtual Status readat(int64_t position, int64_t nbytes, int64_t* bytes_read, void* out) override;
+    virtual Status read(uint8_t** buf, size_t *length) override;

Review comment:
       Better named `read_all`

##########
File path: be/src/exec/json_scanner.h
##########
@@ -0,0 +1,146 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to
+    Status open() override;
+
+    // Get next tuple
+    Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) override;
+
+    // Close this scanner
+    void close() override;
+private:
+    Status open_next_reader();
+private:
+    const std::vector<TBrokerRangeDesc>& _ranges;
+    const std::vector<TNetworkAddress>& _broker_addresses;
+
+    std::string _jsonpath;
+    std::string _jsonpath_file;
+    // std::shared_ptr<rapidjson::Document> _jsonPathDoc;
+
+    // used to hold current StreamLoadPipe
+    std::shared_ptr<StreamLoadPipe> _stream_load_pipe;
+    // Reader
+    JsonReader* _cur_file_reader;
+    int _next_range;
+    bool _cur_file_eof; // is read over?
+    bool _scanner_eof;
+};
+
+
+class JsonDataInternal {
+public:
+    JsonDataInternal(rapidjson::Value* v);
+    ~JsonDataInternal();
+    rapidjson::Value::ConstValueIterator get_next();
+
+private:
+    bool is_end();
+
+private:
+    rapidjson::Value* _json_values;
+    rapidjson::Value::ConstValueIterator  _iterator;
+};
+
+class JsonReader {
+public:
+    JsonReader(RuntimeState* state, ScannerCounter* counter, SmallFileMgr *fileMgr, RuntimeProfile* profile, FileReader* file_reader,
+            std::string& jsonpath, std::string& jsonpath_file);
+    ~JsonReader();
+    Status read(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof);
+
+private:
+    void init_jsonpath(SmallFileMgr *fileMgr, std::string& jsonpath, std::string& jsonpath_file);
+    void fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len);
+    size_t get_data_by_jsonpath();
+    int parse_jsonpath_from_file(SmallFileMgr *smallFileMgr, std::string& fileinfo);
+    Status parse_json_doc(bool *eof);
+    Status set_tuple_value(rapidjson::Value& objectValue, Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool *valid, int *nullcount);
+    Status set_tuple_value_from_map(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool *valid);
+    Status handle_simple_json(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof);
+    Status handle_complex_json(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof);
+    Status write_data_to_tuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool);
+    void close();
+
+private:
+    static constexpr char const *JSON_PATH = "jsonpath";
+    static constexpr char const *DORIS_DATA = "doris_data";

Review comment:
       Better to change it to "RECORDS", to be compatible with MySQL

##########
File path: be/src/exec/json_scanner.cpp
##########
@@ -0,0 +1,517 @@
+// 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 "exec/json_scanner.h"
+#include <algorithm>
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state, _counter, _state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+        _json_values(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::is_end() {
+    return _json_values->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::get_next() {
+    if (is_end()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        RuntimeState* state, ScannerCounter* counter,
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _state(state),
+            _counter(counter),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    init_jsonpath(fileMgr, jsonpath, jsonpath_file);
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::init_jsonpath(SmallFileMgr *fileMgr, std::string& jsonpath, std::string& jsonpath_file) {
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonpath_doc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+                _parse_jsonpath_flag = -1;// failed, has none object
+            } else {
+                _parse_jsonpath_flag = 1;// success
+            }
+        } else {
+            _parse_jsonpath_flag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parse_jsonpath_flag = parse_jsonpath_from_file(fileMgr, jsonpath_file);
+    } else {
+        _parse_jsonpath_flag = 0;
+    }
+    return ;
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parse_jsonpath_from_file(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parse_jsonpath_from_file Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonpath_doc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parse_json_doc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr;
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_json_doc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::get_data_by_jsonpath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    _jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonpath_doc[JsonReader::JSON_PATH];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* json_values = JsonFunctions::get_json_object_from_parsed_json(value, &_json_doc);
+        if (json_values == NULL) {
+            return -1;
+        }
+        if (json_values->IsArray()) {
+            max_lines = std::max(max_lines, (size_t)json_values->Size());
+        } else {
+            max_lines = std::max(max_lines, (size_t)1);
+        }
+        _jmap.emplace(column, json_values);
+    }
+    return max_lines;
+}
+
+void JsonReader::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {
+    tuple->set_not_null(slot_desc->null_indicator_offset());
+    void* slot = tuple->get_slot(slot_desc->tuple_offset());
+    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(str_slot->ptr, value, len);
+    str_slot->len = len;
+    return;
+}
+
+Status JsonReader::write_data_to_tuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool) {
+    const char *str_value = NULL;
+    uint8_t tmp_buf[128] = {0};
+    int32_t wbytes = 0;
+    switch (value->GetType()) {
+        case rapidjson::Type::kStringType:
+            str_value = value->GetString();
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)str_value, strlen(str_value));
+            break;
+        case rapidjson::Type::kNumberType:
+            if (value->IsUint()) {
+                wbytes = sprintf((char*)tmp_buf, "%u", value->GetUint());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt()) {
+                wbytes = sprintf((char*)tmp_buf, "%d", value->GetInt());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsUint64()) {
+                wbytes = sprintf((char*)tmp_buf, "%lu", value->GetUint64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt64()) {
+                wbytes = sprintf((char*)tmp_buf, "%ld", value->GetInt64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else {
+                wbytes = sprintf((char*)tmp_buf, "%f", value->GetDouble());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            }
+            break;
+        case rapidjson::Type::kFalseType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"false", 5);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"0", 1);
+            break;
+        case rapidjson::Type::kTrueType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"true", 4);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"1", 1);
+            break;
+        case rapidjson::Type::kNullType:
+            if (desc->is_nullable()) {
+                tuple->set_null(desc->null_indicator_offset());
+            } else  {
+                std::stringstream str_error;
+                str_error << "Json value is null, but the column `" << desc->col_name() << "` is not nullable.";
+                LOG(WARNING) << str_error.str();
+                return Status::RuntimeError(str_error.str());
+            }
+            break;
+        default:
+            std::stringstream str_error;
+            str_error << "Invalid JsonType " << value->GetType() << ", Column Name `" << desc->col_name() << "`.";
+            LOG(WARNING) << str_error.str();
+            return Status::RuntimeError(str_error.str());
+    }
+    return Status::OK();
+}
+
+/**
+ * handle input a simple json
+ * For example:
+ *  1. {"doris_data": [{"colunm1":"value1", "colunm2":10}, {"colunm1":"value2", "colunm2":30}]}
+ *  2. {"colunm1":"value1", "colunm2":10}
+ */
+Status JsonReader::set_tuple_value(rapidjson::Value& objectValue, Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool *valid, int *nullcount) {
+    for (auto v : slot_descs) {
+        if (objectValue.HasMember(v->col_name().c_str())) {
+            rapidjson::Value& value = objectValue[v->col_name().c_str()];
+            RETURN_IF_ERROR(write_data_to_tuple(&value, v, tuple, tuple_pool));
+        } else {
+            if (v->is_nullable()) {
+                tuple->set_null(v->null_indicator_offset());
+                (*nullcount)++;
+            } else  {
+                std::stringstream str_error;
+                str_error << "The column `" << v->col_name() << "` is not nullable, but it's not found in jsondata.";
+                LOG(WARNING) << str_error.str();
+                _state->append_error_msg_to_file("", str_error.str());
+                _counter->num_rows_filtered++;
+                *valid = false; // current row is invalid
+                break;
+            }
+        }
+    }
+    *valid = true;
+    return Status::OK();
+}
+
+Status JsonReader::handle_simple_json(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof) {
+    do {
+        bool valid = false;
+        int nullcount = 0;
+        if (_next_line >= _total_lines) {//parse json and generic document
+            RETURN_IF_ERROR(parse_json_doc(eof));
+            if (*eof) {// read all data, then return
+                return Status::OK();
+            }
+            if (_json_doc.HasMember(JsonReader::DORIS_DATA) && _json_doc[JsonReader::DORIS_DATA].IsArray() ) {
+                _total_lines = _json_doc[JsonReader::DORIS_DATA].Size();
+            } else {
+                _total_lines = 1; // only one row
+            }
+            _next_line = 0;
+        }
+
+        if (_json_doc.HasMember(JsonReader::DORIS_DATA) && _json_doc[JsonReader::DORIS_DATA].IsArray()) {//handle case 1
+            rapidjson::Value& valueArray = _json_doc[JsonReader::DORIS_DATA];
+            rapidjson::Value& objectValue = valueArray[_next_line];// json object
+            RETURN_IF_ERROR(set_tuple_value(objectValue, tuple, slot_descs, tuple_pool, &valid, &nullcount));
+        } else {// handle case 2
+            RETURN_IF_ERROR(set_tuple_value(_json_doc, tuple, slot_descs, tuple_pool, &valid, &nullcount));
+        }
+        _next_line++;
+        if (!valid || nullcount == slot_descs.size()) {// All fields is null, read next one

Review comment:
       If `nullcount == slot_descs.size()` is true, here you skip this line, but not update the `_counter->num_rows_filtered`.

##########
File path: be/src/exec/json_scanner.h
##########
@@ -0,0 +1,146 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to
+    Status open() override;
+
+    // Get next tuple
+    Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) override;
+
+    // Close this scanner
+    void close() override;
+private:
+    Status open_next_reader();
+private:
+    const std::vector<TBrokerRangeDesc>& _ranges;
+    const std::vector<TNetworkAddress>& _broker_addresses;
+
+    std::string _jsonpath;
+    std::string _jsonpath_file;
+    // std::shared_ptr<rapidjson::Document> _jsonPathDoc;
+
+    // used to hold current StreamLoadPipe
+    std::shared_ptr<StreamLoadPipe> _stream_load_pipe;
+    // Reader
+    JsonReader* _cur_file_reader;
+    int _next_range;
+    bool _cur_file_eof; // is read over?
+    bool _scanner_eof;
+};
+
+
+class JsonDataInternal {
+public:
+    JsonDataInternal(rapidjson::Value* v);
+    ~JsonDataInternal();
+    rapidjson::Value::ConstValueIterator get_next();
+
+private:
+    bool is_end();
+
+private:
+    rapidjson::Value* _json_values;
+    rapidjson::Value::ConstValueIterator  _iterator;
+};
+
+class JsonReader {
+public:
+    JsonReader(RuntimeState* state, ScannerCounter* counter, SmallFileMgr *fileMgr, RuntimeProfile* profile, FileReader* file_reader,
+            std::string& jsonpath, std::string& jsonpath_file);
+    ~JsonReader();
+    Status read(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof);
+
+private:
+    void init_jsonpath(SmallFileMgr *fileMgr, std::string& jsonpath, std::string& jsonpath_file);
+    void fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len);
+    size_t get_data_by_jsonpath();
+    int parse_jsonpath_from_file(SmallFileMgr *smallFileMgr, std::string& fileinfo);
+    Status parse_json_doc(bool *eof);
+    Status set_tuple_value(rapidjson::Value& objectValue, Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool *valid, int *nullcount);
+    Status set_tuple_value_from_map(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool *valid);
+    Status handle_simple_json(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof);
+    Status handle_complex_json(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof);

Review comment:
       ```suggestion
       Status handle_complex_json(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool* eof);
   ```
   There are many `slot_descs` in other places has same problem.

##########
File path: be/src/exec/json_scanner.cpp
##########
@@ -0,0 +1,517 @@
+// 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 "exec/json_scanner.h"
+#include <algorithm>
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state, _counter, _state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+        _json_values(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::is_end() {
+    return _json_values->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::get_next() {
+    if (is_end()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        RuntimeState* state, ScannerCounter* counter,
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _state(state),
+            _counter(counter),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    init_jsonpath(fileMgr, jsonpath, jsonpath_file);
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::init_jsonpath(SmallFileMgr *fileMgr, std::string& jsonpath, std::string& jsonpath_file) {
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonpath_doc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+                _parse_jsonpath_flag = -1;// failed, has none object
+            } else {
+                _parse_jsonpath_flag = 1;// success
+            }
+        } else {
+            _parse_jsonpath_flag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parse_jsonpath_flag = parse_jsonpath_from_file(fileMgr, jsonpath_file);
+    } else {
+        _parse_jsonpath_flag = 0;
+    }
+    return ;
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parse_jsonpath_from_file(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parse_jsonpath_from_file Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonpath_doc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parse_json_doc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr;
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_json_doc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::get_data_by_jsonpath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    _jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonpath_doc[JsonReader::JSON_PATH];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* json_values = JsonFunctions::get_json_object_from_parsed_json(value, &_json_doc);
+        if (json_values == NULL) {
+            return -1;
+        }
+        if (json_values->IsArray()) {
+            max_lines = std::max(max_lines, (size_t)json_values->Size());
+        } else {
+            max_lines = std::max(max_lines, (size_t)1);
+        }
+        _jmap.emplace(column, json_values);
+    }
+    return max_lines;
+}
+
+void JsonReader::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {
+    tuple->set_not_null(slot_desc->null_indicator_offset());
+    void* slot = tuple->get_slot(slot_desc->tuple_offset());
+    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(str_slot->ptr, value, len);
+    str_slot->len = len;
+    return;
+}
+
+Status JsonReader::write_data_to_tuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool) {
+    const char *str_value = NULL;
+    uint8_t tmp_buf[128] = {0};
+    int32_t wbytes = 0;
+    switch (value->GetType()) {
+        case rapidjson::Type::kStringType:
+            str_value = value->GetString();
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)str_value, strlen(str_value));
+            break;
+        case rapidjson::Type::kNumberType:
+            if (value->IsUint()) {
+                wbytes = sprintf((char*)tmp_buf, "%u", value->GetUint());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt()) {
+                wbytes = sprintf((char*)tmp_buf, "%d", value->GetInt());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsUint64()) {
+                wbytes = sprintf((char*)tmp_buf, "%lu", value->GetUint64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt64()) {
+                wbytes = sprintf((char*)tmp_buf, "%ld", value->GetInt64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else {
+                wbytes = sprintf((char*)tmp_buf, "%f", value->GetDouble());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            }
+            break;
+        case rapidjson::Type::kFalseType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"false", 5);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"0", 1);
+            break;
+        case rapidjson::Type::kTrueType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"true", 4);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"1", 1);
+            break;
+        case rapidjson::Type::kNullType:
+            if (desc->is_nullable()) {
+                tuple->set_null(desc->null_indicator_offset());
+            } else  {
+                std::stringstream str_error;
+                str_error << "Json value is null, but the column `" << desc->col_name() << "` is not nullable.";
+                LOG(WARNING) << str_error.str();
+                return Status::RuntimeError(str_error.str());
+            }
+            break;
+        default:
+            std::stringstream str_error;
+            str_error << "Invalid JsonType " << value->GetType() << ", Column Name `" << desc->col_name() << "`.";
+            LOG(WARNING) << str_error.str();
+            return Status::RuntimeError(str_error.str());
+    }
+    return Status::OK();
+}
+
+/**
+ * handle input a simple json
+ * For example:
+ *  1. {"doris_data": [{"colunm1":"value1", "colunm2":10}, {"colunm1":"value2", "colunm2":30}]}
+ *  2. {"colunm1":"value1", "colunm2":10}
+ */
+Status JsonReader::set_tuple_value(rapidjson::Value& objectValue, Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool *valid, int *nullcount) {
+    for (auto v : slot_descs) {
+        if (objectValue.HasMember(v->col_name().c_str())) {
+            rapidjson::Value& value = objectValue[v->col_name().c_str()];
+            RETURN_IF_ERROR(write_data_to_tuple(&value, v, tuple, tuple_pool));
+        } else {
+            if (v->is_nullable()) {
+                tuple->set_null(v->null_indicator_offset());
+                (*nullcount)++;
+            } else  {
+                std::stringstream str_error;
+                str_error << "The column `" << v->col_name() << "` is not nullable, but it's not found in jsondata.";
+                LOG(WARNING) << str_error.str();
+                _state->append_error_msg_to_file("", str_error.str());
+                _counter->num_rows_filtered++;
+                *valid = false; // current row is invalid
+                break;
+            }
+        }
+    }
+    *valid = true;
+    return Status::OK();
+}
+
+Status JsonReader::handle_simple_json(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof) {
+    do {
+        bool valid = false;
+        int nullcount = 0;
+        if (_next_line >= _total_lines) {//parse json and generic document
+            RETURN_IF_ERROR(parse_json_doc(eof));
+            if (*eof) {// read all data, then return
+                return Status::OK();
+            }
+            if (_json_doc.HasMember(JsonReader::DORIS_DATA) && _json_doc[JsonReader::DORIS_DATA].IsArray() ) {
+                _total_lines = _json_doc[JsonReader::DORIS_DATA].Size();
+            } else {
+                _total_lines = 1; // only one row
+            }
+            _next_line = 0;
+        }
+
+        if (_json_doc.HasMember(JsonReader::DORIS_DATA) && _json_doc[JsonReader::DORIS_DATA].IsArray()) {//handle case 1

Review comment:
       What is `case 1`, And what is `case 2`?

##########
File path: be/src/exec/json_scanner.cpp
##########
@@ -0,0 +1,517 @@
+// 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 "exec/json_scanner.h"
+#include <algorithm>
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state, _counter, _state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+        _json_values(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::is_end() {
+    return _json_values->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::get_next() {
+    if (is_end()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        RuntimeState* state, ScannerCounter* counter,
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _state(state),
+            _counter(counter),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    init_jsonpath(fileMgr, jsonpath, jsonpath_file);
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::init_jsonpath(SmallFileMgr *fileMgr, std::string& jsonpath, std::string& jsonpath_file) {
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonpath_doc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+                _parse_jsonpath_flag = -1;// failed, has none object
+            } else {
+                _parse_jsonpath_flag = 1;// success
+            }
+        } else {
+            _parse_jsonpath_flag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parse_jsonpath_flag = parse_jsonpath_from_file(fileMgr, jsonpath_file);
+    } else {
+        _parse_jsonpath_flag = 0;
+    }
+    return ;
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parse_jsonpath_from_file(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parse_jsonpath_from_file Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonpath_doc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parse_json_doc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr;
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_json_doc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::get_data_by_jsonpath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    _jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonpath_doc[JsonReader::JSON_PATH];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* json_values = JsonFunctions::get_json_object_from_parsed_json(value, &_json_doc);
+        if (json_values == NULL) {
+            return -1;
+        }
+        if (json_values->IsArray()) {
+            max_lines = std::max(max_lines, (size_t)json_values->Size());
+        } else {
+            max_lines = std::max(max_lines, (size_t)1);
+        }
+        _jmap.emplace(column, json_values);
+    }
+    return max_lines;
+}
+
+void JsonReader::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {
+    tuple->set_not_null(slot_desc->null_indicator_offset());
+    void* slot = tuple->get_slot(slot_desc->tuple_offset());
+    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(str_slot->ptr, value, len);
+    str_slot->len = len;
+    return;
+}
+
+Status JsonReader::write_data_to_tuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool) {
+    const char *str_value = NULL;
+    uint8_t tmp_buf[128] = {0};
+    int32_t wbytes = 0;
+    switch (value->GetType()) {
+        case rapidjson::Type::kStringType:
+            str_value = value->GetString();
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)str_value, strlen(str_value));
+            break;
+        case rapidjson::Type::kNumberType:
+            if (value->IsUint()) {
+                wbytes = sprintf((char*)tmp_buf, "%u", value->GetUint());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt()) {
+                wbytes = sprintf((char*)tmp_buf, "%d", value->GetInt());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsUint64()) {
+                wbytes = sprintf((char*)tmp_buf, "%lu", value->GetUint64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt64()) {
+                wbytes = sprintf((char*)tmp_buf, "%ld", value->GetInt64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else {
+                wbytes = sprintf((char*)tmp_buf, "%f", value->GetDouble());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            }
+            break;
+        case rapidjson::Type::kFalseType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"false", 5);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"0", 1);
+            break;
+        case rapidjson::Type::kTrueType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"true", 4);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"1", 1);
+            break;
+        case rapidjson::Type::kNullType:
+            if (desc->is_nullable()) {
+                tuple->set_null(desc->null_indicator_offset());
+            } else  {
+                std::stringstream str_error;
+                str_error << "Json value is null, but the column `" << desc->col_name() << "` is not nullable.";
+                LOG(WARNING) << str_error.str();
+                return Status::RuntimeError(str_error.str());
+            }
+            break;
+        default:
+            std::stringstream str_error;
+            str_error << "Invalid JsonType " << value->GetType() << ", Column Name `" << desc->col_name() << "`.";
+            LOG(WARNING) << str_error.str();
+            return Status::RuntimeError(str_error.str());
+    }
+    return Status::OK();
+}
+
+/**
+ * handle input a simple json
+ * For example:
+ *  1. {"doris_data": [{"colunm1":"value1", "colunm2":10}, {"colunm1":"value2", "colunm2":30}]}
+ *  2. {"colunm1":"value1", "colunm2":10}
+ */
+Status JsonReader::set_tuple_value(rapidjson::Value& objectValue, Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool *valid, int *nullcount) {
+    for (auto v : slot_descs) {
+        if (objectValue.HasMember(v->col_name().c_str())) {
+            rapidjson::Value& value = objectValue[v->col_name().c_str()];
+            RETURN_IF_ERROR(write_data_to_tuple(&value, v, tuple, tuple_pool));
+        } else {
+            if (v->is_nullable()) {
+                tuple->set_null(v->null_indicator_offset());
+                (*nullcount)++;
+            } else  {
+                std::stringstream str_error;
+                str_error << "The column `" << v->col_name() << "` is not nullable, but it's not found in jsondata.";
+                LOG(WARNING) << str_error.str();
+                _state->append_error_msg_to_file("", str_error.str());
+                _counter->num_rows_filtered++;
+                *valid = false; // current row is invalid
+                break;
+            }
+        }
+    }
+    *valid = true;
+    return Status::OK();
+}
+
+Status JsonReader::handle_simple_json(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof) {
+    do {
+        bool valid = false;
+        int nullcount = 0;
+        if (_next_line >= _total_lines) {//parse json and generic document
+            RETURN_IF_ERROR(parse_json_doc(eof));
+            if (*eof) {// read all data, then return
+                return Status::OK();
+            }
+            if (_json_doc.HasMember(JsonReader::DORIS_DATA) && _json_doc[JsonReader::DORIS_DATA].IsArray() ) {
+                _total_lines = _json_doc[JsonReader::DORIS_DATA].Size();
+            } else {
+                _total_lines = 1; // only one row
+            }
+            _next_line = 0;
+        }
+
+        if (_json_doc.HasMember(JsonReader::DORIS_DATA) && _json_doc[JsonReader::DORIS_DATA].IsArray()) {//handle case 1
+            rapidjson::Value& valueArray = _json_doc[JsonReader::DORIS_DATA];
+            rapidjson::Value& objectValue = valueArray[_next_line];// json object
+            RETURN_IF_ERROR(set_tuple_value(objectValue, tuple, slot_descs, tuple_pool, &valid, &nullcount));
+        } else {// handle case 2
+            RETURN_IF_ERROR(set_tuple_value(_json_doc, tuple, slot_descs, tuple_pool, &valid, &nullcount));
+        }
+        _next_line++;
+        if (!valid || nullcount == slot_descs.size()) {// All fields is null, read next one
+            continue;
+        }
+        break; // get a valid row, then break
+    } while (_next_line <= _total_lines);
+    return Status::OK();
+}
+
+Status JsonReader::set_tuple_value_from_map(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool *valid) {
+    std::unordered_map<std::string, JsonDataInternal>::iterator it_map;
+    for (auto v : slot_descs) {
+        it_map = _jmap.find(v->col_name());
+        if (it_map == _jmap.end()) {
+            return Status::RuntimeError("The column name of table is not foud in jsonpath.");
+        }
+        rapidjson::Value::ConstValueIterator value = it_map->second.get_next();
+        if (value == nullptr) {
+            if (v->is_nullable()) {
+                tuple->set_null(v->null_indicator_offset());
+            } else  {
+                std::stringstream str_error;
+                str_error << "The column `" << it_map->first << "` is not nullable, but it's not found in jsondata.";
+                LOG(WARNING) << str_error.str();

Review comment:
       Not print any log here, or it may print lots of logs

##########
File path: be/src/exec/json_scanner.cpp
##########
@@ -0,0 +1,517 @@
+// 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 "exec/json_scanner.h"
+#include <algorithm>
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state, _counter, _state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+        _json_values(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::is_end() {
+    return _json_values->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::get_next() {
+    if (is_end()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        RuntimeState* state, ScannerCounter* counter,
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _state(state),
+            _counter(counter),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    init_jsonpath(fileMgr, jsonpath, jsonpath_file);
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::init_jsonpath(SmallFileMgr *fileMgr, std::string& jsonpath, std::string& jsonpath_file) {
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonpath_doc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+                _parse_jsonpath_flag = -1;// failed, has none object
+            } else {
+                _parse_jsonpath_flag = 1;// success
+            }
+        } else {
+            _parse_jsonpath_flag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parse_jsonpath_flag = parse_jsonpath_from_file(fileMgr, jsonpath_file);
+    } else {
+        _parse_jsonpath_flag = 0;
+    }
+    return ;
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parse_jsonpath_from_file(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parse_jsonpath_from_file Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonpath_doc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parse_json_doc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr;
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_json_doc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::get_data_by_jsonpath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    _jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonpath_doc[JsonReader::JSON_PATH];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* json_values = JsonFunctions::get_json_object_from_parsed_json(value, &_json_doc);
+        if (json_values == NULL) {
+            return -1;
+        }
+        if (json_values->IsArray()) {
+            max_lines = std::max(max_lines, (size_t)json_values->Size());
+        } else {
+            max_lines = std::max(max_lines, (size_t)1);
+        }
+        _jmap.emplace(column, json_values);
+    }
+    return max_lines;
+}
+
+void JsonReader::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {
+    tuple->set_not_null(slot_desc->null_indicator_offset());
+    void* slot = tuple->get_slot(slot_desc->tuple_offset());
+    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(str_slot->ptr, value, len);
+    str_slot->len = len;
+    return;
+}
+
+Status JsonReader::write_data_to_tuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool) {
+    const char *str_value = NULL;
+    uint8_t tmp_buf[128] = {0};
+    int32_t wbytes = 0;
+    switch (value->GetType()) {
+        case rapidjson::Type::kStringType:
+            str_value = value->GetString();
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)str_value, strlen(str_value));
+            break;
+        case rapidjson::Type::kNumberType:
+            if (value->IsUint()) {
+                wbytes = sprintf((char*)tmp_buf, "%u", value->GetUint());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt()) {
+                wbytes = sprintf((char*)tmp_buf, "%d", value->GetInt());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsUint64()) {
+                wbytes = sprintf((char*)tmp_buf, "%lu", value->GetUint64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt64()) {
+                wbytes = sprintf((char*)tmp_buf, "%ld", value->GetInt64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else {
+                wbytes = sprintf((char*)tmp_buf, "%f", value->GetDouble());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            }
+            break;
+        case rapidjson::Type::kFalseType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"false", 5);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"0", 1);
+            break;
+        case rapidjson::Type::kTrueType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"true", 4);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"1", 1);
+            break;
+        case rapidjson::Type::kNullType:
+            if (desc->is_nullable()) {
+                tuple->set_null(desc->null_indicator_offset());
+            } else  {
+                std::stringstream str_error;
+                str_error << "Json value is null, but the column `" << desc->col_name() << "` is not nullable.";
+                LOG(WARNING) << str_error.str();
+                return Status::RuntimeError(str_error.str());
+            }
+            break;
+        default:
+            std::stringstream str_error;
+            str_error << "Invalid JsonType " << value->GetType() << ", Column Name `" << desc->col_name() << "`.";
+            LOG(WARNING) << str_error.str();
+            return Status::RuntimeError(str_error.str());
+    }
+    return Status::OK();
+}
+
+/**
+ * handle input a simple json
+ * For example:
+ *  1. {"doris_data": [{"colunm1":"value1", "colunm2":10}, {"colunm1":"value2", "colunm2":30}]}
+ *  2. {"colunm1":"value1", "colunm2":10}
+ */
+Status JsonReader::set_tuple_value(rapidjson::Value& objectValue, Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool *valid, int *nullcount) {
+    for (auto v : slot_descs) {
+        if (objectValue.HasMember(v->col_name().c_str())) {
+            rapidjson::Value& value = objectValue[v->col_name().c_str()];
+            RETURN_IF_ERROR(write_data_to_tuple(&value, v, tuple, tuple_pool));
+        } else {
+            if (v->is_nullable()) {
+                tuple->set_null(v->null_indicator_offset());
+                (*nullcount)++;
+            } else  {
+                std::stringstream str_error;
+                str_error << "The column `" << v->col_name() << "` is not nullable, but it's not found in jsondata.";
+                LOG(WARNING) << str_error.str();

Review comment:
       Not print any log here

##########
File path: be/src/exec/json_scanner.cpp
##########
@@ -0,0 +1,517 @@
+// 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 "exec/json_scanner.h"
+#include <algorithm>
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state, _counter, _state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+        _json_values(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::is_end() {
+    return _json_values->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::get_next() {
+    if (is_end()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        RuntimeState* state, ScannerCounter* counter,
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _state(state),
+            _counter(counter),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    init_jsonpath(fileMgr, jsonpath, jsonpath_file);
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::init_jsonpath(SmallFileMgr *fileMgr, std::string& jsonpath, std::string& jsonpath_file) {
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonpath_doc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+                _parse_jsonpath_flag = -1;// failed, has none object
+            } else {
+                _parse_jsonpath_flag = 1;// success
+            }
+        } else {
+            _parse_jsonpath_flag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parse_jsonpath_flag = parse_jsonpath_from_file(fileMgr, jsonpath_file);
+    } else {
+        _parse_jsonpath_flag = 0;
+    }
+    return ;
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parse_jsonpath_from_file(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parse_jsonpath_from_file Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonpath_doc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parse_json_doc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr;
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_json_doc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::get_data_by_jsonpath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    _jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonpath_doc[JsonReader::JSON_PATH];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* json_values = JsonFunctions::get_json_object_from_parsed_json(value, &_json_doc);
+        if (json_values == NULL) {
+            return -1;
+        }
+        if (json_values->IsArray()) {
+            max_lines = std::max(max_lines, (size_t)json_values->Size());
+        } else {
+            max_lines = std::max(max_lines, (size_t)1);
+        }
+        _jmap.emplace(column, json_values);
+    }
+    return max_lines;
+}
+
+void JsonReader::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {
+    tuple->set_not_null(slot_desc->null_indicator_offset());
+    void* slot = tuple->get_slot(slot_desc->tuple_offset());
+    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(str_slot->ptr, value, len);
+    str_slot->len = len;
+    return;
+}
+
+Status JsonReader::write_data_to_tuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool) {
+    const char *str_value = NULL;
+    uint8_t tmp_buf[128] = {0};
+    int32_t wbytes = 0;
+    switch (value->GetType()) {
+        case rapidjson::Type::kStringType:
+            str_value = value->GetString();
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)str_value, strlen(str_value));
+            break;
+        case rapidjson::Type::kNumberType:
+            if (value->IsUint()) {
+                wbytes = sprintf((char*)tmp_buf, "%u", value->GetUint());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt()) {
+                wbytes = sprintf((char*)tmp_buf, "%d", value->GetInt());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsUint64()) {
+                wbytes = sprintf((char*)tmp_buf, "%lu", value->GetUint64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt64()) {
+                wbytes = sprintf((char*)tmp_buf, "%ld", value->GetInt64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else {
+                wbytes = sprintf((char*)tmp_buf, "%f", value->GetDouble());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            }
+            break;
+        case rapidjson::Type::kFalseType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"false", 5);

Review comment:
       Remove unused code

##########
File path: be/src/exec/json_scanner.cpp
##########
@@ -0,0 +1,517 @@
+// 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 "exec/json_scanner.h"
+#include <algorithm>
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state, _counter, _state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+        _json_values(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::is_end() {
+    return _json_values->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::get_next() {
+    if (is_end()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        RuntimeState* state, ScannerCounter* counter,
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _state(state),
+            _counter(counter),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    init_jsonpath(fileMgr, jsonpath, jsonpath_file);
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::init_jsonpath(SmallFileMgr *fileMgr, std::string& jsonpath, std::string& jsonpath_file) {
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonpath_doc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+                _parse_jsonpath_flag = -1;// failed, has none object
+            } else {
+                _parse_jsonpath_flag = 1;// success
+            }
+        } else {
+            _parse_jsonpath_flag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parse_jsonpath_flag = parse_jsonpath_from_file(fileMgr, jsonpath_file);
+    } else {
+        _parse_jsonpath_flag = 0;
+    }
+    return ;
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parse_jsonpath_from_file(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parse_jsonpath_from_file Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonpath_doc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parse_json_doc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr;
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_json_doc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::get_data_by_jsonpath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    _jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonpath_doc[JsonReader::JSON_PATH];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* json_values = JsonFunctions::get_json_object_from_parsed_json(value, &_json_doc);
+        if (json_values == NULL) {
+            return -1;
+        }
+        if (json_values->IsArray()) {
+            max_lines = std::max(max_lines, (size_t)json_values->Size());
+        } else {
+            max_lines = std::max(max_lines, (size_t)1);
+        }
+        _jmap.emplace(column, json_values);
+    }
+    return max_lines;
+}
+
+void JsonReader::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {
+    tuple->set_not_null(slot_desc->null_indicator_offset());
+    void* slot = tuple->get_slot(slot_desc->tuple_offset());
+    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(str_slot->ptr, value, len);
+    str_slot->len = len;
+    return;
+}
+
+Status JsonReader::write_data_to_tuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool) {
+    const char *str_value = NULL;
+    uint8_t tmp_buf[128] = {0};
+    int32_t wbytes = 0;
+    switch (value->GetType()) {
+        case rapidjson::Type::kStringType:
+            str_value = value->GetString();
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)str_value, strlen(str_value));
+            break;
+        case rapidjson::Type::kNumberType:
+            if (value->IsUint()) {
+                wbytes = sprintf((char*)tmp_buf, "%u", value->GetUint());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt()) {
+                wbytes = sprintf((char*)tmp_buf, "%d", value->GetInt());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsUint64()) {
+                wbytes = sprintf((char*)tmp_buf, "%lu", value->GetUint64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt64()) {
+                wbytes = sprintf((char*)tmp_buf, "%ld", value->GetInt64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else {
+                wbytes = sprintf((char*)tmp_buf, "%f", value->GetDouble());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            }
+            break;
+        case rapidjson::Type::kFalseType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"false", 5);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"0", 1);
+            break;
+        case rapidjson::Type::kTrueType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"true", 4);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"1", 1);
+            break;
+        case rapidjson::Type::kNullType:
+            if (desc->is_nullable()) {
+                tuple->set_null(desc->null_indicator_offset());
+            } else  {
+                std::stringstream str_error;
+                str_error << "Json value is null, but the column `" << desc->col_name() << "` is not nullable.";
+                LOG(WARNING) << str_error.str();

Review comment:
       Not print any log here

##########
File path: be/src/exec/json_scanner.cpp
##########
@@ -0,0 +1,517 @@
+// 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 "exec/json_scanner.h"
+#include <algorithm>
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state, _counter, _state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+        _json_values(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::is_end() {
+    return _json_values->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::get_next() {
+    if (is_end()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        RuntimeState* state, ScannerCounter* counter,
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _state(state),
+            _counter(counter),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    init_jsonpath(fileMgr, jsonpath, jsonpath_file);
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::init_jsonpath(SmallFileMgr *fileMgr, std::string& jsonpath, std::string& jsonpath_file) {
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonpath_doc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+                _parse_jsonpath_flag = -1;// failed, has none object
+            } else {
+                _parse_jsonpath_flag = 1;// success
+            }
+        } else {
+            _parse_jsonpath_flag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parse_jsonpath_flag = parse_jsonpath_from_file(fileMgr, jsonpath_file);
+    } else {
+        _parse_jsonpath_flag = 0;
+    }
+    return ;
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parse_jsonpath_from_file(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parse_jsonpath_from_file Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonpath_doc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parse_json_doc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr;
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_json_doc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::get_data_by_jsonpath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    _jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonpath_doc[JsonReader::JSON_PATH];

Review comment:
       ```suggestion
       const rapidjson::Value& arr_json_path = _jsonpath_doc[JsonReader::JSON_PATH];
   ```

##########
File path: gensrc/thrift/BackendService.thrift
##########
@@ -54,11 +55,12 @@ struct TRoutineLoadTask {
     6: optional string db
     7: optional string tbl
     8: optional string label
-    9: optional i64 max_interval_s
-    10: optional i64 max_batch_rows
-    11: optional i64 max_batch_size
-    12: optional TKafkaLoadInfo kafka_load_info
-    13: optional PaloInternalService.TExecPlanFragmentParams params
+    9: optional PlanNodes.TFileFormatType format

Review comment:
       Agree

##########
File path: be/src/exec/json_scanner.cpp
##########
@@ -0,0 +1,517 @@
+// 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 "exec/json_scanner.h"
+#include <algorithm>
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state, _counter, _state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+        _json_values(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::is_end() {
+    return _json_values->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::get_next() {
+    if (is_end()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        RuntimeState* state, ScannerCounter* counter,
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _state(state),
+            _counter(counter),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    init_jsonpath(fileMgr, jsonpath, jsonpath_file);
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::init_jsonpath(SmallFileMgr *fileMgr, std::string& jsonpath, std::string& jsonpath_file) {
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonpath_doc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+                _parse_jsonpath_flag = -1;// failed, has none object
+            } else {
+                _parse_jsonpath_flag = 1;// success
+            }
+        } else {
+            _parse_jsonpath_flag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parse_jsonpath_flag = parse_jsonpath_from_file(fileMgr, jsonpath_file);
+    } else {
+        _parse_jsonpath_flag = 0;
+    }
+    return ;
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parse_jsonpath_from_file(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parse_jsonpath_from_file Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonpath_doc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parse_json_doc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr;
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_json_doc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::get_data_by_jsonpath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    _jmap.clear();
+    const rapidjson::Value& arrJsonPath = _jsonpath_doc[JsonReader::JSON_PATH];
+    for (int i = 0; i < arrJsonPath.Size(); i++) {
+        const rapidjson::Value& info = arrJsonPath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* json_values = JsonFunctions::get_json_object_from_parsed_json(value, &_json_doc);
+        if (json_values == NULL) {
+            return -1;
+        }
+        if (json_values->IsArray()) {
+            max_lines = std::max(max_lines, (size_t)json_values->Size());
+        } else {
+            max_lines = std::max(max_lines, (size_t)1);
+        }
+        _jmap.emplace(column, json_values);
+    }
+    return max_lines;
+}
+
+void JsonReader::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {
+    tuple->set_not_null(slot_desc->null_indicator_offset());
+    void* slot = tuple->get_slot(slot_desc->tuple_offset());
+    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(str_slot->ptr, value, len);
+    str_slot->len = len;
+    return;
+}
+
+Status JsonReader::write_data_to_tuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool) {
+    const char *str_value = NULL;
+    uint8_t tmp_buf[128] = {0};
+    int32_t wbytes = 0;
+    switch (value->GetType()) {
+        case rapidjson::Type::kStringType:
+            str_value = value->GetString();
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)str_value, strlen(str_value));
+            break;
+        case rapidjson::Type::kNumberType:
+            if (value->IsUint()) {
+                wbytes = sprintf((char*)tmp_buf, "%u", value->GetUint());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt()) {
+                wbytes = sprintf((char*)tmp_buf, "%d", value->GetInt());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsUint64()) {
+                wbytes = sprintf((char*)tmp_buf, "%lu", value->GetUint64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt64()) {
+                wbytes = sprintf((char*)tmp_buf, "%ld", value->GetInt64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else {
+                wbytes = sprintf((char*)tmp_buf, "%f", value->GetDouble());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            }
+            break;
+        case rapidjson::Type::kFalseType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"false", 5);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"0", 1);
+            break;
+        case rapidjson::Type::kTrueType:
+            //fill_slot(tuple, desc, tuple_pool, (uint8_t*)"true", 4);
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"1", 1);
+            break;
+        case rapidjson::Type::kNullType:
+            if (desc->is_nullable()) {
+                tuple->set_null(desc->null_indicator_offset());
+            } else  {
+                std::stringstream str_error;
+                str_error << "Json value is null, but the column `" << desc->col_name() << "` is not nullable.";
+                LOG(WARNING) << str_error.str();
+                return Status::RuntimeError(str_error.str());
+            }
+            break;
+        default:
+            std::stringstream str_error;
+            str_error << "Invalid JsonType " << value->GetType() << ", Column Name `" << desc->col_name() << "`.";
+            LOG(WARNING) << str_error.str();

Review comment:
       Not print any log here

##########
File path: fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java
##########
@@ -270,8 +281,20 @@ protected void setOptional(CreateRoutineLoadStmt stmt) throws UserException {
         if (stmt.getMaxBatchSize() != -1) {
             this.maxBatchSizeBytes = stmt.getMaxBatchSize();
         }
+        if (stmt.getFormat().equals("json")) {
+            this.format = "json";
+            if (stmt.getJsonPath() != null && !stmt.getJsonPath().isEmpty()) {

Review comment:
       ```suggestion
               if (!Strings.isNullOrEmpty(stmt.getJsonPath())) {
   ```

##########
File path: fe/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java
##########
@@ -328,6 +357,19 @@ private void checkJobProperties() throws UserException {
             timezone = ConnectContext.get().getSessionVariable().getTimeZone();
         }
         timezone = TimeUtils.checkTimeZoneValidAndStandardize(jobProperties.getOrDefault(LoadStmt.TIMEZONE, timezone));
+
+        format = jobProperties.get(FORMAT);
+        if (format != null) {
+            if (!format.equalsIgnoreCase("json")) {
+                format = "";// if it's not json, then it's mean csv and set empty

Review comment:
       I think we can just throw an exception here, means user specified a format which we don't support.

##########
File path: fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java
##########
@@ -175,6 +175,17 @@ public boolean isFinalState() {
     protected long maxBatchRows = DEFAULT_MAX_BATCH_ROWS;
     protected long maxBatchSizeBytes = DEFAULT_MAX_BATCH_SIZE;
 
+    /**
+     * RoutineLoad support json data.
+     * Require Params:
+     *   1) format = "json"
+     *   2) jsonPathFile = "/XXX/xx/jsonpath.json" or jsonPath = "$.XXX.xxx"

Review comment:
       ```suggestion
        *   2) jsonPathFile = "FILE:jsonpath.json" or jsonPath = "$.XXX.xxx"
   ```

##########
File path: fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java
##########
@@ -270,8 +281,20 @@ protected void setOptional(CreateRoutineLoadStmt stmt) throws UserException {
         if (stmt.getMaxBatchSize() != -1) {
             this.maxBatchSizeBytes = stmt.getMaxBatchSize();
         }
+        if (stmt.getFormat().equals("json")) {
+            this.format = "json";
+            if (stmt.getJsonPath() != null && !stmt.getJsonPath().isEmpty()) {
+                this.jsonPath = stmt.getJsonPath();
+            } else if (stmt.getJsonPathFile() != null && !stmt.getJsonPathFile().isEmpty()) {

Review comment:
       ```suggestion
               } else if (!Strings.isNullOrEmpty(stmt.getJsonPathFile()) {
   ```

##########
File path: docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/STREAM LOAD.md
##########
@@ -120,7 +120,8 @@ under the License.
     9. 导入含有BITMAP列的表,可以是表中的列或者数据中的列用于生成BITMAP列,也可以使用bitmap_empty填充空的Bitmap
         curl --location-trusted -u root -H "columns: k1, k2, v1=to_bitmap(k1), v2=bitmap_empty()" -T testData http://host:port/api/testDb/testTbl/_stream_load
 
- 
+    10. 导入json数据格式

Review comment:
       And a example about how to specify the `jsonpath_file` in stream load.




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

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] imay commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

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



##########
File path: be/src/exec/file_reader.h
##########
@@ -34,6 +34,16 @@ class FileReader {
     // is set to zero.
     virtual Status read(uint8_t* buf, size_t* buf_len, bool* eof) = 0;
     virtual Status readat(int64_t position, int64_t nbytes, int64_t* bytes_read, void* out) = 0;
+
+    /**
+     * if read eof then return Status::OK and length is set 0 and buf is set NULL,
+     *  other return readed bytes.
+     *
+     * !! Important !!
+     * the buf must be deleted by user, otherwise leak memory
+     * !! Important !!
+     */
+    virtual Status read_all(uint8_t** buf, size_t *length) = 0;

Review comment:
       Is read_one_message a better name?

##########
File path: docs/zh-CN/sql-reference/sql-statements/Data Manipulation/STREAM LOAD.md
##########
@@ -67,11 +67,11 @@ under the License.
         比如指定导入到p1, p2分区,-H "partitions: p1, p2"
 
         timeout: 指定导入的超时时间。单位秒。默认是 600 秒。可设置范围为 1 秒 ~ 259200 秒。
-        
+
         strict_mode: 用户指定此次导入是否开启严格模式,默认为开启。关闭方式为 -H "strict_mode: false"。
 
         timezone: 指定本次导入所使用的时区。默认为东八区。该参数会影响所有导入涉及的和时区有关的函数结果。
-        
+
         exec_mem_limit: 导入内存限制。默认为 2GB。单位为字节。

Review comment:
       lack format, jsonpath, jsonpathfile explain

##########
File path: docs/zh-CN/sql-reference/sql-statements/Data Manipulation/ROUTINE LOAD.md
##########
@@ -310,6 +318,84 @@ under the License.
             "property.client.id" = "my_client_id"
         );
 
+    4. 为 example_db 的 example_tbl 创建一个名为 test1 的 Kafka 例行导入任务,导入的简单json数据。
+        1)数据样例, doris_data为固定关键字
+        {
+            "doris_data":[
+                {"category":"a9jadhx","author":"test","price":895},
+                {"category":"axdfa1","author":"EvelynWaugh","price":1299}
+            ]
+        }
+        2) 创建任务,可不设置jsonpath或者jsonpath_file
+        CREATE ROUTINE LOAD example_db.test1 ON example_tbl
+        COLUMNS(category, author, price)
+        PROPERTIES
+        (
+            "desired_concurrent_number"="3",
+            "max_batch_interval" = "20",
+            "max_batch_rows" = "300000",
+            "max_batch_size" = "209715200",
+            "strict_mode" = "false",
+            "format" = "json"
+        )
+        FROM KAFKA
+        (
+            "kafka_broker_list" = "broker1:9092,broker2:9092,broker3:9092",
+            "kafka_topic" = "my_topic",
+            "kafka_partitions" = "0,1,2",
+            "kafka_offsets" = "0,0,0"
+        );
+
+    5. 通过jsonpath参数,为 example_db 的 example_tbl 创建一个名为 test1 的 Kafka 例行导入任务,导入的数据格式为json。
+
+        CREATE ROUTINE LOAD example_db.test1 ON example_tbl
+        COLUMNS(category, author, price)
+        PROPERTIES
+        (
+            "desired_concurrent_number"="3",
+            "max_batch_interval" = "20",
+            "max_batch_rows" = "300000",
+            "max_batch_size" = "209715200",
+            "strict_mode" = "false",
+            "format" = "json",
+            "jsonpath" = "{\"jsonpath\":[{\"column\":\"category\",\"value\":\"$.store.book.category\"},{\"column\":\"author\",\"value\":\"$.store.book.author\"},,{\"column\":\"price\",\"value\":\"$.store.book.price\"}]}"

Review comment:
       Why not use `jsonpaths=["$.store.book.category", "$.store.book.author", "$.store.book.price"]`
   

##########
File path: docs/zh-CN/sql-reference/sql-statements/Data Manipulation/ROUTINE LOAD.md
##########
@@ -137,15 +137,23 @@ under the License.
             采样窗口内,允许的最大错误行数。必须大于等于0。默认是 0,即不允许有错误行。
             采样窗口为 max_batch_rows * 10。即如果在采样窗口内,错误行数大于 max_error_number,则会导致例行作业被暂停,需要人工介入检查数据质量问题。
             被 where 条件过滤掉的行不算错误行。
-        
+
         4. strict_mode
 
             是否开启严格模式,默认为开启。如果开启后,非空原始数据的列类型变换如果结果为 NULL,则会被过滤。指定方式为 "strict_mode" = "true"
 
         5. timezone
-            
+
             指定导入作业所使用的时区。默认为使用 Session 的 timezone 参数。该参数会影响所有导入涉及的和时区有关的函数结果。
 
+        6. format
+
+            指定导入数据格式,默认是csv,支持json格式。
+
+        7. jsonpath、jsonpath_file

Review comment:
       1. jsonpaths and jsonpaths_file seems a better name
   2. should give explain about format of jsonpaths
   3. should give explain about how this works




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

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 #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

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



##########
File path: fe/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java
##########
@@ -328,6 +357,21 @@ private void checkJobProperties() throws UserException {
             timezone = ConnectContext.get().getSessionVariable().getTimeZone();
         }
         timezone = TimeUtils.checkTimeZoneValidAndStandardize(jobProperties.getOrDefault(LoadStmt.TIMEZONE, timezone));
+
+        format = jobProperties.get(FORMAT);
+        if (format != null) {
+            if (format.equalsIgnoreCase("csv")) {
+                format = "";// if it's not json, then it's mean csv and set empty
+            } else if (format.equalsIgnoreCase("json")) {
+                format = "json";
+                jsonPath = jobProperties.get(JSON_PATH);
+                jsonPathFile = jobProperties.get(JSON_PATH_FILE);
+            } else {
+                throw new UserException("Format type is invalid. format='"+format+"'");

Review comment:
       ```suggestion
                   throw new UserException("Format type is invalid. format='" + format + "'");
   ```

##########
File path: be/src/exec/json_scanner.cpp
##########
@@ -0,0 +1,515 @@
+// 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 "exec/json_scanner.h"
+#include <algorithm>
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state, _counter, _state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+        _json_values(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::is_end() {
+    return _json_values->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::get_next() {
+    if (is_end()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        RuntimeState* state, ScannerCounter* counter,
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _state(state),
+            _counter(counter),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    init_jsonpath(fileMgr, jsonpath, jsonpath_file);
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::init_jsonpath(SmallFileMgr *fileMgr, std::string& jsonpath, std::string& jsonpath_file) {
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonpath_doc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+                _parse_jsonpath_flag = -1;// failed, has none object
+            } else {
+                _parse_jsonpath_flag = 1;// success
+            }
+        } else {
+            _parse_jsonpath_flag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parse_jsonpath_flag = parse_jsonpath_from_file(fileMgr, jsonpath_file);
+    } else {
+        _parse_jsonpath_flag = 0;
+    }
+    return ;
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parse_jsonpath_from_file(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 2) {
+        LOG(WARNING)<< "parse_jsonpath_from_file Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[0]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[1], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonPathFile;
+    st = Env::Default()->new_random_access_file(file_path, &jsonPathFile);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonPathFile->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonPathFile->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonpath_doc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parse_json_doc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr;
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read_all(&json_str, &length));
+    if (length == 0) {
+        *eof = true;
+        return Status::OK();
+    }
+    //  parse jsondata to JsonDoc
+    if (_json_doc.Parse((char*)json_str, length).HasParseError()) {
+        delete[] json_str;
+        return Status::InternalError("Parse json data for JsonDoc is failed.");
+    }
+    delete[] json_str;
+    return Status::OK();
+}
+
+size_t JsonReader::get_data_by_jsonpath() {
+    size_t max_lines = 0;
+    //iterator jsonpath to find object and save it to Map
+    _jmap.clear();
+    const rapidjson::Value& arr_jsonpath = _jsonpath_doc[JsonReader::JSON_PATH];
+    for (int i = 0; i < arr_jsonpath.Size(); i++) {
+        const rapidjson::Value& info = arr_jsonpath[i];
+        if (!info.IsObject() || !info.HasMember("column") || !info.HasMember("value") ||
+                !info["column"].IsString() || !info["value"].IsString()) {
+            return -1;
+        }
+
+        std::string column = info["column"].GetString();
+        std::string value = info["value"].GetString();
+        // if jsonValues is null, because not match in jsondata.
+        rapidjson::Value* json_values = JsonFunctions::get_json_object_from_parsed_json(value, &_json_doc);
+        if (json_values == NULL) {
+            return -1;
+        }
+        if (json_values->IsArray()) {
+            max_lines = std::max(max_lines, (size_t)json_values->Size());
+        } else {
+            max_lines = std::max(max_lines, (size_t)1);
+        }
+        _jmap.emplace(column, json_values);
+    }
+    return max_lines;
+}
+
+void JsonReader::fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len) {
+    tuple->set_not_null(slot_desc->null_indicator_offset());
+    void* slot = tuple->get_slot(slot_desc->tuple_offset());
+    StringValue* str_slot = reinterpret_cast<StringValue*>(slot);
+    str_slot->ptr = reinterpret_cast<char*>(mem_pool->allocate(len));
+    memcpy(str_slot->ptr, value, len);
+    str_slot->len = len;
+    return;
+}
+
+Status JsonReader::write_data_to_tuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool) {
+    const char *str_value = NULL;
+    uint8_t tmp_buf[128] = {0};
+    int32_t wbytes = 0;
+    switch (value->GetType()) {
+        case rapidjson::Type::kStringType:
+            str_value = value->GetString();
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)str_value, strlen(str_value));
+            break;
+        case rapidjson::Type::kNumberType:
+            if (value->IsUint()) {
+                wbytes = sprintf((char*)tmp_buf, "%u", value->GetUint());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt()) {
+                wbytes = sprintf((char*)tmp_buf, "%d", value->GetInt());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsUint64()) {
+                wbytes = sprintf((char*)tmp_buf, "%lu", value->GetUint64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else if (value->IsInt64()) {
+                wbytes = sprintf((char*)tmp_buf, "%ld", value->GetInt64());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            } else {
+                wbytes = sprintf((char*)tmp_buf, "%f", value->GetDouble());
+                fill_slot(tuple, desc, tuple_pool, tmp_buf, wbytes);
+            }
+            break;
+        case rapidjson::Type::kFalseType:
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"0", 1);
+            break;
+        case rapidjson::Type::kTrueType:
+            fill_slot(tuple, desc, tuple_pool, (uint8_t*)"1", 1);
+            break;
+        case rapidjson::Type::kNullType:
+            if (desc->is_nullable()) {
+                tuple->set_null(desc->null_indicator_offset());
+            } else  {
+                std::stringstream str_error;
+                str_error << "Json value is null, but the column `" << desc->col_name() << "` is not nullable.";
+                return Status::RuntimeError(str_error.str());
+            }
+            break;
+        default:
+            std::stringstream str_error;
+            str_error << "Invalid JsonType " << value->GetType() << ", Column Name `" << desc->col_name() << "`.";
+            return Status::RuntimeError(str_error.str());
+    }
+    return Status::OK();
+}
+
+Status JsonReader::set_tuple_value(rapidjson::Value& objectValue, Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool *valid, int *nullcount) {
+    for (auto v : slot_descs) {
+        if (objectValue.HasMember(v->col_name().c_str())) {
+            rapidjson::Value& value = objectValue[v->col_name().c_str()];
+            RETURN_IF_ERROR(write_data_to_tuple(&value, v, tuple, tuple_pool));
+        } else {
+            if (v->is_nullable()) {
+                tuple->set_null(v->null_indicator_offset());
+                (*nullcount)++;
+            } else  {
+                std::stringstream str_error;
+                str_error << "The column `" << v->col_name() << "` is not nullable, but it's not found in jsondata.";
+                _state->append_error_msg_to_file("", str_error.str());
+                _counter->num_rows_filtered++;
+                *valid = false; // current row is invalid
+                break;
+            }
+        }
+    }
+    *valid = true;
+    return Status::OK();
+}
+
+/**
+ * handle input a simple json
+ * For example:
+ *  case 1. {"RECORDS": [{"colunm1":"value1", "colunm2":10}, {"colunm1":"value2", "colunm2":30}]}
+ *  case 2. {"colunm1":"value1", "colunm2":10}
+ */
+Status JsonReader::handle_simple_json(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool* eof) {
+    do {
+        bool valid = false;
+        int nullcount = 0;
+        if (_next_line >= _total_lines) {//parse json and generic document
+            RETURN_IF_ERROR(parse_json_doc(eof));
+            if (*eof) {// read all data, then return
+                return Status::OK();
+            }
+            if (_json_doc.HasMember(JsonReader::DORIS_RECORDS) && _json_doc[JsonReader::DORIS_RECORDS].IsArray() ) {
+                _total_lines = _json_doc[JsonReader::DORIS_RECORDS].Size();
+            } else {
+                _total_lines = 1; // only one row
+            }
+            _next_line = 0;
+        }
+
+        if (_json_doc.HasMember(JsonReader::DORIS_RECORDS) && _json_doc[JsonReader::DORIS_RECORDS].IsArray()) {//handle case 1
+            rapidjson::Value& values = _json_doc[JsonReader::DORIS_RECORDS];
+            rapidjson::Value& objectValue = values[_next_line];// json object
+            RETURN_IF_ERROR(set_tuple_value(objectValue, tuple, slot_descs, tuple_pool, &valid, &nullcount));
+        } else {// handle case 2
+            RETURN_IF_ERROR(set_tuple_value(_json_doc, tuple, slot_descs, tuple_pool, &valid, &nullcount));
+        }
+        _next_line++;
+        if (!valid) {
+            continue;
+        }
+        if (nullcount == slot_descs.size()) {// All fields is null, read next one

Review comment:
       Whenever you skip the line, you should print the line to the runtime state, so that user can find which line is wrong.
   So I think the case `nullcount == slot_descs.size()` can be just put into the `set_tuple_value()` method.




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

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] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

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



##########
File path: docs/documentation/cn/sql-reference/sql-statements/Data Manipulation/ROUTINE LOAD.md
##########
@@ -301,6 +309,84 @@ under the License.
             "property.client.id" = "my_client_id"
         );
 
+    4. 为 example_db 的 example_tbl 创建一个名为 test1 的 Kafka 例行导入任务,导入的简单json数据。
+        1)数据样例, doris_data为固定关键字
+        {
+            "doris_data":[
+                {"category":"a9jadhx","author":"test","price":895},
+                {"category":"axdfa1","author":"EvelynWaugh","price":1299}
+            ]
+        }
+        2) 创建任务,可不设置jsonpath或者jsonpath_file
+        CREATE ROUTINE LOAD example_db.test1 ON example_tbl
+        COLUMNS(category, author, price)
+        PROPERTIES
+        (
+            "desired_concurrent_number"="3",
+            "max_batch_interval" = "20",
+            "max_batch_rows" = "300000",
+            "max_batch_size" = "209715200",
+            "strict_mode" = "false",
+            "format" = "json"
+        )
+        FROM KAFKA
+        (
+            "kafka_broker_list" = "broker1:9092,broker2:9092,broker3:9092",
+            "kafka_topic" = "my_topic",
+            "kafka_partitions" = "0,1,2",
+            "kafka_offsets" = "0,0,0"
+        );
+
+    5. 通过jsonpath参数,为 example_db 的 example_tbl 创建一个名为 test1 的 Kafka 例行导入任务,导入的数据格式为json。
+
+        CREATE ROUTINE LOAD example_db.test1 ON example_tbl
+        COLUMNS(category, author, price)
+        PROPERTIES
+        (
+            "desired_concurrent_number"="3",
+            "max_batch_interval" = "20",
+            "max_batch_rows" = "300000",
+            "max_batch_size" = "209715200",
+            "strict_mode" = "false",
+            "format" = "json",
+            "jsonpath" = "{\"jsonpath\":[{\"column\":\"category\",\"value\":\"$.store.book.category\"},{\"column\":\"author\",\"value\":\"$.store.book.author\"},,{\"column\":\"price\",\"value\":\"$.store.book.price\"}]}"
+        )
+        FROM KAFKA
+        (
+            "kafka_broker_list" = "broker1:9092,broker2:9092,broker3:9092",
+            "kafka_topic" = "my_topic",
+            "kafka_partitions" = "0,1,2",
+            "kafka_offsets" = "0,0,0"
+        );
+
+    6. 通过jsonpath_file参数, 为 example_db 的 example_tbl 创建一个名为 test1 的 Kafka 例行导入任务,导入的数据格式为json。
+        1)通过create file导入jsonpath文件到Doris集群中,然后通过show file查看文件id以及md5, 例如:
+        mysql> show file;
+        +-------+--------------------------+---------+-----------------+----------+-----------+----------------------------------+
+        | Id    | DbName                   | Catalog | FileName        | FileSize | IsContent | MD5                              |
+        +-------+--------------------------+---------+-----------------+----------+-----------+----------------------------------+
+        | 43017 | default_cluster:mediavad | kafka   | myjsonpath.json | 215      | true      | 3f3ab257be8a422e0044abe5ed51d410 |
+        +-------+--------------------------+---------+-----------------+----------+-----------+----------------------------------+
+        2)创建导入任务
+        CREATE ROUTINE LOAD example_db.test1 ON example_tbl
+        COLUMNS(category, author, price)
+        PROPERTIES
+        (
+            "desired_concurrent_number"="3",
+            "max_batch_interval" = "20",
+            "max_batch_rows" = "300000",
+            "max_batch_size" = "209715200",
+            "strict_mode" = "false",
+            "format" = "json",
+            "jsonpath_file" = "43017:3f3ab257be8a422e0044abe5ed51d410"

Review comment:
       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.

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] kangkaisen commented on issue #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

Posted by GitBox <gi...@apache.org>.
kangkaisen commented on issue #3230:
URL: https://github.com/apache/incubator-doris/pull/3230#issuecomment-616272107


   > I have just review the code related to stream load. I will review the routine load part later.
   > 
   > And I found a json lib which support full function of jsonpath:
   > `https://github.com/danielaparker/jsoncons`
   > 
   > It is very active project, and base on its benchmark, it's performance is good as rapidjson, except for the floating data type read time:
   > 
   > https://github.com/danielaparker/json_benchmarks/blob/master/report/performance.md
   > https://github.com/danielaparker/json_benchmarks/blob/master/report/performance_fp.md
   > 
   > Maybe we can introduce this lib later.
   
   @morningman  AFAIK, the simdjson is most fast json parse in the word. https://github.com/simdjson/simdjson


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

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] worker24h commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

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



##########
File path: be/src/exec/json_scanner.h
##########
@@ -0,0 +1,140 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to
+    Status open() override;
+
+    // Get next tuple
+    Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) override;
+
+    // Close this scanner
+    void close() override;
+private:
+    Status open_next_reader();
+private:
+    //const TBrokerScanRangeParams& _params;
+    const std::vector<TBrokerRangeDesc>& _ranges;
+    const std::vector<TNetworkAddress>& _broker_addresses;
+
+    std::string _jsonpath;
+    std::string _jsonpath_file;
+    // std::shared_ptr<rapidjson::Document> _jsonPathDoc;
+
+    // used to hold current StreamLoadPipe
+    std::shared_ptr<StreamLoadPipe> _stream_load_pipe;
+    // Reader
+    JsonReader* _cur_file_reader;
+    int _next_range;
+    bool _cur_file_eof; // is read over?
+    bool _scanner_eof;
+};
+
+
+class JsonDataInternal {
+public:
+    JsonDataInternal(rapidjson::Value* v);
+    ~JsonDataInternal();
+    rapidjson::Value::ConstValueIterator getNext();
+
+private:
+    bool isEnd();
+
+private:
+    rapidjson::Value* _jsonValues;
+    rapidjson::Value::ConstValueIterator  _iterator;
+};
+
+class JsonReader {
+public:
+    JsonReader(SmallFileMgr *fileMgr, RuntimeProfile* profile, FileReader* file_reader,
+            std::string& jsonpath, std::string& jsonpath_file);
+    ~JsonReader();
+    Status read(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof);
+
+private:
+    void fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len);
+    size_t getDataByJsonPath();
+    int parseJsonPathFromFile(SmallFileMgr *smallFileMgr, std::string& fileinfo);
+    Status parseJsonDoc(bool *eof);
+    Status handleSimpleJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof);
+    Status handleComplexJson(Tuple* tuple, std::vector<SlotDescriptor*> slot_descs, MemPool* tuple_pool, bool* eof);
+    Status writeDataToTuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool);

Review comment:
       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.

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] kangpinghuang commented on a change in pull request #3230: Support load json-data into Doris by RoutineLoad or StreamLoad

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



##########
File path: be/src/exec/broker_reader.h
##########
@@ -52,6 +52,7 @@ class BrokerReader : public FileReader {
     // Read 
     virtual Status read(uint8_t* buf, size_t* buf_len, bool* eof) override;
     virtual Status readat(int64_t position, int64_t nbytes, int64_t* bytes_read, void* out) override;
+    virtual Status read_all(uint8_t** buf, size_t *length) override;

Review comment:
       ```suggestion
       virtual Status read_all(uint8_t** buf, size_t* length) override;
   ```

##########
File path: be/src/exec/file_reader.h
##########
@@ -34,6 +34,16 @@ class FileReader {
     // is set to zero.
     virtual Status read(uint8_t* buf, size_t* buf_len, bool* eof) = 0;
     virtual Status readat(int64_t position, int64_t nbytes, int64_t* bytes_read, void* out) = 0;
+
+    /**
+     * if read eof then return Status::OK and length is set 0 and buf is set NULL,
+     *  other return readed bytes.
+     *
+     * !! Important !!
+     * the buf must be deleted by user, otherwise leak memory
+     * !! Important !!
+     */
+    virtual Status read_all(uint8_t** buf, size_t *length) = 0;

Review comment:
       comment to explain why add this api

##########
File path: be/src/exec/json_scanner.h
##########
@@ -0,0 +1,146 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_

Review comment:
       use #pragma once

##########
File path: be/src/exec/json_scanner.h
##########
@@ -0,0 +1,146 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to
+    Status open() override;
+
+    // Get next tuple
+    Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) override;
+
+    // Close this scanner
+    void close() override;
+private:
+    Status open_next_reader();
+private:
+    const std::vector<TBrokerRangeDesc>& _ranges;
+    const std::vector<TNetworkAddress>& _broker_addresses;
+
+    std::string _jsonpath;
+    std::string _jsonpath_file;
+    // std::shared_ptr<rapidjson::Document> _jsonPathDoc;

Review comment:
       remove it if not used

##########
File path: be/src/exec/json_scanner.h
##########
@@ -0,0 +1,146 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to

Review comment:
       ```suggestion
       // Open this scanner, will initialize information needed
   ```

##########
File path: be/src/exec/json_scanner.h
##########
@@ -0,0 +1,146 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to
+    Status open() override;
+
+    // Get next tuple
+    Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) override;
+
+    // Close this scanner
+    void close() override;
+private:
+    Status open_next_reader();
+private:
+    const std::vector<TBrokerRangeDesc>& _ranges;
+    const std::vector<TNetworkAddress>& _broker_addresses;
+
+    std::string _jsonpath;
+    std::string _jsonpath_file;
+    // std::shared_ptr<rapidjson::Document> _jsonPathDoc;
+
+    // used to hold current StreamLoadPipe
+    std::shared_ptr<StreamLoadPipe> _stream_load_pipe;
+    // Reader
+    JsonReader* _cur_file_reader;
+    int _next_range;
+    bool _cur_file_eof; // is read over?
+    bool _scanner_eof;
+};
+
+
+class JsonDataInternal {
+public:
+    JsonDataInternal(rapidjson::Value* v);
+    ~JsonDataInternal();
+    rapidjson::Value::ConstValueIterator get_next();
+
+private:
+    bool is_end();

Review comment:
       ```suggestion
       bool _is_end();
   ```

##########
File path: be/src/exec/json_scanner.h
##########
@@ -0,0 +1,146 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to
+    Status open() override;
+
+    // Get next tuple
+    Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) override;
+
+    // Close this scanner
+    void close() override;
+private:
+    Status open_next_reader();
+private:
+    const std::vector<TBrokerRangeDesc>& _ranges;
+    const std::vector<TNetworkAddress>& _broker_addresses;
+
+    std::string _jsonpath;
+    std::string _jsonpath_file;
+    // std::shared_ptr<rapidjson::Document> _jsonPathDoc;
+
+    // used to hold current StreamLoadPipe
+    std::shared_ptr<StreamLoadPipe> _stream_load_pipe;
+    // Reader
+    JsonReader* _cur_file_reader;
+    int _next_range;
+    bool _cur_file_eof; // is read over?
+    bool _scanner_eof;
+};
+
+
+class JsonDataInternal {
+public:
+    JsonDataInternal(rapidjson::Value* v);
+    ~JsonDataInternal();
+    rapidjson::Value::ConstValueIterator get_next();
+
+private:
+    bool is_end();
+
+private:
+    rapidjson::Value* _json_values;
+    rapidjson::Value::ConstValueIterator  _iterator;
+};
+
+class JsonReader {
+public:
+    JsonReader(RuntimeState* state, ScannerCounter* counter, SmallFileMgr *fileMgr, RuntimeProfile* profile, FileReader* file_reader,
+            std::string& jsonpath, std::string& jsonpath_file);
+    ~JsonReader();
+    Status read(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool* eof);
+
+private:
+    void init_jsonpath(SmallFileMgr *fileMgr, std::string& jsonpath, std::string& jsonpath_file);
+    void fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len);
+    size_t get_data_by_jsonpath();
+    int parse_jsonpath_from_file(SmallFileMgr *smallFileMgr, std::string& fileinfo);
+    Status parse_json_doc(bool *eof);
+    Status set_tuple_value(rapidjson::Value& objectValue, Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool *valid);
+    Status set_tuple_value_from_map(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool *valid);
+    Status handle_simple_json(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool* eof);
+    Status handle_complex_json(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool* eof);
+    Status write_data_to_tuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool);
+    void close();
+
+private:
+    static constexpr char const *JSON_PATH = "jsonpath";

Review comment:
       ```suggestion
       static constexpr char const* JSON_PATH = "jsonpath";
   ```

##########
File path: be/src/exec/json_scanner.h
##########
@@ -0,0 +1,146 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to
+    Status open() override;
+
+    // Get next tuple
+    Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) override;
+
+    // Close this scanner
+    void close() override;
+private:
+    Status open_next_reader();
+private:
+    const std::vector<TBrokerRangeDesc>& _ranges;
+    const std::vector<TNetworkAddress>& _broker_addresses;
+
+    std::string _jsonpath;
+    std::string _jsonpath_file;
+    // std::shared_ptr<rapidjson::Document> _jsonPathDoc;
+
+    // used to hold current StreamLoadPipe
+    std::shared_ptr<StreamLoadPipe> _stream_load_pipe;
+    // Reader
+    JsonReader* _cur_file_reader;
+    int _next_range;
+    bool _cur_file_eof; // is read over?
+    bool _scanner_eof;
+};
+
+
+class JsonDataInternal {
+public:
+    JsonDataInternal(rapidjson::Value* v);
+    ~JsonDataInternal();
+    rapidjson::Value::ConstValueIterator get_next();
+
+private:
+    bool is_end();
+
+private:
+    rapidjson::Value* _json_values;
+    rapidjson::Value::ConstValueIterator  _iterator;
+};
+
+class JsonReader {
+public:
+    JsonReader(RuntimeState* state, ScannerCounter* counter, SmallFileMgr *fileMgr, RuntimeProfile* profile, FileReader* file_reader,
+            std::string& jsonpath, std::string& jsonpath_file);
+    ~JsonReader();
+    Status read(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool* eof);
+
+private:
+    void init_jsonpath(SmallFileMgr *fileMgr, std::string& jsonpath, std::string& jsonpath_file);
+    void fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len);
+    size_t get_data_by_jsonpath();
+    int parse_jsonpath_from_file(SmallFileMgr *smallFileMgr, std::string& fileinfo);
+    Status parse_json_doc(bool *eof);
+    Status set_tuple_value(rapidjson::Value& objectValue, Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool *valid);
+    Status set_tuple_value_from_map(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool *valid);
+    Status handle_simple_json(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool* eof);
+    Status handle_complex_json(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool* eof);
+    Status write_data_to_tuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool);
+    void close();
+
+private:
+    static constexpr char const *JSON_PATH = "jsonpath";
+    static constexpr char const *DORIS_RECORDS = "RECORDS";

Review comment:
       ```suggestion
       static constexpr char const* DORIS_RECORDS = "RECORDS";
   ```

##########
File path: be/src/exec/json_scanner.h
##########
@@ -0,0 +1,146 @@
+// 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.
+
+#ifndef BE_SRC_JSON_SCANNER_H_
+#define BE_SRC_JSON_SCANNER_H_
+
+#include <memory>
+#include <vector>
+#include <string>
+#include <map>
+#include <sstream>
+#include <rapidjson/document.h>
+
+#include "exec/base_scanner.h"
+#include "common/status.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "gen_cpp/Types_types.h"
+#include "util/slice.h"
+#include "util/runtime_profile.h"
+#include "runtime/mem_pool.h"
+#include "runtime/tuple.h"
+#include "runtime/descriptors.h"
+#include "runtime/stream_load/load_stream_mgr.h"
+#include "runtime/small_file_mgr.h"
+
+namespace doris {
+class Tuple;
+class SlotDescriptor;
+class RuntimeState;
+class TupleDescriptor;
+class MemTracker;
+class JsonReader;
+
+class JsonScanner : public BaseScanner {
+public:
+    JsonScanner(
+        RuntimeState* state,
+        RuntimeProfile* profile,
+        const TBrokerScanRangeParams& params,
+        const std::vector<TBrokerRangeDesc>& ranges,
+        const std::vector<TNetworkAddress>& broker_addresses,
+        ScannerCounter* counter);
+    ~JsonScanner();
+
+    // Open this scanner, will initialize information need to
+    Status open() override;
+
+    // Get next tuple
+    Status get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) override;
+
+    // Close this scanner
+    void close() override;
+private:
+    Status open_next_reader();
+private:
+    const std::vector<TBrokerRangeDesc>& _ranges;
+    const std::vector<TNetworkAddress>& _broker_addresses;
+
+    std::string _jsonpath;
+    std::string _jsonpath_file;
+    // std::shared_ptr<rapidjson::Document> _jsonPathDoc;
+
+    // used to hold current StreamLoadPipe
+    std::shared_ptr<StreamLoadPipe> _stream_load_pipe;
+    // Reader
+    JsonReader* _cur_file_reader;
+    int _next_range;
+    bool _cur_file_eof; // is read over?
+    bool _scanner_eof;
+};
+
+
+class JsonDataInternal {
+public:
+    JsonDataInternal(rapidjson::Value* v);
+    ~JsonDataInternal();
+    rapidjson::Value::ConstValueIterator get_next();
+
+private:
+    bool is_end();
+
+private:
+    rapidjson::Value* _json_values;
+    rapidjson::Value::ConstValueIterator  _iterator;
+};
+
+class JsonReader {
+public:
+    JsonReader(RuntimeState* state, ScannerCounter* counter, SmallFileMgr *fileMgr, RuntimeProfile* profile, FileReader* file_reader,
+            std::string& jsonpath, std::string& jsonpath_file);
+    ~JsonReader();
+    Status read(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool* eof);
+
+private:
+    void init_jsonpath(SmallFileMgr *fileMgr, std::string& jsonpath, std::string& jsonpath_file);
+    void fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len);
+    size_t get_data_by_jsonpath();
+    int parse_jsonpath_from_file(SmallFileMgr *smallFileMgr, std::string& fileinfo);
+    Status parse_json_doc(bool *eof);
+    Status set_tuple_value(rapidjson::Value& objectValue, Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool *valid);
+    Status set_tuple_value_from_map(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool *valid);
+    Status handle_simple_json(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool* eof);
+    Status handle_complex_json(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool* eof);
+    Status write_data_to_tuple(rapidjson::Value::ConstValueIterator value, SlotDescriptor* desc, Tuple* tuple, MemPool* tuple_pool);
+    void close();
+
+private:
+    static constexpr char const *JSON_PATH = "jsonpath";
+    static constexpr char const *DORIS_RECORDS = "RECORDS";
+    int _next_line;
+    int _total_lines;
+    RuntimeState* _state;
+    ScannerCounter* _counter;
+    RuntimeProfile* _profile;
+    FileReader*_file_reader;
+    bool _closed;
+    /**
+     * _parse_jsonpath_flag == 1, jsonpath is valid
+     * _parse_jsonpath_flag == 0, jsonpath is empty, default
+     * _parse_jsonpath_flag == -1, jsonpath parse is error, it will return ERROR
+     */
+    int _parse_jsonpath_flag;

Review comment:
       ```suggestion
       short _parse_jsonpath_flag;
   ```

##########
File path: be/src/exec/json_scanner.cpp
##########
@@ -0,0 +1,518 @@
+// 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 "exec/json_scanner.h"
+#include <algorithm>
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;

Review comment:
       break?

##########
File path: be/src/exec/json_scanner.cpp
##########
@@ -0,0 +1,518 @@
+// 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 "exec/json_scanner.h"
+#include <algorithm>
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state, _counter, _state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+        _json_values(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::is_end() {
+    return _json_values->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::get_next() {
+    if (is_end()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        RuntimeState* state, ScannerCounter* counter,
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _state(state),
+            _counter(counter),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    init_jsonpath(fileMgr, jsonpath, jsonpath_file);
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::init_jsonpath(SmallFileMgr *fileMgr, std::string& jsonpath, std::string& jsonpath_file) {
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonpath_doc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+                _parse_jsonpath_flag = -1;// failed, has none object
+            } else {
+                _parse_jsonpath_flag = 1;// success
+            }
+        } else {
+            _parse_jsonpath_flag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parse_jsonpath_flag = parse_jsonpath_from_file(fileMgr, jsonpath_file);
+    } else {
+        _parse_jsonpath_flag = 0;
+    }
+    return ;
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parse_jsonpath_from_file(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 3) {
+        LOG(WARNING)<< "parse_jsonpath_from_file Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[1]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[2], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonpath_file;
+    st = Env::Default()->new_random_access_file(file_path, &jsonpath_file);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonpath_file->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonpath_file->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonpath_doc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parse_json_doc(bool *eof) {
+    // read all, must be delete json_str
+    uint8_t* json_str = nullptr;
+    size_t length = 0;
+    RETURN_IF_ERROR(_file_reader->read_all(&json_str, &length));

Review comment:
       I think use OwnedSlice is better.

##########
File path: be/src/exec/json_scanner.cpp
##########
@@ -0,0 +1,518 @@
+// 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 "exec/json_scanner.h"
+#include <algorithm>
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;

Review comment:
       why minus 1 here?

##########
File path: be/src/exec/json_scanner.cpp
##########
@@ -0,0 +1,518 @@
+// 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 "exec/json_scanner.h"
+#include <algorithm>
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state, _counter, _state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+        _json_values(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::is_end() {
+    return _json_values->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::get_next() {
+    if (is_end()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        RuntimeState* state, ScannerCounter* counter,
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _state(state),
+            _counter(counter),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    init_jsonpath(fileMgr, jsonpath, jsonpath_file);
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::init_jsonpath(SmallFileMgr *fileMgr, std::string& jsonpath, std::string& jsonpath_file) {
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonpath_doc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+                _parse_jsonpath_flag = -1;// failed, has none object
+            } else {
+                _parse_jsonpath_flag = 1;// success
+            }
+        } else {
+            _parse_jsonpath_flag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parse_jsonpath_flag = parse_jsonpath_from_file(fileMgr, jsonpath_file);
+    } else {
+        _parse_jsonpath_flag = 0;
+    }
+    return ;
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parse_jsonpath_from_file(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {
+    std::vector<std::string> parts = strings::Split(fileinfo, ":", strings::SkipWhitespace());
+    if (parts.size() != 3) {
+        LOG(WARNING)<< "parse_jsonpath_from_file Invalid fileinfo: " << fileinfo;
+        return -1;
+    }
+    int64_t file_id = std::stol(parts[1]);
+    std::string file_path;
+    Status st = smallFileMgr->get_file(file_id, parts[2], &file_path);
+    if (!st.ok()) {
+        return -1;
+    }
+    std::unique_ptr<RandomAccessFile> jsonpath_file;
+    st = Env::Default()->new_random_access_file(file_path, &jsonpath_file);
+    if (!st.ok()) {
+        return -1;
+    }
+    uint64_t size = 0;
+    jsonpath_file->size(&size);
+    if (size == 0) {
+        return 0;
+    }
+    boost::scoped_array<char> pBuf(new char[size]);
+    Slice slice(pBuf.get(), size);
+    st = jsonpath_file->read_at(0, slice);
+    if  (!st.ok()) {
+        return -1;
+    }
+
+    if (!_jsonpath_doc.Parse(slice.get_data()).HasParseError()) {
+        if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+            return -1;//failed, has none object
+        } else {
+            return 1;// success
+        }
+    } else {
+        return -1;// parse failed
+    }
+}
+
+Status JsonReader::parse_json_doc(bool *eof) {

Review comment:
       ```suggestion
   Status JsonReader::parse_json_doc(bool* eof) {
   ```

##########
File path: be/src/exec/json_scanner.cpp
##########
@@ -0,0 +1,518 @@
+// 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 "exec/json_scanner.h"
+#include <algorithm>
+#include "gutil/strings/split.h"
+#include "runtime/exec_env.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/raw_value.h"
+#include "runtime/runtime_state.h"
+#include "exprs/expr.h"
+#include "env/env.h"
+#include "exec/local_file_reader.h"
+#include "exec/broker_reader.h"
+#include "exprs/json_functions.h"
+
+namespace doris {
+
+JsonScanner::JsonScanner(RuntimeState* state,
+                         RuntimeProfile* profile,
+                         const TBrokerScanRangeParams& params,
+                         const std::vector<TBrokerRangeDesc>& ranges,
+                         const std::vector<TNetworkAddress>& broker_addresses,
+                         ScannerCounter* counter) : BaseScanner(state, profile, params, counter),
+                          _ranges(ranges),
+                          _broker_addresses(broker_addresses),
+                          _cur_file_reader(nullptr),
+                          _next_range(0),
+                          _cur_file_eof(false),
+                          _scanner_eof(false) {
+
+}
+
+JsonScanner::~JsonScanner() {
+    close();
+}
+
+Status JsonScanner::open() {
+    return BaseScanner::open();
+}
+
+Status JsonScanner::get_next(Tuple* tuple, MemPool* tuple_pool, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    // Get one line
+    while (!_scanner_eof) {
+        if (_cur_file_reader == nullptr || _cur_file_eof) {
+            RETURN_IF_ERROR(open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+            _cur_file_eof = false;
+        }
+        RETURN_IF_ERROR(_cur_file_reader->read(_src_tuple, _src_slot_descs, tuple_pool, &_cur_file_eof));
+
+        if (_cur_file_eof) {
+            continue; // read next file
+        }
+        COUNTER_UPDATE(_rows_read_counter, 1);
+        SCOPED_TIMER(_materialize_timer);
+        if (fill_dest_tuple(Slice(), tuple, tuple_pool)) {
+            break;// break if true
+        }
+    }
+    if (_scanner_eof) {
+        *eof = true;
+    } else {
+        *eof = false;
+    }
+    return Status::OK();
+}
+
+Status JsonScanner::open_next_reader() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+    const TBrokerRangeDesc& range = _ranges[_next_range++];
+    int64_t start_offset = range.start_offset;
+    if (start_offset != 0) {
+        start_offset -= 1;
+    }
+    FileReader *file = NULL;
+    switch (range.file_type) {
+    case TFileType::FILE_LOCAL: {
+        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
+        RETURN_IF_ERROR(file_reader->open());
+        file = 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());
+        file = broker_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(3) << "unknown stream load id: " << UniqueId(range.load_id);
+            return Status::InternalError("unknown stream load id");
+        }
+        file = _stream_load_pipe.get();
+        break;
+    }
+    default: {
+        std::stringstream ss;
+        ss << "Unknown file type, type=" << range.file_type;
+        return Status::InternalError(ss.str());
+    }
+    }
+
+    std::string jsonpath = "";
+    std::string jsonpath_file = "";
+    if (range.__isset.jsonpath) {
+        jsonpath = range.jsonpath;
+    } else if (range.__isset.jsonpath_file) {
+        jsonpath_file = range.jsonpath_file;
+    }
+    _cur_file_reader = new JsonReader(_state, _counter, _state->exec_env()->small_file_mgr(), _profile, file, jsonpath, jsonpath_file);
+
+    return Status::OK();
+}
+
+void JsonScanner::close() {
+    if (_cur_file_reader != nullptr) {
+        delete _cur_file_reader;
+        _cur_file_reader = nullptr;
+        if (_stream_load_pipe != nullptr) {
+            _stream_load_pipe.reset();
+        }
+    }
+}
+
+////// class JsonDataInternal
+JsonDataInternal::JsonDataInternal(rapidjson::Value* v) :
+        _json_values(v), _iterator(v->Begin()) {
+}
+
+JsonDataInternal::~JsonDataInternal() {
+
+}
+bool JsonDataInternal::is_end() {
+    return _json_values->End() == _iterator;
+}
+
+rapidjson::Value::ConstValueIterator JsonDataInternal::get_next() {
+    if (is_end()) {
+        return nullptr;
+    }
+    return _iterator++;
+}
+
+
+////// class JsonReader
+JsonReader::JsonReader(
+        RuntimeState* state, ScannerCounter* counter,
+        SmallFileMgr *fileMgr,
+        RuntimeProfile* profile,
+        FileReader* file_reader,
+        std::string& jsonpath,
+        std::string& jsonpath_file) :
+            _next_line(0),
+            _total_lines(0),
+            _state(state),
+            _counter(counter),
+            _profile(profile),
+            _file_reader(file_reader),
+            _closed(false) {
+    _bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
+    _read_timer = ADD_TIMER(_profile, "FileReadTime");
+
+    init_jsonpath(fileMgr, jsonpath, jsonpath_file);
+}
+
+JsonReader::~JsonReader() {
+    close();
+}
+
+void JsonReader::init_jsonpath(SmallFileMgr *fileMgr, std::string& jsonpath, std::string& jsonpath_file) {
+    //parse jsonpath
+    if (!jsonpath.empty()) {
+        if (!_jsonpath_doc.Parse(jsonpath.c_str()).HasParseError()) {
+            if (!_jsonpath_doc.HasMember(JsonReader::JSON_PATH) || !_jsonpath_doc[JsonReader::JSON_PATH].IsArray()) {
+                _parse_jsonpath_flag = -1;// failed, has none object
+            } else {
+                _parse_jsonpath_flag = 1;// success
+            }
+        } else {
+            _parse_jsonpath_flag = -1;// parse failed
+        }
+    } else if (!jsonpath_file.empty()) {
+        //Read jsonpath from file, has format: file_id:md5
+        _parse_jsonpath_flag = parse_jsonpath_from_file(fileMgr, jsonpath_file);
+    } else {
+        _parse_jsonpath_flag = 0;
+    }
+    return ;
+}
+
+void JsonReader::close() {
+    if (_closed) {
+        return;
+    }
+    if (typeid(*_file_reader) == typeid(doris::BrokerReader) || typeid(*_file_reader) == typeid(doris::LocalFileReader)) {
+        _file_reader->close();
+        delete _file_reader;
+    }
+    _closed = true;
+}
+
+int JsonReader::parse_jsonpath_from_file(SmallFileMgr *smallFileMgr, std::string& fileinfo ) {

Review comment:
       ```suggestion
   int JsonReader::parse_jsonpath_from_file(SmallFileMgr *smallFileMgr, const std::string& fileinfo ) {
   ```




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

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