[Load] Support load json-data into Doris by RoutineLoad or StreamLoad (#3553)

Doris support load json-data by RoutineLoad or StreamLoad
This commit is contained in:
worker24h
2020-05-21 13:00:49 +08:00
committed by GitHub
parent 792307ae54
commit ef8fd1fcbe
42 changed files with 2473 additions and 191 deletions

View File

@ -93,6 +93,7 @@ set(EXEC_FILES
parquet_scanner.cpp
parquet_reader.cpp
orc_scanner.cpp
json_scanner.cpp
assert_num_rows_node.cpp
)

View File

@ -121,6 +121,11 @@ Status BrokerReader::open() {
return Status::OK();
}
//not support
Status BrokerReader::read_one_message(uint8_t** buf, size_t* length) {
return Status::NotSupported("Not support");
}
Status BrokerReader::read(uint8_t* buf, size_t* buf_len, bool* eof) {
DCHECK_NE(*buf_len, 0);
RETURN_IF_ERROR(readat(_cur_offset, (int64_t)*buf_len, (int64_t*)buf_len, buf));

View File

@ -52,6 +52,7 @@ public:
// 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_one_message(uint8_t** buf, size_t* length) override;
virtual int64_t size() override;
virtual Status seek(int64_t position) override;
virtual Status tell(int64_t* position) override;

View File

@ -27,6 +27,7 @@
#include "exec/broker_scanner.h"
#include "exec/parquet_scanner.h"
#include "exec/orc_scanner.h"
#include "exec/json_scanner.h"
#include "exprs/expr.h"
#include "util/runtime_profile.h"
@ -289,6 +290,14 @@ std::unique_ptr<BaseScanner> BrokerScanNode::create_scanner(const TBrokerScanRan
scan_range.broker_addresses,
counter);
break;
case TFileFormatType::FORMAT_JSON:
scan = new JsonScanner(_runtime_state,
runtime_profile(),
scan_range.params,
scan_range.ranges,
scan_range.broker_addresses,
counter);
break;
default:
scan = new BrokerScanner(
_runtime_state,

View File

@ -34,6 +34,7 @@
#include "exec/local_file_reader.h"
#include "exec/broker_reader.h"
#include "exec/decompressor.h"
#include "json_scanner.h"
#include "util/utf8_check.h"
namespace doris {
@ -180,6 +181,7 @@ Status BrokerScanner::create_decompressor(TFileFormatType::type type) {
CompressType compress_type;
switch (type) {
case TFileFormatType::FORMAT_CSV_PLAIN:
case TFileFormatType::FORMAT_JSON:
compress_type = CompressType::UNCOMPRESSED;
break;
case TFileFormatType::FORMAT_CSV_GZ:

View File

@ -34,6 +34,18 @@ public:
// 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;
/**
* This interface is used read a whole message, For example: read a message from kafka.
*
* 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_one_message(uint8_t** buf, size_t* length) = 0;
virtual int64_t size () = 0;
virtual Status seek(int64_t position) = 0;
virtual Status tell(int64_t* position) = 0;

View File

@ -0,0 +1,550 @@
// 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) {
break;
}
_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 = nullptr;
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 = "";
bool strip_outer_array = false;
if (range.__isset.jsonpaths) {
jsonpath = range.jsonpaths;
}
if (range.__isset.strip_outer_array) {
strip_outer_array = range.strip_outer_array;
}
_cur_file_reader = new JsonReader(_state, _counter, _profile, file, jsonpath, strip_outer_array);
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() {
}
rapidjson::Value::ConstValueIterator JsonDataInternal::get_next() {
if (_json_values->End() == _iterator) {
return nullptr;
}
return _iterator++;
}
////// class JsonReader
JsonReader::JsonReader(
RuntimeState* state, ScannerCounter* counter,
RuntimeProfile* profile,
FileReader* file_reader,
std::string& jsonpath,
bool strip_outer_array) :
_next_line(0),
_total_lines(0),
_state(state),
_counter(counter),
_profile(profile),
_file_reader(file_reader),
_closed(false),
_strip_outer_array(strip_outer_array) {
_bytes_read_counter = ADD_COUNTER(_profile, "BytesRead", TUnit::BYTES);
_read_timer = ADD_TIMER(_profile, "FileReadTime");
init_jsonpath(jsonpath);
}
JsonReader::~JsonReader() {
close();
}
void JsonReader::init_jsonpath(std::string& jsonpath) {
//parse jsonpath
if (!jsonpath.empty()) {
if (!_jsonpaths_doc.Parse(jsonpath.c_str()).HasParseError()) {
if (!_jsonpaths_doc.IsArray()) {
_parse_jsonpath_flag = -1;// failed, has none object
} else {
_parse_jsonpath_flag = 1;// success
}
} else {
_parse_jsonpath_flag = -1;// parse failed
}
} 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;
}
Status JsonReader::parse_json_doc(bool* eof) {
// read a whole message, must be delete json_str by `delete[]`
uint8_t* json_str = nullptr;
size_t length = 0;
RETURN_IF_ERROR(_file_reader->read_one_message(&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;
std::stringstream str_error;
str_error << "Parse json data for JsonDoc is failed. code = " << _json_doc.GetParseError()
<< ", error-info:" << rapidjson::GetParseError_En(_json_doc.GetParseError());
return Status::InternalError(str_error.str());
}
if (!_json_doc.IsArray() && _strip_outer_array) {
delete[] json_str;
return Status::InternalError("JSON ROOT node is array-object, `strip_outer_array` must be TRUE.");
}
delete[] json_str;
return Status::OK();
}
size_t JsonReader::get_data_by_jsonpath(const std::vector<SlotDescriptor*>& slot_descs) {
size_t max_lines = 0;
//iterator jsonpath to find object and save it to Map
_jmap.clear();
for (int i = 0; i < _jsonpaths_doc.Size(); i++) {
const rapidjson::Value& path = _jsonpaths_doc[i];
if (!path.IsString()) {
return -1;
}
// if jsonValues is null, because not match in jsondata.
rapidjson::Value* json_values = JsonFunctions::get_json_object_from_parsed_json(path.GetString(), &_json_doc, _json_doc.GetAllocator());
if (json_values == nullptr) {
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(slot_descs[i]->col_name(), 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 = nullptr;
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 = 0;
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;
}
}
}
if (nullcount == slot_descs.size()) {
_state->append_error_msg_to_file("", "The all fields is null, this is a invalid row.");
_counter->num_rows_filtered++;
*valid = false;
return Status::OK();
}
*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;
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.IsArray() ) {
_total_lines = _json_doc.Size();
} else {
_total_lines = 1; // only one row
}
_next_line = 0;
}
if (_json_doc.IsArray()) {//handle case 1
rapidjson::Value& objectValue = _json_doc[_next_line];// json object
RETURN_IF_ERROR(set_tuple_value(objectValue, tuple, slot_descs, tuple_pool, &valid));
} else {// handle case 2
RETURN_IF_ERROR(set_tuple_value(_json_doc, tuple, slot_descs, tuple_pool, &valid));
}
_next_line++;
if (!valid) {
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, const 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.";
_state->append_error_msg_to_file("", str_error.str());
_counter->num_rows_filtered++;
*valid = false; // current row is invalid
break;
}
} else {
RETURN_IF_ERROR(write_data_to_tuple(value, v, tuple, tuple_pool));
}
}
*valid = true;
return Status::OK();
}
Status JsonReader::handle_nest_complex_json(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool* eof) {
do {
bool valid = false;
if (_next_line >= _total_lines) {
RETURN_IF_ERROR(parse_json_doc(eof));
if (*eof) {
return Status::OK();
}
_total_lines = get_data_by_jsonpath(slot_descs);
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;
}
RETURN_IF_ERROR(set_tuple_value_from_map(tuple, slot_descs, tuple_pool, &valid));
_next_line++;
if (!valid) {// read a invalid row, then read next one
continue;
}
break; // read a valid row, then break
} while (_next_line <= _total_lines);
return Status::OK();
}
/**
* flat array for json
* For example:
* [{"colunm1":"value1", "colunm2":10}, {"colunm1":"value2", "colunm2":30}]
* Result:
* colunm1 colunm2
* ------------------
* value1 10
* value2 30
*/
Status JsonReader::handle_flat_array_complex_json(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool* eof) {
do {
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();
}
_total_lines = _json_doc.Size();
_next_line = 0;
}
int nullcount = 0;
bool valid = true;
size_t limit = std::min(slot_descs.size(), (size_t)_jsonpaths_doc.Size());
rapidjson::Value& objectValue = _json_doc[_next_line];
for (size_t i = 0; i < limit; i++) {
const rapidjson::Value& path = _jsonpaths_doc[i];
if (!path.IsString()) {
return Status::InternalError("Jsonpath is not string.");
}
// if jsonValues is null, because not match in jsondata.
rapidjson::Value* json_values = JsonFunctions::get_json_object_from_parsed_json(path.GetString(), &objectValue, _json_doc.GetAllocator());
if (json_values == nullptr) {
if (slot_descs[i]->is_nullable()) {
tuple->set_null(slot_descs[i]->null_indicator_offset());
nullcount++;
} else {
std::stringstream str_error;
str_error << "The column `" << slot_descs[i]->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;
}
} else {
RETURN_IF_ERROR(write_data_to_tuple(json_values, slot_descs[i], tuple, tuple_pool));
}
}
_next_line++;
if (!valid) {
continue;
}
if (nullcount == _jsonpaths_doc.Size()) {
_state->append_error_msg_to_file("", "The all fields is null, this is a invalid row.");
_counter->num_rows_filtered++;
continue;
}
break; // get a valid row, then break
} while (_next_line <= _total_lines);
return Status::OK();
}
Status JsonReader::handle_complex_json(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool* eof) {
if (_strip_outer_array) {
return handle_flat_array_complex_json(tuple, slot_descs, tuple_pool, eof);
} else {
return handle_nest_complex_json(tuple, slot_descs, tuple_pool, eof);
}
}
Status JsonReader::read(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool* eof) {
if (_parse_jsonpath_flag == -1) {
return Status::InternalError("Parse jsonpath is failed.");
} else if (_parse_jsonpath_flag == 0) {// input a simple json-string
return handle_simple_json(tuple, slot_descs, tuple_pool, eof);
} else {// input a complex json-string and a json-path
return handle_complex_json(tuple, slot_descs, tuple_pool, eof);
}
}
} // end of namespace

147
be/src/exec/json_scanner.h Normal file
View File

@ -0,0 +1,147 @@
// 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 <rapidjson/error/en.h>
#include <rapidjson/stringbuffer.h>
#include <rapidjson/writer.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 needed
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;
// 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:
rapidjson::Value* _json_values;
rapidjson::Value::ConstValueIterator _iterator;
};
class JsonReader {
public:
JsonReader(RuntimeState* state, ScannerCounter* counter, RuntimeProfile* profile, FileReader* file_reader,
std::string& jsonpath, bool strip_outer_array);
~JsonReader();
Status read(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool* eof);
private:
void init_jsonpath(std::string& jsonpath);
void fill_slot(Tuple* tuple, SlotDescriptor* slot_desc, MemPool* mem_pool, const uint8_t* value, int32_t len);
size_t get_data_by_jsonpath(const std::vector<SlotDescriptor*>& slot_descs);
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_nest_complex_json(Tuple* tuple, const std::vector<SlotDescriptor*>& slot_descs, MemPool* tuple_pool, bool* eof);
Status handle_flat_array_complex_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:
int _next_line;
int _total_lines;
RuntimeState* _state;
ScannerCounter* _counter;
RuntimeProfile* _profile;
FileReader*_file_reader;
bool _closed;
bool _strip_outer_array;
/**
* _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
*/
short _parse_jsonpath_flag;
RuntimeProfile::Counter* _bytes_read_counter;
RuntimeProfile::Counter* _read_timer;
rapidjson::Document _jsonpaths_doc;
rapidjson::Document _json_doc;
//key: column name
std::unordered_map<std::string, JsonDataInternal> _jmap;
};
} // end namesapce
#endif

View File

@ -52,6 +52,25 @@ bool LocalFileReader::closed() {
return _fp == nullptr;
}
// Read all bytes
Status LocalFileReader::read_one_message(uint8_t** buf, size_t* length) {
bool eof;
int64_t file_size = size() - _current_offset;
if (file_size <= 0) {
*buf = nullptr;
*length = 0;
return Status::OK();
}
*length = file_size;
*buf = new uint8_t[file_size];
read(*buf, length, &eof);
if (*length == 0) {
delete *buf;
*buf = nullptr;
}
return Status::OK();
}
Status LocalFileReader::read(uint8_t* buf, size_t* buf_len, bool* eof) {
readat(_current_offset, (int64_t)*buf_len, (int64_t*)buf_len, buf);
if (*buf_len == 0) {
@ -114,5 +133,4 @@ Status LocalFileReader::tell(int64_t* position) {
return Status::OK();
}
}

View File

@ -37,6 +37,7 @@ public:
// is set to zero.
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_one_message(uint8_t** buf, size_t* length) override;
virtual int64_t size () override;
virtual Status seek(int64_t position) override;
virtual Status tell(int64_t* position) override;

View File

@ -34,6 +34,7 @@
namespace doris {
ParquetScanner::ParquetScanner(RuntimeState* state,
RuntimeProfile* profile,
const TBrokerScanRangeParams& params,

View File

@ -43,7 +43,7 @@ namespace doris {
// static const re2::RE2 JSON_PATTERN("^([a-zA-Z0-9_\\-\\:\\s#\\|\\.]*)(?:\\[([0-9]+)\\])?");
// json path cannot contains: ", [, ]
static const re2::RE2 JSON_PATTERN("^([^\\\"\\[\\]]*)(?:\\[([0-9]+)\\])?");
static const re2::RE2 JSON_PATTERN("^([^\\\"\\[\\]]*)(?:\\[([0-9]+|\\*)\\])?");
void JsonFunctions::init() {
}
@ -58,7 +58,7 @@ IntVal JsonFunctions::get_json_int(
rapidjson::Document document;
rapidjson::Value* root =
get_json_object(context, json_string, path_string, JSON_FUN_INT, &document);
if (root->IsInt()) {
if (root != nullptr && root->IsInt()) {
return IntVal(root->GetInt());
} else {
return IntVal::null();
@ -76,7 +76,7 @@ StringVal JsonFunctions::get_json_string(
rapidjson::Document document;
rapidjson::Value* root =
get_json_object(context, json_string, path_string, JSON_FUN_STRING, &document);
if (root->IsNull()) {
if (root == nullptr || root->IsNull()) {
return StringVal::null();
} else if (root->IsString()) {
return AnyValUtil::from_string_temp(context, root->GetString());
@ -98,7 +98,9 @@ DoubleVal JsonFunctions::get_json_double(
rapidjson::Document document;
rapidjson::Value* root =
get_json_object(context, json_string, path_string, JSON_FUN_DOUBLE, &document);
if (root->IsInt()) {
if (root == nullptr || root->IsNull()) {
return DoubleVal::null();
} else if (root->IsInt()) {
return DoubleVal(static_cast<double>(root->GetInt()));
} else if (root->IsDouble()) {
return DoubleVal(root->GetDouble());
@ -107,6 +109,105 @@ DoubleVal JsonFunctions::get_json_double(
}
}
rapidjson::Value* JsonFunctions::match_value(std::vector<JsonPath>& parsed_paths, rapidjson::Value* document, rapidjson::Document::AllocatorType& mem_allocator, bool is_insert_null) {
rapidjson::Value* root = document;
rapidjson::Value* array_obj = nullptr;
for (int i = 1; i < parsed_paths.size(); i++) {
VLOG(10) << "parsed_paths: " << parsed_paths[i].debug_string();
if (root->IsNull()) {
return nullptr;
}
if (UNLIKELY(!parsed_paths[i].is_valid)) {
return nullptr;
}
std::string& col = parsed_paths[i].key;
int index = parsed_paths[i].idx;
if (LIKELY(!col.empty())) {
if (root->IsArray()) {
array_obj = static_cast<rapidjson::Value*>(
mem_allocator.Malloc(sizeof(rapidjson::Value)));
array_obj->SetArray();
bool is_null = true;
// if array ,loop the array,find out all Objects,then find the results from the objects
for (int j = 0; j < root->Size(); j++) {
rapidjson::Value* json_elem = &((*root)[j]);
if (json_elem->IsArray() || json_elem->IsNull()) {
continue;
} else {
if (!json_elem->IsObject()) {
continue;
}
if (!json_elem->HasMember(col.c_str())) {
if (is_insert_null) {//not found item, then insert a null object.
is_null = false;
rapidjson::Value nullObject(rapidjson::kNullType);
array_obj->PushBack(nullObject, mem_allocator);
}
continue;
}
rapidjson::Value* obj = &((*json_elem)[col.c_str()]);
if (obj->IsArray()) {
is_null = false;
for (int k = 0; k < obj->Size(); k++) {
array_obj->PushBack((*obj)[k], mem_allocator);
}
} else if (!obj->IsNull()) {
is_null = false;
array_obj->PushBack(*obj, mem_allocator);
}
}
}
root = is_null ? &(array_obj->SetNull()) : array_obj;
} else if (root->IsObject()){
if (!root->HasMember(col.c_str())) {
return nullptr;
} else {
root = &((*root)[col.c_str()]);
}
} else {
// root is not a nested type, return NULL
return nullptr;
}
}
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()) {
return nullptr;
} else if (index == -2) {
// [*]
array_obj = static_cast<rapidjson::Value*>(
mem_allocator.Malloc(sizeof(rapidjson::Value)));
array_obj->SetArray();
for (int j = 0; j < root->Size(); j++) {
rapidjson::Value v;
v.CopyFrom((*root)[j], mem_allocator);
array_obj->PushBack(v, mem_allocator);
}
root = array_obj;
} else if (index >= root->Size()) {
return nullptr;
} else {
root = &((*root)[index]);
}
} else {
return nullptr;
}
}
}
return root;
}
rapidjson::Value* JsonFunctions::get_json_object(
FunctionContext* context,
const std::string& json_string,
@ -158,82 +259,42 @@ rapidjson::Value* JsonFunctions::get_json_object(
document->SetNull();
return document;
}
return match_value(*parsed_paths, document, document->GetAllocator());
}
rapidjson::Value* root = document;
rapidjson::Value* array_obj = NULL;
for (int i = 1; i < (*parsed_paths).size(); i++) {
VLOG(10) << "parsed_paths: " << (*parsed_paths)[i].debug_string();
if (root->IsNull()) {
break;
}
rapidjson::Value* JsonFunctions::get_json_object_from_parsed_json (
const std::string& path_string,
rapidjson::Value* document,
rapidjson::Document::AllocatorType& mem_allocator) {
if (UNLIKELY(!(*parsed_paths)[i].is_valid)) {
root->SetNull();
}
// 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;
#ifndef BE_TEST
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, &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, &parsed_paths);
#endif
std::string& col = (*parsed_paths)[i].key;
int index = (*parsed_paths)[i].idx;
if (LIKELY(!col.empty())) {
if (root->IsArray()) {
array_obj = static_cast<rapidjson::Value*>(
document->GetAllocator().Malloc(sizeof(rapidjson::Value)));
array_obj->SetArray();
bool is_null = true;
VLOG(10) << "first parsed path: " << parsed_paths[0].debug_string();
// if array ,loop the array,find out all Objects,then find the results from the objects
for (int j = 0; j < root->Size(); j++) {
rapidjson::Value* json_elem = &((*root)[j]);
if (!parsed_paths[0].is_valid) {
return nullptr;
}
if (json_elem->IsArray() || json_elem->IsNull()) {
continue;
} else {
if (!json_elem->IsObject() || !json_elem->HasMember(col.c_str())) {
continue;
}
rapidjson::Value* obj = &((*json_elem)[col.c_str()]);
if (obj->IsArray()) {
is_null = false;
for (int k = 0; k < obj->Size(); k++) {
array_obj->PushBack((*obj)[k], document->GetAllocator());
}
} else if (!obj->IsNull()) {
is_null = false;
array_obj->PushBack(*obj, document->GetAllocator());
}
}
}
root = is_null ? &(array_obj->SetNull()) : array_obj;
} else if (root->IsObject()){
if (!root->HasMember(col.c_str())) {
root->SetNull();
} else {
root = &((*root)[col.c_str()]);
}
} else {
// root is not a nested type, return NULL
root->SetNull();
}
}
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()) {
root->SetNull();
} else {
root = &((*root)[index]);
}
} else {
root->SetNull();
}
}
}
return root;
rapidjson::Value* root = match_value(parsed_paths, document, mem_allocator, true);
if (root == document) {// not found
return nullptr;
}
return root;
}
void JsonFunctions::json_path_prepare(
@ -280,7 +341,11 @@ void JsonFunctions::get_parsed_paths(
} else {
int idx = -1;
if (!index.empty()) {
idx = atoi(index.c_str());
if (index == "*") {
idx = -2;
} else {
idx = atoi(index.c_str());
}
}
parsed_paths->emplace_back(col, idx, true);
}
@ -301,4 +366,3 @@ void JsonFunctions::json_path_close(
}
}

View File

@ -24,7 +24,11 @@
namespace doris {
enum JsonFunctionType {
JSON_FUN_INT = 0, JSON_FUN_DOUBLE, JSON_FUN_STRING
JSON_FUN_INT = 0,
JSON_FUN_DOUBLE,
JSON_FUN_STRING,
JSON_FUN_UNKOWN //The last
};
class Expr;
@ -32,20 +36,20 @@ class OpcodeRegistry;
class TupleRow;
struct JsonPath {
std::string key; // key of a json object
int idx; // array index of a json array, -1 means not set
bool is_valid; // true if the path is successfully parsed
std::string key; // key of a json object
int idx; // array index of a json array, -1 means not set
bool is_valid; // true if the path is successfully parsed
JsonPath(const std::string& key_, int idx_, bool is_valid_):
key(key_),
idx(idx_),
is_valid(is_valid_) {}
JsonPath(const std::string& key_, int idx_, bool is_valid_):
key(key_),
idx(idx_),
is_valid(is_valid_) {}
std::string debug_string() {
std::stringstream ss;
ss << "key: " << key << ", idx: " << idx << ", valid: " << is_valid;
return ss.str();
}
std::string debug_string() {
std::stringstream ss;
ss << "key: " << key << ", idx: " << idx << ", valid: " << is_valid;
return ss.str();
}
};
class JsonFunctions {
@ -62,22 +66,32 @@ public:
const doris_udf::StringVal& path);
static rapidjson::Value* get_json_object(
FunctionContext* context,
FunctionContext* context,
const std::string& json_string, const std::string& path_string,
const JsonFunctionType& fntype, rapidjson::Document* document);
static void json_path_prepare(
doris_udf::FunctionContext*,
doris_udf::FunctionContext::FunctionStateScope);
/**
* The `document` parameter must be has parsed.
*/
static rapidjson::Value* get_json_object_from_parsed_json(
const std::string& path_string,
rapidjson::Value* document,
rapidjson::Document::AllocatorType& mem_allocator);
static void json_path_close(
doris_udf::FunctionContext*,
doris_udf::FunctionContext::FunctionStateScope);
static void json_path_prepare(
doris_udf::FunctionContext*,
doris_udf::FunctionContext::FunctionStateScope);
static void json_path_close(
doris_udf::FunctionContext*,
doris_udf::FunctionContext::FunctionStateScope);
private:
static void get_parsed_paths(
const std::vector<std::string>& path_exprs,
std::vector<JsonPath>* parsed_paths);
static rapidjson::Value* match_value(std::vector<JsonPath>& parsed_paths,
rapidjson::Value* document, rapidjson::Document::AllocatorType& mem_allocator,
bool is_insert_null = false);
static void get_parsed_paths(
const std::vector<std::string>& path_exprs,
std::vector<JsonPath>* parsed_paths);
};
}
#endif

View File

@ -71,6 +71,8 @@ TStreamLoadPutResult k_stream_load_put_result;
static TFileFormatType::type parse_format(const std::string& format_str) {
if (boost::iequals(format_str, "CSV")) {
return TFileFormatType::FORMAT_CSV_PLAIN;
} else if (boost::iequals(format_str, "JSON")) {
return TFileFormatType::FORMAT_JSON;
}
return TFileFormatType::FORMAT_UNKNOWN;
}
@ -78,6 +80,7 @@ static TFileFormatType::type parse_format(const std::string& format_str) {
static bool is_format_support_streaming(TFileFormatType::type format) {
switch (format) {
case TFileFormatType::FORMAT_CSV_PLAIN:
case TFileFormatType::FORMAT_JSON:
return true;
default:
return false;
@ -362,7 +365,18 @@ Status StreamLoadAction::_process_put(HttpRequest* http_req, StreamLoadContext*
return Status::InvalidArgument("Invalid mem limit format");
}
}
if (!http_req->header(HTTP_EXEC_JSONPATHS).empty()) {
request.__set_jsonpaths(http_req->header(HTTP_EXEC_JSONPATHS));
}
if (!http_req->header(HTTP_EXEC_STRIP_OUTER_ARRAY).empty()) {
if (boost::iequals(http_req->header(HTTP_EXEC_STRIP_OUTER_ARRAY), "true")) {
request.__set_strip_outer_array(true);
} else {
request.__set_strip_outer_array(false);
}
} else {
request.__set_strip_outer_array(false);
}
if (ctx->timeout_second != -1) {
request.__set_timeout(ctx->timeout_second);
}

View File

@ -36,6 +36,8 @@ static const std::string HTTP_NEGATIVE = "negative";
static const std::string HTTP_STRICT_MODE = "strict_mode";
static const std::string HTTP_TIMEZONE = "timezone";
static const std::string HTTP_EXEC_MEM_LIMIT = "exec_mem_limit";
static const std::string HTTP_EXEC_JSONPATHS = "jsonpaths";
static const std::string HTTP_EXEC_STRIP_OUTER_ARRAY = "strip_outer_array";
static const std::string HTTP_100_CONTINUE = "100-continue";

View File

@ -14,10 +14,13 @@
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
#include "librdkafka/rdkafka.h"
#include "librdkafka/rdkafkacpp.h"
#include "runtime/routine_load/data_consumer.h"
#include "runtime/routine_load/data_consumer_group.h"
#include "runtime/routine_load/kafka_consumer_pipe.h"
#include "runtime/stream_load/stream_load_context.h"
namespace doris {
@ -101,6 +104,14 @@ Status KafkaDataConsumerGroup::start_all(StreamLoadContext* ctx) {
// copy one
std::map<int32_t, int64_t> cmt_offset = ctx->kafka_info->cmt_offset;
//improve performance
Status (KafkaConsumerPipe::*append_data)(const char* data, size_t size);
if (ctx->format == TFileFormatType::FORMAT_JSON) {
append_data = &KafkaConsumerPipe::append_json;
} else {
append_data = &KafkaConsumerPipe::append_with_line_delimiter;
}
MonotonicStopWatch watch;
watch.start();
Status st;
@ -112,6 +123,9 @@ Status KafkaDataConsumerGroup::start_all(StreamLoadContext* ctx) {
<< ", received rows=" << ctx->max_batch_rows - left_rows
<< ", received bytes=" << ctx->max_batch_size - left_bytes
<< ", eos: " << eos
<< ", left_time: " << left_time
<< ", left_rows: " << left_rows
<< ", left_bytes: " << left_bytes
<< ", blocking get time(us): " << _queue.total_get_wait_time() / 1000
<< ", blocking put time(us): " << _queue.total_put_wait_time() / 1000;
@ -152,9 +166,9 @@ Status KafkaDataConsumerGroup::start_all(StreamLoadContext* ctx) {
<< ", offset: " << msg->offset()
<< ", len: " << msg->len();
st = kafka_pipe->append_with_line_delimiter(
static_cast<const char *>(msg->payload()),
(kafka_pipe.get()->*append_data)(static_cast<const char *>(msg->payload()),
static_cast<size_t>(msg->len()));
if (st.ok()) {
left_rows--;
left_bytes -= msg->len();

View File

@ -37,7 +37,6 @@ public:
KafkaConsumerPipe(size_t max_buffered_bytes = 1024 * 1024,
size_t min_chunk_size = 64 * 1024)
: StreamLoadPipe(max_buffered_bytes, min_chunk_size) {
}
virtual ~KafkaConsumerPipe() {}
@ -52,6 +51,10 @@ public:
st = append("\n", 1);
return st;
}
Status append_json(const char* data, size_t size) {
return append_and_flush(data, size);
}
};
} // end namespace doris

View File

@ -106,7 +106,9 @@ Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) {
put_result.params = std::move(task.params);
put_result.__isset.params = true;
ctx->put_result = std::move(put_result);
if(task.__isset.format) {
ctx->format = task.format;
}
// the routine load task'txn has alreay began in FE.
// so it need to rollback if encounter error.
ctx->need_rollback = true;

View File

@ -137,6 +137,11 @@ public:
int64_t max_batch_rows = 100000;
int64_t max_batch_size = 100 * 1024 * 1024; // 100MB
// for parse json-data
std::string data_format = "";
std::string jsonpath_file = "";
std::string jsonpath = "";
// only used to check if we receive whole body
size_t body_bytes = 0;
size_t receive_bytes = 0;

View File

@ -45,6 +45,13 @@ public:
return Status::OK();
}
Status append_and_flush(const char* data, size_t size) {
ByteBufferPtr buf = ByteBuffer::allocate(BitUtil::RoundUpToPowerOfTwo(size+1));
buf->put_bytes(data, size);
buf->flip();
return _append(buf);
}
Status append(const char* data, size_t size) override {
size_t pos = 0;
if (_write_buf != nullptr) {
@ -60,7 +67,7 @@ public:
_write_buf.reset();
}
}
// need to allocate a new chunk
// need to allocate a new chunk, min chunk is 64k
size_t chunk_size = std::max(_min_chunk_size, size - pos);
chunk_size = BitUtil::RoundUpToPowerOfTwo(chunk_size);
_write_buf = ByteBuffer::allocate(chunk_size);
@ -77,6 +84,33 @@ public:
return _append(buf);
}
Status read_one_message(uint8_t** data, size_t* length) override {
std::unique_lock<std::mutex> l(_lock);
while (!_cancelled && !_finished && _buf_queue.empty()) {
_get_cond.wait(l);
}
// cancelled
if (_cancelled) {
return Status::InternalError("cancelled");
}
// finished
if (_buf_queue.empty()) {
DCHECK(_finished);
*data = nullptr;
*length = 0;
return Status::OK();
}
auto buf = _buf_queue.front();
*length = buf->remaining();
*data = new uint8_t[*length];
buf->get_bytes((char*)(*data) , *length);
_buf_queue.pop_front();
_buffered_bytes -= buf->limit;
_put_cond.notify_one();
return Status::OK();
}
Status read(uint8_t* data, size_t* data_size, bool* eof) override {
size_t bytes_read = 0;
while (bytes_read < *data_size) {
@ -181,6 +215,7 @@ private:
return Status::OK();
}
// Blocking queue
std::mutex _lock;
size_t _buffered_bytes;

View File

@ -35,6 +35,8 @@ set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/test/exec")
#ADD_BE_TEST(csv_scanner_test)
#ADD_BE_TEST(csv_scan_node_test)
# ADD_BE_TEST(csv_scan_bench_test)
ADD_BE_TEST(json_scanner_test)
ADD_BE_TEST(json_scanner_test_with_jsonpath)
ADD_BE_TEST(parquet_scanner_test)
ADD_BE_TEST(orc_scanner_test)
ADD_BE_TEST(plain_text_line_reader_uncompressed_test)

View File

@ -0,0 +1,445 @@
// 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/broker_scan_node.h"
#include <string>
#include <map>
#include <vector>
#include <gtest/gtest.h>
#include <time.h>
#include "common/object_pool.h"
#include "runtime/tuple.h"
#include "exec/local_file_reader.h"
#include "exprs/cast_functions.h"
#include "runtime/descriptors.h"
#include "runtime/exec_env.h"
#include "runtime/runtime_state.h"
#include "runtime/row_batch.h"
#include "runtime/user_function_cache.h"
#include "gen_cpp/Descriptors_types.h"
#include "gen_cpp/PlanNodes_types.h"
namespace doris {
class JsonSannerTest : public testing::Test {
public:
JsonSannerTest() : _runtime_state(TQueryGlobals()) {
init();
_runtime_state._instance_mem_tracker.reset(new MemTracker());
_runtime_state._exec_env = ExecEnv::GetInstance();
}
void init();
static void SetUpTestCase() {
UserFunctionCache::instance()->init("./be/test/runtime/test_data/user_function_cache/normal");
CastFunctions::init();
}
protected:
virtual void SetUp() {
}
virtual void TearDown() {
}
private:
int create_src_tuple(TDescriptorTable& t_desc_table, int next_slot_id);
int create_dst_tuple(TDescriptorTable& t_desc_table, int next_slot_id);
void create_expr_info();
void init_desc_table();
RuntimeState _runtime_state;
ObjectPool _obj_pool;
std::map<std::string, SlotDescriptor*> _slots_map;
TBrokerScanRangeParams _params;
DescriptorTbl* _desc_tbl;
TPlanNode _tnode;
};
#define TUPLE_ID_DST 0
#define TUPLE_ID_SRC 1
#define CLOMN_NUMBERS 4
#define DST_TUPLE_SLOT_ID_START 1
#define SRC_TUPLE_SLOT_ID_START 5
int JsonSannerTest::create_src_tuple(TDescriptorTable& t_desc_table, int next_slot_id) {
const char *clomnNames[] = {"category","author","title","price"};
for (int i = 0; i < CLOMN_NUMBERS; i++)
{
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 1;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = i;
slot_desc.byteOffset = i*16+8;
slot_desc.nullIndicatorByte = i/8;
slot_desc.nullIndicatorBit = i%8;
slot_desc.colName = clomnNames[i];
slot_desc.slotIdx = i + 1;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
{
// TTupleDescriptor source
TTupleDescriptor t_tuple_desc;
t_tuple_desc.id = TUPLE_ID_SRC;
t_tuple_desc.byteSize = CLOMN_NUMBERS*16+8;
t_tuple_desc.numNullBytes = 0;
t_tuple_desc.tableId = 0;
t_tuple_desc.__isset.tableId = true;
t_desc_table.tupleDescriptors.push_back(t_tuple_desc);
}
return next_slot_id;
}
int JsonSannerTest::create_dst_tuple(TDescriptorTable& t_desc_table, int next_slot_id) {
int32_t byteOffset = 8;
{//category
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 0;
slot_desc.byteOffset = byteOffset;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 0;
slot_desc.colName = "category";
slot_desc.slotIdx = 1;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
byteOffset += 16;
{// author
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 1;
slot_desc.byteOffset = byteOffset;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 1;
slot_desc.colName = "author";
slot_desc.slotIdx = 2;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
byteOffset += 16;
{// title
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 2;
slot_desc.byteOffset = byteOffset;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 2;
slot_desc.colName = "title";
slot_desc.slotIdx = 3;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
byteOffset += 16;
{// price
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::DOUBLE);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 3;
slot_desc.byteOffset = byteOffset;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 3;
slot_desc.colName = "price";
slot_desc.slotIdx = 4;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
byteOffset += 8;
t_desc_table.__isset.slotDescriptors = true;
{
// TTupleDescriptor dest
TTupleDescriptor t_tuple_desc;
t_tuple_desc.id = TUPLE_ID_DST;
t_tuple_desc.byteSize = byteOffset+8;
t_tuple_desc.numNullBytes = 0;
t_tuple_desc.tableId = 0;
t_tuple_desc.__isset.tableId = true;
t_desc_table.tupleDescriptors.push_back(t_tuple_desc);
}
return next_slot_id;
}
void JsonSannerTest::init_desc_table() {
TDescriptorTable t_desc_table;
// table descriptors
TTableDescriptor t_table_desc;
t_table_desc.id = 0;
t_table_desc.tableType = TTableType::BROKER_TABLE;
t_table_desc.numCols = 0;
t_table_desc.numClusteringCols = 0;
t_desc_table.tableDescriptors.push_back(t_table_desc);
t_desc_table.__isset.tableDescriptors = true;
int next_slot_id = 1;
next_slot_id = create_dst_tuple(t_desc_table, next_slot_id);
next_slot_id = create_src_tuple(t_desc_table, next_slot_id);
DescriptorTbl::create(&_obj_pool, t_desc_table, &_desc_tbl);
_runtime_state.set_desc_tbl(_desc_tbl);
}
void JsonSannerTest::create_expr_info() {
TTypeDesc varchar_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(5000);
node.__set_scalar_type(scalar_type);
varchar_type.types.push_back(node);
}
// category VARCHAR --> VARCHAR
{
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = SRC_TUPLE_SLOT_ID_START; // category id in src tuple
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(DST_TUPLE_SLOT_ID_START, expr);
_params.src_slot_ids.push_back(SRC_TUPLE_SLOT_ID_START);
}
// author VARCHAR --> VARCHAR
{
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = SRC_TUPLE_SLOT_ID_START+1; // author id in src tuple
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(DST_TUPLE_SLOT_ID_START+1, expr);
_params.src_slot_ids.push_back(SRC_TUPLE_SLOT_ID_START+1);
}
// title VARCHAR --> VARCHAR
{
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = SRC_TUPLE_SLOT_ID_START+2; // log_time id in src tuple
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(DST_TUPLE_SLOT_ID_START+2, expr);
_params.src_slot_ids.push_back(SRC_TUPLE_SLOT_ID_START+2);
}
// price VARCHAR --> DOUBLE
{
TTypeDesc int_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::BIGINT);
node.__set_scalar_type(scalar_type);
int_type.types.push_back(node);
}
TExprNode cast_expr;
cast_expr.node_type = TExprNodeType::CAST_EXPR;
cast_expr.type = int_type;
cast_expr.__set_opcode(TExprOpcode::CAST);
cast_expr.__set_num_children(1);
cast_expr.__set_output_scale(-1);
cast_expr.__isset.fn = true;
cast_expr.fn.name.function_name = "casttodouble";
cast_expr.fn.binary_type = TFunctionBinaryType::BUILTIN;
cast_expr.fn.arg_types.push_back(varchar_type);
cast_expr.fn.ret_type = int_type;
cast_expr.fn.has_var_args = false;
cast_expr.fn.__set_signature("casttodouble(VARCHAR(*))");
cast_expr.fn.__isset.scalar_fn = true;
cast_expr.fn.scalar_fn.symbol = "doris::CastFunctions::cast_to_double_val";
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = SRC_TUPLE_SLOT_ID_START + 3; // price id in src tuple
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(cast_expr);
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(DST_TUPLE_SLOT_ID_START + 3, expr);
_params.src_slot_ids.push_back(SRC_TUPLE_SLOT_ID_START + 3);
}
// _params.__isset.expr_of_dest_slot = true;
_params.__set_dest_tuple_id(TUPLE_ID_DST);
_params.__set_src_tuple_id(TUPLE_ID_SRC);
}
void JsonSannerTest::init() {
create_expr_info();
init_desc_table();
// Node Id
_tnode.node_id = 0;
_tnode.node_type = TPlanNodeType::SCHEMA_SCAN_NODE;
_tnode.num_children = 0;
_tnode.limit = -1;
_tnode.row_tuples.push_back(0);
_tnode.nullable_tuples.push_back(false);
_tnode.broker_scan_node.tuple_id = 0;
_tnode.__isset.broker_scan_node = true;
}
TEST_F(JsonSannerTest, normal_simple_arrayjson) {
BrokerScanNode scan_node(&_obj_pool, _tnode, *_desc_tbl);
auto status = scan_node.prepare(&_runtime_state);
ASSERT_TRUE(status.ok());
// set scan range
std::vector<TScanRangeParams> scan_ranges;
{
TScanRangeParams scan_range_params;
TBrokerScanRange broker_scan_range;
broker_scan_range.params = _params;
TBrokerRangeDesc range;
range.start_offset = 0;
range.size = -1;
range.format_type = TFileFormatType::FORMAT_JSON;
range.splittable = true;
range.path = "./be/test/exec/test_data/json_scanner/test_simple2.json";
range.file_type = TFileType::FILE_LOCAL;
broker_scan_range.ranges.push_back(range);
scan_range_params.scan_range.__set_broker_scan_range(broker_scan_range);
scan_ranges.push_back(scan_range_params);
}
scan_node.set_scan_ranges(scan_ranges);
status = scan_node.open(&_runtime_state);
ASSERT_TRUE(status.ok());
MemTracker tracker;
// Get batch
RowBatch batch(scan_node.row_desc(), _runtime_state.batch_size(), &tracker);
bool eof = false;
status = scan_node.get_next(&_runtime_state, &batch, &eof);
ASSERT_TRUE(status.ok());
ASSERT_EQ(2, batch.num_rows());
ASSERT_FALSE(eof);
batch.reset();
status = scan_node.get_next(&_runtime_state, &batch, &eof);
ASSERT_TRUE(status.ok());
ASSERT_EQ(0, batch.num_rows());
ASSERT_TRUE(eof);
scan_node.close(&_runtime_state);
{
std::stringstream ss;
scan_node.runtime_profile()->pretty_print(&ss);
LOG(INFO) << ss.str();
}
}
}
int main(int argc, char** argv) {
::testing::InitGoogleTest(&argc, argv);
doris::CpuInfo::init();
return RUN_ALL_TESTS();
}

View File

@ -0,0 +1,426 @@
// 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/broker_scan_node.h"
#include <string>
#include <map>
#include <vector>
#include <gtest/gtest.h>
#include <time.h>
#include "common/object_pool.h"
#include "runtime/tuple.h"
#include "exec/local_file_reader.h"
#include "exprs/cast_functions.h"
#include "runtime/descriptors.h"
#include "runtime/exec_env.h"
#include "runtime/runtime_state.h"
#include "runtime/row_batch.h"
#include "runtime/user_function_cache.h"
#include "gen_cpp/Descriptors_types.h"
#include "gen_cpp/PlanNodes_types.h"
namespace doris {
class JsonSannerTest : public testing::Test {
public:
JsonSannerTest() : _runtime_state(TQueryGlobals()) {
init();
_runtime_state._instance_mem_tracker.reset(new MemTracker());
_runtime_state._exec_env = ExecEnv::GetInstance();
}
void init();
static void SetUpTestCase() {
UserFunctionCache::instance()->init("./be/test/runtime/test_data/user_function_cache/normal");
CastFunctions::init();
}
protected:
virtual void SetUp() {
}
virtual void TearDown() {
}
private:
int create_src_tuple(TDescriptorTable& t_desc_table, int next_slot_id);
int create_dst_tuple(TDescriptorTable& t_desc_table, int next_slot_id);
void create_expr_info();
void init_desc_table();
RuntimeState _runtime_state;
ObjectPool _obj_pool;
std::map<std::string, SlotDescriptor*> _slots_map;
TBrokerScanRangeParams _params;
DescriptorTbl* _desc_tbl;
TPlanNode _tnode;
};
#define TUPLE_ID_DST 0
#define TUPLE_ID_SRC 1
#define CLOMN_NUMBERS 4
#define DST_TUPLE_SLOT_ID_START 1
#define SRC_TUPLE_SLOT_ID_START 5
int JsonSannerTest::create_src_tuple(TDescriptorTable& t_desc_table, int next_slot_id) {
const char *clomnNames[] = {"k1", "kind", "ip", "value"};
for (int i = 0; i < CLOMN_NUMBERS; i++)
{
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 1;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = i;
slot_desc.byteOffset = i*16+8;
slot_desc.nullIndicatorByte = i/8;
slot_desc.nullIndicatorBit = i%8;
slot_desc.colName = clomnNames[i];
slot_desc.slotIdx = i + 1;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
{
// TTupleDescriptor source
TTupleDescriptor t_tuple_desc;
t_tuple_desc.id = TUPLE_ID_SRC;
t_tuple_desc.byteSize = CLOMN_NUMBERS*16+8;
t_tuple_desc.numNullBytes = 0;
t_tuple_desc.tableId = 0;
t_tuple_desc.__isset.tableId = true;
t_desc_table.tupleDescriptors.push_back(t_tuple_desc);
}
return next_slot_id;
}
int JsonSannerTest::create_dst_tuple(TDescriptorTable& t_desc_table, int next_slot_id) {
int32_t byteOffset = 8;
{//k1
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 0;
slot_desc.byteOffset = byteOffset;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 0;
slot_desc.colName = "k1";
slot_desc.slotIdx = 1;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
byteOffset += 16;
{//kind
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 1;
slot_desc.byteOffset = byteOffset;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 1;
slot_desc.colName = "kind";
slot_desc.slotIdx = 1;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
byteOffset += 16;
{// ip
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 2;
slot_desc.byteOffset = byteOffset;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 2;
slot_desc.colName = "ip";
slot_desc.slotIdx = 3;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
byteOffset += 16;
{// value
TSlotDescriptor slot_desc;
slot_desc.id = next_slot_id++;
slot_desc.parent = 0;
TTypeDesc type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(65535);
node.__set_scalar_type(scalar_type);
type.types.push_back(node);
}
slot_desc.slotType = type;
slot_desc.columnPos = 3;
slot_desc.byteOffset = byteOffset;
slot_desc.nullIndicatorByte = 0;
slot_desc.nullIndicatorBit = 3;
slot_desc.colName = "value";
slot_desc.slotIdx = 4;
slot_desc.isMaterialized = true;
t_desc_table.slotDescriptors.push_back(slot_desc);
}
byteOffset += 16;
t_desc_table.__isset.slotDescriptors = true;
{
// TTupleDescriptor dest
TTupleDescriptor t_tuple_desc;
t_tuple_desc.id = TUPLE_ID_DST;
t_tuple_desc.byteSize = byteOffset + 8;
t_tuple_desc.numNullBytes = 0;
t_tuple_desc.tableId = 0;
t_tuple_desc.__isset.tableId = true;
t_desc_table.tupleDescriptors.push_back(t_tuple_desc);
}
return next_slot_id;
}
void JsonSannerTest::init_desc_table() {
TDescriptorTable t_desc_table;
// table descriptors
TTableDescriptor t_table_desc;
t_table_desc.id = 0;
t_table_desc.tableType = TTableType::BROKER_TABLE;
t_table_desc.numCols = 0;
t_table_desc.numClusteringCols = 0;
t_desc_table.tableDescriptors.push_back(t_table_desc);
t_desc_table.__isset.tableDescriptors = true;
int next_slot_id = 1;
next_slot_id = create_dst_tuple(t_desc_table, next_slot_id);
next_slot_id = create_src_tuple(t_desc_table, next_slot_id);
DescriptorTbl::create(&_obj_pool, t_desc_table, &_desc_tbl);
_runtime_state.set_desc_tbl(_desc_tbl);
}
void JsonSannerTest::create_expr_info() {
TTypeDesc varchar_type;
{
TTypeNode node;
node.__set_type(TTypeNodeType::SCALAR);
TScalarType scalar_type;
scalar_type.__set_type(TPrimitiveType::VARCHAR);
scalar_type.__set_len(5000);
node.__set_scalar_type(scalar_type);
varchar_type.types.push_back(node);
}
// k1 VARCHAR --> VARCHAR
{
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = SRC_TUPLE_SLOT_ID_START; // k1 id in src tuple
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(DST_TUPLE_SLOT_ID_START, expr);
_params.src_slot_ids.push_back(SRC_TUPLE_SLOT_ID_START);
}
// kind VARCHAR --> VARCHAR
{
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = SRC_TUPLE_SLOT_ID_START+1; // kind id in src tuple
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(DST_TUPLE_SLOT_ID_START+1, expr);
_params.src_slot_ids.push_back(SRC_TUPLE_SLOT_ID_START+1);
}
// ip VARCHAR --> VARCHAR
{
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = SRC_TUPLE_SLOT_ID_START+2; // ip id in src tuple
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(DST_TUPLE_SLOT_ID_START+2, expr);
_params.src_slot_ids.push_back(SRC_TUPLE_SLOT_ID_START+2);
}
// value VARCHAR --> VARCHAR
{
TExprNode slot_ref;
slot_ref.node_type = TExprNodeType::SLOT_REF;
slot_ref.type = varchar_type;
slot_ref.num_children = 0;
slot_ref.__isset.slot_ref = true;
slot_ref.slot_ref.slot_id = SRC_TUPLE_SLOT_ID_START+3; // valuep id in src tuple
slot_ref.slot_ref.tuple_id = 1;
TExpr expr;
expr.nodes.push_back(slot_ref);
_params.expr_of_dest_slot.emplace(DST_TUPLE_SLOT_ID_START+3, expr);
_params.src_slot_ids.push_back(SRC_TUPLE_SLOT_ID_START+3);
}
// _params.__isset.expr_of_dest_slot = true;
_params.__set_dest_tuple_id(TUPLE_ID_DST);
_params.__set_src_tuple_id(TUPLE_ID_SRC);
}
void JsonSannerTest::init() {
create_expr_info();
init_desc_table();
// Node Id
_tnode.node_id = 0;
_tnode.node_type = TPlanNodeType::SCHEMA_SCAN_NODE;
_tnode.num_children = 0;
_tnode.limit = -1;
_tnode.row_tuples.push_back(0);
_tnode.nullable_tuples.push_back(false);
_tnode.broker_scan_node.tuple_id = 0;
_tnode.__isset.broker_scan_node = true;
}
TEST_F(JsonSannerTest, normal) {
BrokerScanNode scan_node(&_obj_pool, _tnode, *_desc_tbl);
auto status = scan_node.prepare(&_runtime_state);
ASSERT_TRUE(status.ok());
// set scan range
std::vector<TScanRangeParams> scan_ranges;
{
TScanRangeParams scan_range_params;
TBrokerScanRange broker_scan_range;
broker_scan_range.params = _params;
TBrokerRangeDesc range;
range.start_offset = 0;
range.size = -1;
range.format_type = TFileFormatType::FORMAT_JSON;
range.splittable = true;
range.__isset.strip_outer_array = true;
range.strip_outer_array = true;
range.__isset.jsonpaths = true;
range.jsonpaths = "[\"$.k1\", \"$.kind\", \"$.keyname.ip\", \"$.keyname.value\"]";
range.path = "./be/test/exec/test_data/json_scanner/test_array.json";
range.file_type = TFileType::FILE_LOCAL;
broker_scan_range.ranges.push_back(range);
scan_range_params.scan_range.__set_broker_scan_range(broker_scan_range);
scan_ranges.push_back(scan_range_params);
}
scan_node.set_scan_ranges(scan_ranges);
status = scan_node.open(&_runtime_state);
ASSERT_TRUE(status.ok());
MemTracker tracker;
// Get batch
RowBatch batch(scan_node.row_desc(), _runtime_state.batch_size(), &tracker);
bool eof = false;
status = scan_node.get_next(&_runtime_state, &batch, &eof);
ASSERT_TRUE(status.ok());
ASSERT_EQ(2, batch.num_rows());
ASSERT_FALSE(eof);
batch.reset();
status = scan_node.get_next(&_runtime_state, &batch, &eof);
ASSERT_TRUE(status.ok());
ASSERT_EQ(0, batch.num_rows());
ASSERT_TRUE(eof);
scan_node.close(&_runtime_state);
{
std::stringstream ss;
scan_node.runtime_profile()->pretty_print(&ss);
LOG(INFO) << ss.str();
}
}
}
int main(int argc, char** argv) {
::testing::InitGoogleTest(&argc, argv);
doris::CpuInfo::init();
return RUN_ALL_TESTS();
}

View File

@ -0,0 +1,4 @@
[
{"k1":"v1", "kind":"server", "keyname":{"ip":"10.10.0.1", "value":"20"}},
{"k1":"v1-1", "kind":"server", "keyname":{"ip":"10.20.10.1", "value":"20"}}
]

View File

@ -0,0 +1,5 @@
[
{"category":"reference","author":"NigelRees","title":"SayingsoftheCentury","price":8.95},
{"category":"fiction","author":"EvelynWaugh","title":"SwordofHonour","price":12.99}
]

View File

@ -137,7 +137,7 @@ TEST_F(JsonFunctionTest, int)
rapidjson::Document document3_1;
rapidjson::Value* res3_1 = JsonFunctions::get_json_object(nullptr, json_string3, path_string3_1,
JSON_FUN_INT, &document3_1);
ASSERT_TRUE(res3_1->IsNull());
ASSERT_TRUE(res3_1 == nullptr);
std::string path_string3_2("$.couponFee");
rapidjson::Document document3_2;
@ -186,10 +186,101 @@ TEST_F(JsonFunctionTest, special_char)
ASSERT_EQ(std::string(res3->GetString()), "v1");
}
TEST_F(JsonFunctionTest, json_path1)
{
std::string json_raw_data("[{\"k1\":\"v1\", \"keyname\":{\"ip\":\"10.10.0.1\", \"value\":20}},{\"k1\":\"v1-1\", \"keyname\":{\"ip\":\"10.20.10.1\", \"value\":20}}]");
rapidjson::Document jsonDoc;
if (jsonDoc.Parse(json_raw_data.c_str()).HasParseError()) {
ASSERT_TRUE(false);
}
rapidjson::Value* res3;
res3 = JsonFunctions::get_json_object_from_parsed_json("$.[*].keyname.ip", &jsonDoc, jsonDoc.GetAllocator());
ASSERT_TRUE(res3->IsArray());
for (int i = 0; i < res3->Size(); i++) {
std::cout<< (*res3)[i].GetString() << std::endl;
}
res3 = JsonFunctions::get_json_object_from_parsed_json("$.[*].k1", &jsonDoc, jsonDoc.GetAllocator());
ASSERT_TRUE(res3->IsArray());
for (int i = 0; i < res3->Size(); i++) {
std::cout<< (*res3)[i].GetString() << std::endl;
}
}
TEST_F(JsonFunctionTest, json_path_get_nullobject)
{
std::string json_raw_data("[{\"a\":\"a1\", \"b\":\"b1\", \"c\":\"c1\"},{\"a\":\"a2\", \"c\":\"c2\"},{\"a\":\"a3\", \"b\":\"b3\", \"c\":\"c3\"}]");
rapidjson::Document jsonDoc;
if (jsonDoc.Parse(json_raw_data.c_str()).HasParseError()) {
ASSERT_TRUE(false);
}
rapidjson::Value* res3 = JsonFunctions::get_json_object_from_parsed_json("$.[*].b", &jsonDoc, jsonDoc.GetAllocator());
ASSERT_TRUE(res3->IsArray());
ASSERT_EQ(res3->Size(), 3);
for (int i = 0; i < res3->Size(); i++) {
if ((*res3)[i].GetType() == rapidjson::Type::kNullType) {
std::cout<< "null " ;
} else {
std::cout<< (*res3)[i].GetString() << " ";
}
}
std::cout<< " " << std::endl;
}
TEST_F(JsonFunctionTest, json_path_test)
{
{
std::string json_raw_data("[{\"a\":\"a1\", \"b\":\"b1\"}, {\"a\":\"a2\", \"b\":\"b2\"}]");
rapidjson::Document jsonDoc;
if (jsonDoc.Parse(json_raw_data.c_str()).HasParseError()) {
ASSERT_TRUE(false);
}
rapidjson::Value* res3 = JsonFunctions::get_json_object_from_parsed_json("$.[*].a", &jsonDoc, jsonDoc.GetAllocator());
ASSERT_TRUE(res3->IsArray());
ASSERT_EQ(res3->Size(), 2);
for (int i = 0; i < res3->Size(); i++) {
if ((*res3)[i].GetType() == rapidjson::Type::kNullType) {
std::cout<< "null " ;
} else {
std::cout<< (*res3)[i].GetString() << " ";
}
}
std::cout<< " " << std::endl;
}
{
std::string json_raw_data("{\"a\":[\"a1\",\"a2\"], \"b\":[\"b1\",\"b2\"]}");
rapidjson::Document jsonDoc;
if (jsonDoc.Parse(json_raw_data.c_str()).HasParseError()) {
ASSERT_TRUE(false);
}
rapidjson::Value* res3 = JsonFunctions::get_json_object_from_parsed_json("$.a", &jsonDoc, jsonDoc.GetAllocator());
ASSERT_TRUE(res3->IsArray());
ASSERT_EQ(res3->Size(), 2);
for (int i = 0; i < res3->Size(); i++) {
if ((*res3)[i].GetType() == rapidjson::Type::kNullType) {
std::cout<< "null " ;
} else {
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";
if (!doris::config::init(conffile.c_str(), false)) {
fprintf(stderr, "error read config file. \n");
return -1;

View File

@ -105,7 +105,7 @@ FROM data_source
Example:
`PARTITION(p1, p2, p3)`
4. job_properties
A generic parameter that specifies a routine load job.
@ -163,10 +163,21 @@ FROM data_source
Whether to enable strict mode, the default is on. If turned on, the column type transformation of non-null raw data is filtered if the result is NULL. Specified as "strict_mode" = "true"
            
5. timezone
5. `timezone`
Specifies the time zone in which the job will be loaded. The default by using session variable's timezone. This parameter affects all function results related to the time zone involved in the load.
6. `format`
Specifies the format of the imported data. Support csv and json, the default is csv.
7. `jsonpaths`
There are two ways to import json: simple mode and matched mode. If jsonpath is set, it will be the matched mode import, otherwise it will be the simple mode import, please refer to the example for details.
8. `strip_outer_array`
Boolean type, true to indicate that json data starts with an array object and flattens objects in the array object, default value is false.
5. data_source
The type of data source. Current support:
@ -366,6 +377,90 @@ FROM data_source
);
```
4. Create a Kafka routine load task named test1 for the example_tbl of example_db. The load data is a simple json.
```
CREATE ROUTINE LOAD example_db.test_json_label_1 ON table1
COLUMNS(category,price,author)
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"
);
```
It support two kinds data style:
1){"category":"a9jadhx","author":"test","price":895}
2)[
{"category":"a9jadhx","author":"test","price":895},
{"category":"axdfa1","author":"EvelynWaugh","price":1299}
]
5. Matched load json by jsonpaths.
```
CREATE TABLE `example_tbl` (
`category` varchar(24) NULL COMMENT "",
`author` varchar(24) NULL COMMENT "",
`timestamp` bigint(20) NULL COMMENT "",
`dt` int(11) NULL COMMENT "",
`price` double REPLACE
) ENGINE=OLAP
AGGREGATE KEY(`category`,`author`,`timestamp`,`dt`)
COMMENT "OLAP"
PARTITION BY RANGE(`dt`)
(PARTITION p0 VALUES [("-2147483648"), ("20200509")),
PARTITION p20200509 VALUES [("20200509"), ("20200510")),
PARTITION p20200510 VALUES [("20200510"), ("20200511")),
PARTITION p20200511 VALUES [("20200511"), ("20200512")))
DISTRIBUTED BY HASH(`category`,`author`,`timestamp`) BUCKETS 4
PROPERTIES (
"storage_type" = "COLUMN",
"replication_num" = "1"
);
CREATE ROUTINE LOAD example_db.test1 ON example_tbl
COLUMNS(category, author, price, timestamp, dt=from_unixtime(timestamp, '%Y%m%d'))
PROPERTIES
(
"desired_concurrent_number"="3",
"max_batch_interval" = "20",
"max_batch_rows" = "300000",
"max_batch_size" = "209715200",
"strict_mode" = "false",
"format" = "json",
"jsonpaths" = "[\"$.category\",\"$.author\",\"$.price\",\"$.timestamp\"]",
"strip_outer_array" = "true"
)
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"
);
```
For example json data:
[
{"category":"11","title":"SayingsoftheCentury","price":895,"timestamp":1589191587},
{"category":"22","author":"2avc","price":895,"timestamp":1589191487},
{"category":"33","author":"3avc","title":"SayingsoftheCentury","timestamp":1589191387}
]
Tips:
1)If the json data starts as an array and each object in the array is a record, you need to set the strip_outer_array to true to represent the flat array.
2)If the json data starts with an array, and each object in the array is a record, our ROOT node is actually an object in the array when we set jsonpath.
## keyword
CREATE, ROUTINE, LOAD

View File

@ -111,6 +111,15 @@ Specifies the time zone used for this load. The default is East Eight District.
Memory limit. Default is 2GB. Unit is Bytes.
`format`
Specifies the format of the imported data. Support csv and json, the default is csv.
`jsonpaths`
There are two ways to import json: simple mode and matched mode. If jsonpath is set, it will be the matched mode import, otherwise it will be the simple mode import, please refer to the example for details.
`strip_outer_array`
Boolean type, true to indicate that json data starts with an array object and flattens objects in the array object, default value is false.
RETURN VALUES
After the load is completed, the related content of this load will be returned in Json format. Current field included
@ -187,6 +196,34 @@ Where url is the url given by ErrorURL.
```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. a simple load json
table schema:
`category` varchar(512) NULL COMMENT "",
`author` varchar(512) NULL COMMENT "",
`title` varchar(512) NULL COMMENT "",
`price` double NULL COMMENT ""
json data:
{"category":"C++","author":"avc","title":"C++ primer","price":895}
load command by curl:
curl --location-trusted -u root -H "label:123" -H "format: json" -T testData http://host:port/api/testDb/testTbl/_stream_load
you can load multiple records, for example:
[
{"category":"C++","author":"avc","title":"C++ primer","price":89.5},
{"category":"Java","author":"avc","title":"Effective Java","price":95},
{"category":"Linux","author":"avc","title":"Linux kernel","price":195}
]
11. Matched load json by jsonpaths
json data:
[
{"category":"xuxb111","author":"1avc","title":"SayingsoftheCentury","price":895},
{"category":"xuxb222","author":"2avc","title":"SayingsoftheCentury","price":895},
{"category":"xuxb333","author":"3avc","title":"SayingsoftheCentury","price":895}
]
Matched imports are made by specifying jsonpath parameter, such as `category`, `author`, and `price`, for example:
curl --location-trusted -u root -H "columns: category, price, author" -H "label:123" -H "format: json" -H "jsonpaths: [\"$.category\",\"$.price\",\"$.author\"]" -H "strip_outer_array: true" -T testData http://host:port/api/testDb/testTbl/_stream_load
Tips:
1)If the json data starts as an array and each object in the array is a record, you need to set the strip_outer_array to true to represent the flat array.
2)If the json data starts with an array, and each object in the array is a record, our ROOT node is actually an object in the array when we set jsonpath.
## keyword

View File

@ -41,7 +41,7 @@ under the License.
1. [db.]job_name
导入作业的名称,在同一个 database 内,相同名称只能有一个 job 在运行。
2. tbl_name
指定需要导入的表的名称。
@ -85,12 +85,12 @@ under the License.
COLUMNS (k2, k1, xxx, v1, v2 = k1 + k2);
3. where_predicates
用于指定过滤条件,以过滤掉不需要的列。过滤列可以是映射列或衍生列。
例如我们只希望导入 k1 大于 100 并且 k2 等于 1000 的列,则书写如下:
WHERE k1 > 100 and k2 = 1000
4. partitions
指定导入目的表的哪些 partition 中。如果不指定,则会自动导入到对应的 partition 中。
@ -98,7 +98,7 @@ under the License.
PARTITION(p1, p2, p3)
4. job_properties
4. job_properties
用于指定例行导入作业的通用参数。
语法:
@ -127,7 +127,7 @@ under the License.
这三个参数,用于控制一个子任务的执行时间和处理量。当任意一个达到阈值,则任务结束。
例:
"max_batch_interval" = "20",
"max_batch_rows" = "300000",
"max_batch_size" = "209715200"
@ -137,15 +137,27 @@ 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. jsonpaths
jsonpaths: 导入json方式分为:简单模式和匹配模式。如果设置了jsonpath则为匹配模式导入,否则为简单模式导入,具体可参考示例。
8. strip_outer_array
布尔类型,为true表示json数据以数组对象开始且将数组对象中进行展平,默认值是false。
5. data_source
数据源的类型。当前支持:
@ -156,7 +168,7 @@ under the License.
指定数据源相关的信息。
语法:
(
"key1" = "val1",
"key2" = "val2"
@ -168,7 +180,7 @@ under the License.
Kafka 的 broker 连接信息。格式为 ip:host。多个broker之间以逗号分隔。
示例:
"kafka_broker_list" = "broker1:9092,broker2:9092"
2. kafka_topic
@ -190,8 +202,8 @@ under the License.
示例:
"kafka_partitions" = "0,1,2,3",
"kafka_offsets" = "101,0,OFFSET_BEGINNING,OFFSET_END"
"kafka_offsets" = "101,0,OFFSET_BEGINNING,OFFSET_END"
4. property
指定自定义kafka参数。
@ -199,7 +211,7 @@ under the License.
当参数的 value 为一个文件时,需要在 value 前加上关键词:"FILE:"。
关于如何创建文件,请参阅 "HELP CREATE FILE;"
更多支持的自定义参数,请参阅 librdkafka 的官方 CONFIGURATION 文档中,client 端的配置项。
示例:
"property.client.id" = "12345",
"property.ssl.ca.location" = "FILE:ca.pem"
@ -214,7 +226,7 @@ under the License.
其中:
"property.security.protocol" 和 "property.ssl.ca.location" 为必须,用于指明连接方式为 SSL,以及 CA 证书的位置。
如果 Kafka server 端开启了 client 认证,则还需设置:
"property.ssl.certificate.location"
@ -222,8 +234,8 @@ under the License.
"property.ssl.key.password"
分别用于指定 client 的 public key,private key 以及 private key 的密码。
2.指定kafka partition的默认起始offset
如果没有指定kafka_partitions/kafka_offsets,默认消费所有分区,此时可以指定kafka_default_offsets指定起始 offset。默认为 OFFSET_END,即从末尾开始订阅。
值为
@ -309,6 +321,82 @@ under the License.
"property.ssl.key.password" = "abcdefg",
"property.client.id" = "my_client_id"
);
4. 简单模式导入json
CREATE ROUTINE LOAD example_db.test_json_label_1 ON table1
COLUMNS(category,price,author)
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"
);
支持两种json数据格式:
1){"category":"a9jadhx","author":"test","price":895}
2)[
{"category":"a9jadhx","author":"test","price":895},
{"category":"axdfa1","author":"EvelynWaugh","price":1299}
]
5. 精准导入json数据格式
CREATE TABLE `example_tbl` (
`category` varchar(24) NULL COMMENT "",
`author` varchar(24) NULL COMMENT "",
`timestamp` bigint(20) NULL COMMENT "",
`dt` int(11) NULL COMMENT "",
`price` double REPLACE
) ENGINE=OLAP
AGGREGATE KEY(`category`,`author`,`timestamp`,`dt`)
COMMENT "OLAP"
PARTITION BY RANGE(`dt`)
(PARTITION p0 VALUES [("-2147483648"), ("20200509")),
PARTITION p20200509 VALUES [("20200509"), ("20200510")),
PARTITION p20200510 VALUES [("20200510"), ("20200511")),
PARTITION p20200511 VALUES [("20200511"), ("20200512")))
DISTRIBUTED BY HASH(`category`,`author`,`timestamp`) BUCKETS 4
PROPERTIES (
"storage_type" = "COLUMN",
"replication_num" = "1"
);
CREATE ROUTINE LOAD example_db.test1 ON example_tbl
COLUMNS(category, author, price, timestamp, dt=from_unixtime(timestamp, '%Y%m%d'))
PROPERTIES
(
"desired_concurrent_number"="3",
"max_batch_interval" = "20",
"max_batch_rows" = "300000",
"max_batch_size" = "209715200",
"strict_mode" = "false",
"format" = "json",
"jsonpaths" = "[\"$.category\",\"$.author\",\"$.price\",\"$.timestamp\"]",
"strip_outer_array" = "true"
)
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"
);
json数据格式:
[
{"category":"11","title":"SayingsoftheCentury","price":895,"timestamp":1589191587},
{"category":"22","author":"2avc","price":895,"timestamp":1589191487},
{"category":"33","author":"3avc","title":"SayingsoftheCentury","timestamp":1589191387}
]
说明:
1)如果json数据是以数组开始,并且数组中每个对象是一条记录,则需要将strip_outer_array设置成true,表示展平数组。
2)如果json数据是以数组开始,并且数组中每个对象是一条记录,在设置jsonpath时,我们的ROOT节点实际上是数组中对象。
## keyword

View File

@ -28,7 +28,7 @@ under the License.
## description
NAME:
stream-load: load data to table in streaming
SYNOPSIS
curl --location-trusted -u user:passwd [-H ""...] -T data.file -XPUT http://fe_host:http_port/api/{db}/{table}/_stream_load
@ -36,28 +36,28 @@ under the License.
该语句用于向指定的 table 导入数据,与普通Load区别是,这种导入方式是同步导入。
这种导入方式仍然能够保证一批导入任务的原子性,要么全部数据导入成功,要么全部失败。
该操作会同时更新和此 base table 相关的 rollup table 的数据。
这是一个同步操作,整个数据导入工作完成后返回给用户导入结果。
这是一个同步操作,整个数据导入工作完成后返回给用户导入结果。
当前支持HTTP chunked与非chunked上传两种方式,对于非chunked方式,必须要有Content-Length来标示上传内容长度,这样能够保证数据的完整性。
另外,用户最好设置Expect Header字段内容100-continue,这样可以在某些出错场景下避免不必要的数据传输。
OPTIONS
用户可以通过HTTP的Header部分来传入导入参数
label: 一次导入的标签,相同标签的数据无法多次导入。用户可以通过指定Label的方式来避免一份数据重复导入的问题。
当前Palo内部保留30分钟内最近成功的label。
column_separator:用于指定导入文件中的列分隔符,默认为\t。如果是不可见字符,则需要加\x作为前缀,使用十六进制来表示分隔符。
如hive文件的分隔符\x01,需要指定为-H "column_separator:\x01"
columns:用于指定导入文件中的列和 table 中的列的对应关系。如果源文件中的列正好对应表中的内容,那么是不需要指定这个字段的内容的。
如果源文件与表schema不对应,那么需要这个字段进行一些数据转换。这里有两种形式column,一种是直接对应导入文件中的字段,直接使用字段名表示;
一种是衍生列,语法为 `column_name` = expression。举几个例子帮助理解。
例1: 表中有3个列“c1, c2, c3”,源文件中的三个列一次对应的是"c3,c2,c1"; 那么需要指定-H "columns: c3, c2, c1"
例2: 表中有3个列“c1, c2, c3", 源文件中前三列依次对应,但是有多余1列;那么需要指定-H "columns: c1, c2, c3, xxx";
例2: 表中有3个列“c1, c2, c3", 源文件中前三列依次对应,但是有多余1列;那么需要指定-H "columns: c1, c2, c3, xxx";
最后一个列随意指定个名称占位即可
例3: 表中有3个列“year, month, day"三个列,源文件中只有一个时间列,为”2018-06-01 01:02:03“格式;
那么可以指定-H "columns: col, year = year(col), month=month(col), day=day(col)"完成导入
where: 用于抽取部分数据。用户如果有需要将不需要的数据过滤掉,那么可以通过设定这个选项来达到。
例1: 只导入大于k1列等于20180601的数据,那么可以在导入时候指定-H "where: k1 = 20180601"
@ -67,13 +67,28 @@ under the License.
比如指定导入到p1, p2分区,-H "partitions: p1, p2"
timeout: 指定导入的超时时间。单位秒。默认是 600 秒。可设置范围为 1 秒 ~ 259200 秒。
strict_mode: 用户指定此次导入是否开启严格模式,默认为开启。关闭方式为 -H "strict_mode: false"。
timezone: 指定本次导入所使用的时区。默认为东八区。该参数会影响所有导入涉及的和时区有关的函数结果。
exec_mem_limit: 导入内存限制。默认为 2GB。单位为字节。
format: 指定导入数据格式,默认是csv,支持json格式。
jsonpaths: 导入json方式分为:简单模式和精准模式。
简单模式:没有设置jsonpaths参数即为简单模式,这种模式下要求json数据是对象类型,例如:
{"k1":1, "k2":2, "k3":"hello"},其中k1,k2,k3是列名字。
匹配模式:用于json数据相对复杂,需要通过jsonpaths参数匹配对应的value。
strip_outer_array: 布尔类型,为true表示json数据以数组对象开始且将数组对象中进行展平,默认值是false。例如:
[
{"k1" : 1, "v1" : 2},
{"k1" : 3, "v1" : 4}
]
当strip_outer_array为true,最后导入到doris中会生成两行数据。
RETURN VALUES
导入完成后,会以Json格式返回这次导入的相关内容。当前包括一下字段
Status: 导入最后的状态。
@ -90,28 +105,28 @@ under the License.
LoadBytes: 此次导入的源文件数据量大小
LoadTimeMs: 此次导入所用的时间
ErrorURL: 被过滤数据的具体内容,仅保留前1000条
ERRORS
可以通过以下语句查看导入错误详细信息:
SHOW LOAD WARNINGS ON 'url'
其中 url 为 ErrorURL 给出的 url。
## example
1. 将本地文件'testData'中的数据导入到数据库'testDb'中'testTbl'的表,使用Label用于去重。指定超时时间为 100 秒
curl --location-trusted -u root -H "label:123" -H "timeout:100" -T testData http://host:port/api/testDb/testTbl/_stream_load
2. 将本地文件'testData'中的数据导入到数据库'testDb'中'testTbl'的表,使用Label用于去重, 并且只导入k1等于20180601的数据
curl --location-trusted -u root -H "label:123" -H "where: k1=20180601" -T testData http://host:port/api/testDb/testTbl/_stream_load
3. 将本地文件'testData'中的数据导入到数据库'testDb'中'testTbl'的表, 允许20%的错误率(用户是defalut_cluster中的)
curl --location-trusted -u root -H "label:123" -H "max_filter_ratio:0.2" -T testData http://host:port/api/testDb/testTbl/_stream_load
4. 将本地文件'testData'中的数据导入到数据库'testDb'中'testTbl'的表, 允许20%的错误率,并且指定文件的列名(用户是defalut_cluster中的)
curl --location-trusted -u root -H "label:123" -H "max_filter_ratio:0.2" -H "columns: k2, k1, v1" -T testData http://host:port/api/testDb/testTbl/_stream_load
5. 将本地文件'testData'中的数据导入到数据库'testDb'中'testTbl'的表中的p1, p2分区, 允许20%的错误率。
curl --location-trusted -u root -H "label:123" -H "max_filter_ratio:0.2" -H "partitions: p1, p2" -T testData http://host:port/api/testDb/testTbl/_stream_load
@ -127,7 +142,35 @@ 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数据
表结构:
`category` varchar(512) NULL COMMENT "",
`author` varchar(512) NULL COMMENT "",
`title` varchar(512) NULL COMMENT "",
`price` double NULL COMMENT ""
json数据格式:
{"category":"C++","author":"avc","title":"C++ primer","price":895}
导入命令:
curl --location-trusted -u root -H "label:123" -H "format: json" -T testData http://host:port/api/testDb/testTbl/_stream_load
为了提升吞吐量,支持一次性导入条数据,json数据格式如下:
[
{"category":"C++","author":"avc","title":"C++ primer","price":89.5},
{"category":"Java","author":"avc","title":"Effective Java","price":95},
{"category":"Linux","author":"avc","title":"Linux kernel","price":195}
]
11. 匹配模式,导入json数据
json数据格式:
[
{"category":"xuxb111","author":"1avc","title":"SayingsoftheCentury","price":895},
{"category":"xuxb222","author":"2avc","title":"SayingsoftheCentury","price":895},
{"category":"xuxb333","author":"3avc","title":"SayingsoftheCentury","price":895}
]
通过指定jsonpath进行精准导入,例如只导入category、author、price三个属性
curl --location-trusted -u root -H "columns: category, price, author" -H "label:123" -H "format: json" -H "jsonpaths: [\"$.category\",\"$.price\",\"$.author\"]" -H "strip_outer_array: true" -T testData http://host:port/api/testDb/testTbl/_stream_load
说明:
1)如果json数据是以数组开始,并且数组中每个对象是一条记录,则需要将strip_outer_array设置成true,表示展平数组。
2)如果json数据是以数组开始,并且数组中每个对象是一条记录,在设置jsonpath时,我们的ROOT节点实际上是数组中对象。
## keyword
STREAM,LOAD

View File

@ -34,10 +34,11 @@ import java.util.Map;
import java.util.Optional;
public class CreateFileStmt extends DdlStmt {
public static final String PROP_CATALOG = "catalog";
public static final String PROP_URL = "url";
public static final String PROP_MD5 = "md5";
public static final String PROP_SAVE_CONTENT = "save_content";
public static final String PROP_CATALOG_DEFAULT = "DEFAULT";
private static final String PROP_CATALOG = "catalog";
private static final String PROP_URL = "url";
private static final String PROP_MD5 = "md5";
private static final String PROP_SAVE_CONTENT = "save_content";
private static final ImmutableSet<String> PROPERTIES_SET = new ImmutableSet.Builder<String>()
.add(PROP_CATALOG).add(PROP_URL).add(PROP_MD5).build();
@ -122,7 +123,7 @@ public class CreateFileStmt extends DdlStmt {
catalogName = properties.get(PROP_CATALOG);
if (Strings.isNullOrEmpty(catalogName)) {
throw new AnalysisException("catalog name is missing");
catalogName = PROP_CATALOG_DEFAULT;
}
downloadUrl = properties.get(PROP_URL);

View File

@ -17,6 +17,10 @@
package org.apache.doris.analysis;
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.doris.common.AnalysisException;
import org.apache.doris.common.Config;
import org.apache.doris.common.FeNameFormat;
@ -30,11 +34,6 @@ import org.apache.doris.load.routineload.LoadDataSourceType;
import org.apache.doris.load.routineload.RoutineLoadJob;
import org.apache.doris.qe.ConnectContext;
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import java.util.List;
import java.util.Map;
import java.util.Optional;
@ -90,6 +89,10 @@ public class CreateRoutineLoadStmt extends DdlStmt {
public static final String MAX_BATCH_ROWS_PROPERTY = "max_batch_rows";
public static final String MAX_BATCH_SIZE_PROPERTY = "max_batch_size";
public static final String FORMAT = "format";// the value is csv or json, default is csv
public static final String STRIP_OUTER_ARRAY = "strip_outer_array";
public static final String JSONPATHS = "jsonpaths";
// kafka type properties
public static final String KAFKA_BROKER_LIST_PROPERTY = "kafka_broker_list";
public static final String KAFKA_TOPIC_PROPERTY = "kafka_topic";
@ -107,6 +110,9 @@ public class CreateRoutineLoadStmt extends DdlStmt {
.add(MAX_BATCH_INTERVAL_SEC_PROPERTY)
.add(MAX_BATCH_ROWS_PROPERTY)
.add(MAX_BATCH_SIZE_PROPERTY)
.add(FORMAT)
.add(JSONPATHS)
.add(STRIP_OUTER_ARRAY)
.add(LoadStmt.STRICT_MODE)
.add(LoadStmt.TIMEZONE)
.build();
@ -137,6 +143,16 @@ public class CreateRoutineLoadStmt extends DdlStmt {
private long maxBatchSizeBytes = -1;
private boolean strictMode = true;
private String timezone = TimeUtils.DEFAULT_TIME_ZONE;
/**
* RoutineLoad support json data.
* Require Params:
* 1) dataFormat = "json"
* 2) jsonPaths = "$.XXX.xxx"
*/
private String format = ""; //default is csv.
private String jsonPaths = "";
private boolean stripOuterArray = false;
// kafka related properties
private String kafkaBrokerList;
@ -212,6 +228,18 @@ public class CreateRoutineLoadStmt extends DdlStmt {
return timezone;
}
public String getFormat() {
return format;
}
public boolean isStripOuterArray() {
return stripOuterArray;
}
public String getJsonPaths() {
return jsonPaths;
}
public String getKafkaBrokerList() {
return kafkaBrokerList;
}
@ -328,6 +356,21 @@ public class CreateRoutineLoadStmt extends DdlStmt {
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";
jsonPaths = jobProperties.get(JSONPATHS);
stripOuterArray = Boolean.valueOf(jobProperties.get(STRIP_OUTER_ARRAY));
} else {
throw new UserException("Format type is invalid. format=`" + format + "`");
}
} else {
format = "csv"; // default csv
}
}
private void checkDataSourceProperties() throws AnalysisException {

View File

@ -24,6 +24,7 @@ import org.apache.doris.common.MetaNotFoundException;
import org.apache.doris.common.UserException;
import org.apache.doris.common.util.DebugUtil;
import org.apache.doris.thrift.TExecPlanFragmentParams;
import org.apache.doris.thrift.TFileFormatType;
import org.apache.doris.thrift.TKafkaLoadInfo;
import org.apache.doris.thrift.TLoadSourceType;
import org.apache.doris.thrift.TPlanFragment;
@ -95,6 +96,11 @@ public class KafkaTaskInfo extends RoutineLoadTaskInfo {
tRoutineLoadTask.setMax_interval_s(routineLoadJob.getMaxBatchIntervalS());
tRoutineLoadTask.setMax_batch_rows(routineLoadJob.getMaxBatchRows());
tRoutineLoadTask.setMax_batch_size(routineLoadJob.getMaxBatchSizeBytes());
if (!routineLoadJob.getFormat().isEmpty() && routineLoadJob.getFormat().equalsIgnoreCase("json")) {
tRoutineLoadTask.setFormat(TFileFormatType.FORMAT_JSON);
} else {
tRoutineLoadTask.setFormat(TFileFormatType.FORMAT_CSV_PLAIN);
}
return tRoutineLoadTask;
}

View File

@ -17,6 +17,14 @@
package org.apache.doris.load.routineload;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import com.google.common.collect.EvictingQueue;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.gson.Gson;
import com.google.gson.GsonBuilder;
import org.apache.doris.analysis.ColumnSeparator;
import org.apache.doris.analysis.CreateRoutineLoadStmt;
import org.apache.doris.analysis.Expr;
@ -61,16 +69,6 @@ import org.apache.doris.transaction.AbstractTxnStateChangeCallback;
import org.apache.doris.transaction.TransactionException;
import org.apache.doris.transaction.TransactionState;
import org.apache.doris.transaction.TransactionStatus;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import com.google.common.collect.EvictingQueue;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.gson.Gson;
import com.google.gson.GsonBuilder;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@ -175,6 +173,16 @@ public abstract class RoutineLoadJob extends AbstractTxnStateChangeCallback impl
protected long maxBatchRows = DEFAULT_MAX_BATCH_ROWS;
protected long maxBatchSizeBytes = DEFAULT_MAX_BATCH_SIZE;
/**
* RoutineLoad support json data.
* Require Params:
* 1) format = "json"
* 2) jsonPath = "$.XXX.xxx"
*/
private static final String PROPS_FORMAT = "format";
private static final String PROPS_STRIP_OUTER_ARRAY = "strip_outer_array";
private static final String PROPS_JSONPATHS = "jsonpaths";
protected int currentTaskConcurrentNum;
protected RoutineLoadProgress progress;
@ -271,7 +279,25 @@ public abstract class RoutineLoadJob extends AbstractTxnStateChangeCallback impl
this.maxBatchSizeBytes = stmt.getMaxBatchSize();
}
jobProperties.put(LoadStmt.STRICT_MODE, String.valueOf(stmt.isStrictMode()));
jobProperties.put(LoadStmt.TIMEZONE, stmt.getTimezone());
if (Strings.isNullOrEmpty(stmt.getFormat()) || stmt.getFormat().equals("csv")) {
jobProperties.put(PROPS_FORMAT, "csv");
jobProperties.put(PROPS_STRIP_OUTER_ARRAY, "false");
jobProperties.put(PROPS_JSONPATHS, "");
} else if (stmt.getFormat().equals("json")) {
jobProperties.put(PROPS_FORMAT, "json");
if (!Strings.isNullOrEmpty(stmt.getJsonPaths())) {
jobProperties.put(PROPS_JSONPATHS, stmt.getJsonPaths());
} else {
jobProperties.put(PROPS_JSONPATHS, "");
}
if (stmt.isStripOuterArray()) {
jobProperties.put(PROPS_STRIP_OUTER_ARRAY, "true");
} else {
jobProperties.put(PROPS_STRIP_OUTER_ARRAY, "false");
}
} else {
throw new UserException("Invalid format type.");
}
}
private void setRoutineLoadDesc(RoutineLoadDesc routineLoadDesc) {
@ -424,6 +450,26 @@ public abstract class RoutineLoadJob extends AbstractTxnStateChangeCallback impl
return maxBatchSizeBytes;
}
public String getFormat() {
String value = jobProperties.get(PROPS_FORMAT);
if (value == null) {
return "csv";
}
return value;
}
public boolean isStripOuterArray() {
return Boolean.valueOf(jobProperties.get(PROPS_STRIP_OUTER_ARRAY));
}
public String getJsonPaths() {
String value = jobProperties.get(PROPS_JSONPATHS);
if (value == null) {
return "";
}
return value;
}
public int getSizeOfRoutineLoadTaskInfoList() {
readLock();
try {
@ -1161,7 +1207,11 @@ public abstract class RoutineLoadJob extends AbstractTxnStateChangeCallback impl
jobProperties.put("partitions", partitions == null ? STAR_STRING : Joiner.on(",").join(partitions.getPartitionNames()));
jobProperties.put("columnToColumnExpr", columnDescs == null ? STAR_STRING : Joiner.on(",").join(columnDescs));
jobProperties.put("whereExpr", whereExpr == null ? STAR_STRING : whereExpr.toSql());
jobProperties.put("columnSeparator", columnSeparator == null ? "\t" : columnSeparator.toString());
if (getFormat().equalsIgnoreCase("json")) {
jobProperties.put("dataFormat", "json");
} else {
jobProperties.put("columnSeparator", columnSeparator == null ? "\t" : columnSeparator.toString());
}
jobProperties.put("maxErrorNum", String.valueOf(maxErrorNum));
jobProperties.put("maxBatchIntervalS", String.valueOf(maxBatchIntervalS));
jobProperties.put("maxBatchRows", String.valueOf(maxBatchRows));

View File

@ -17,6 +17,8 @@
package org.apache.doris.planner;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.doris.analysis.Analyzer;
import org.apache.doris.analysis.ArithmeticExpr;
import org.apache.doris.analysis.Expr;
@ -41,15 +43,12 @@ import org.apache.doris.thrift.TBrokerScanNode;
import org.apache.doris.thrift.TBrokerScanRange;
import org.apache.doris.thrift.TBrokerScanRangeParams;
import org.apache.doris.thrift.TExplainLevel;
import org.apache.doris.thrift.TFileFormatType;
import org.apache.doris.thrift.TPlanNode;
import org.apache.doris.thrift.TPlanNodeType;
import org.apache.doris.thrift.TScanRange;
import org.apache.doris.thrift.TScanRangeLocations;
import org.apache.doris.thrift.TUniqueId;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@ -97,6 +96,12 @@ public class StreamLoadScanNode extends LoadScanNode {
TBrokerRangeDesc rangeDesc = new TBrokerRangeDesc();
rangeDesc.file_type = streamLoadTask.getFileType();
rangeDesc.format_type = streamLoadTask.getFormatType();
if (rangeDesc.format_type == TFileFormatType.FORMAT_JSON) {
if (!streamLoadTask.getJsonPaths().isEmpty()) {
rangeDesc.setJsonpaths(streamLoadTask.getJsonPaths());
}
rangeDesc.setStrip_outer_array(streamLoadTask.isStripOuterArray());
}
rangeDesc.splittable = false;
switch (streamLoadTask.getFileType()) {
case FILE_LOCAL:

View File

@ -17,8 +17,6 @@
package org.apache.doris.service;
import static org.apache.doris.thrift.TStatusCode.NOT_IMPLEMENTED_ERROR;
import org.apache.doris.analysis.SetType;
import org.apache.doris.analysis.UserIdentity;
import org.apache.doris.catalog.Catalog;
@ -103,9 +101,9 @@ import org.apache.doris.thrift.TUpdateExportTaskStatusRequest;
import org.apache.doris.thrift.TUpdateMiniEtlTaskStatusRequest;
import org.apache.doris.transaction.TabletCommitInfo;
import org.apache.doris.transaction.TransactionState;
import org.apache.doris.transaction.TxnCommitAttachment;
import org.apache.doris.transaction.TransactionState.TxnSourceType;
import org.apache.doris.transaction.TransactionState.TxnCoordinator;
import org.apache.doris.transaction.TransactionState.TxnSourceType;
import org.apache.doris.transaction.TxnCommitAttachment;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
@ -122,6 +120,8 @@ import java.net.UnknownHostException;
import java.util.List;
import java.util.Map;
import static org.apache.doris.thrift.TStatusCode.NOT_IMPLEMENTED_ERROR;
// Frontend service used to serve all request for this frontend through
// thrift protocol
public class FrontendServiceImpl implements FrontendService.Iface {
@ -863,7 +863,7 @@ public class FrontendServiceImpl implements FrontendService.Iface {
if (!(table instanceof OlapTable)) {
throw new UserException("load table type is not OlapTable, type=" + table.getClass());
}
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request);
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request, db);
StreamLoadPlanner planner = new StreamLoadPlanner(db, (OlapTable) table, streamLoadTask);
TExecPlanFragmentParams plan = planner.plan(streamLoadTask.getId());
// add table indexes to transaction state

View File

@ -17,6 +17,7 @@
package org.apache.doris.task;
import com.google.common.collect.Lists;
import org.apache.doris.analysis.ColumnSeparator;
import org.apache.doris.analysis.Expr;
import org.apache.doris.analysis.ImportColumnDesc;
@ -25,6 +26,7 @@ import org.apache.doris.analysis.ImportWhereStmt;
import org.apache.doris.analysis.PartitionNames;
import org.apache.doris.analysis.SqlParser;
import org.apache.doris.analysis.SqlScanner;
import org.apache.doris.catalog.Database;
import org.apache.doris.common.AnalysisException;
import org.apache.doris.common.Config;
import org.apache.doris.common.UserException;
@ -35,9 +37,6 @@ import org.apache.doris.thrift.TFileFormatType;
import org.apache.doris.thrift.TFileType;
import org.apache.doris.thrift.TStreamLoadPutRequest;
import org.apache.doris.thrift.TUniqueId;
import com.google.common.collect.Lists;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@ -52,6 +51,8 @@ public class StreamLoadTask {
private long txnId;
private TFileType fileType;
private TFileFormatType formatType;
private boolean stripOuterArray;
private String jsonPaths;
// optional
private List<ImportColumnDesc> columnExprDescs = Lists.newArrayList();
@ -70,6 +71,8 @@ public class StreamLoadTask {
this.txnId = txnId;
this.fileType = fileType;
this.formatType = formatType;
this.jsonPaths = "";
this.stripOuterArray = false;
}
public TUniqueId getId() {
@ -124,14 +127,30 @@ public class StreamLoadTask {
return timeout;
}
public static StreamLoadTask fromTStreamLoadPutRequest(TStreamLoadPutRequest request) throws UserException {
public boolean isStripOuterArray() {
return stripOuterArray;
}
public void setStripOuterArray(boolean stripOuterArray) {
this.stripOuterArray = stripOuterArray;
}
public String getJsonPaths() {
return jsonPaths;
}
public void setJsonPath(String jsonPaths) {
this.jsonPaths = jsonPaths;
}
public static StreamLoadTask fromTStreamLoadPutRequest(TStreamLoadPutRequest request, Database db) throws UserException {
StreamLoadTask streamLoadTask = new StreamLoadTask(request.getLoadId(), request.getTxnId(),
request.getFileType(), request.getFormatType());
streamLoadTask.setOptionalFromTSLPutRequest(request);
streamLoadTask.setOptionalFromTSLPutRequest(request, db);
return streamLoadTask;
}
private void setOptionalFromTSLPutRequest(TStreamLoadPutRequest request) throws UserException {
private void setOptionalFromTSLPutRequest(TStreamLoadPutRequest request, Database db) throws UserException {
if (request.isSetColumns()) {
setColumnToColumnExpr(request.getColumns());
}
@ -171,12 +190,22 @@ public class StreamLoadTask {
if (request.isSetExecMemLimit()) {
execMemLimit = request.getExecMemLimit();
}
if (request.getFormatType() == TFileFormatType.FORMAT_JSON) {
if (request.getJsonpaths() != null) {
jsonPaths = request.getJsonpaths();
}
stripOuterArray = request.isStrip_outer_array();
}
}
public static StreamLoadTask fromRoutineLoadJob(RoutineLoadJob routineLoadJob) {
TUniqueId dummyId = new TUniqueId();
TFileFormatType fileFormatType = TFileFormatType.FORMAT_CSV_PLAIN;
if (routineLoadJob.getFormat().equals("json")) {
fileFormatType = TFileFormatType.FORMAT_JSON;
}
StreamLoadTask streamLoadTask = new StreamLoadTask(dummyId, -1L /* dummy txn id*/,
TFileType.FILE_STREAM, TFileFormatType.FORMAT_CSV_PLAIN);
TFileType.FILE_STREAM, fileFormatType);
streamLoadTask.setOptionalFromRoutineLoadJob(routineLoadJob);
return streamLoadTask;
}
@ -193,6 +222,10 @@ public class StreamLoadTask {
strictMode = routineLoadJob.isStrictMode();
timezone = routineLoadJob.getTimezone();
timeout = (int) routineLoadJob.getMaxBatchIntervalS() * 2;
if (!routineLoadJob.getJsonPaths().isEmpty()) {
jsonPaths = routineLoadJob.getJsonPaths();
}
stripOuterArray = routineLoadJob.isStripOuterArray();
}
// used for stream load

View File

@ -87,7 +87,7 @@ public class StreamLoadPlannerTest {
request.setLoadId(new TUniqueId(2, 3));
request.setFileType(TFileType.FILE_STREAM);
request.setFormatType(TFileFormatType.FORMAT_CSV_PLAIN);
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request);
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request, null);
StreamLoadPlanner planner = new StreamLoadPlanner(db, destTable, streamLoadTask);
planner.plan(streamLoadTask.getId());
}
@ -104,4 +104,4 @@ public class StreamLoadPlannerTest {
ImportWhereStmt whereStmt = (ImportWhereStmt) SqlParserUtils.getFirstStmt(parser);
Assert.assertTrue(whereStmt.getExpr() instanceof CompoundPredicate);
}
}
}

View File

@ -132,7 +132,7 @@ public class StreamLoadScanNodeTest {
private StreamLoadScanNode getStreamLoadScanNode(TupleDescriptor dstDesc, TStreamLoadPutRequest request)
throws UserException {
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request);
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request, null);
StreamLoadScanNode scanNode = new StreamLoadScanNode(streamLoadTask.getId(), new PlanNodeId(1), dstDesc, dstTable, streamLoadTask);
return scanNode;
}
@ -195,7 +195,7 @@ public class StreamLoadScanNodeTest {
TStreamLoadPutRequest request = getBaseRequest();
request.setColumns("k1, k2, v1");
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request);
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request, null);
StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request);
scanNode.init(analyzer);
@ -225,7 +225,7 @@ public class StreamLoadScanNodeTest {
TStreamLoadPutRequest request = getBaseRequest();
request.setColumns("k1 k2 v1");
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request);
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request, null);
StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request);
scanNode.init(analyzer);
@ -271,7 +271,7 @@ public class StreamLoadScanNodeTest {
TStreamLoadPutRequest request = getBaseRequest();
request.setColumns("k1,k2,v1, v2=k2");
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request);
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request, null);
StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request);
scanNode.init(analyzer);
scanNode.finalize(analyzer);
@ -319,7 +319,7 @@ public class StreamLoadScanNodeTest {
TStreamLoadPutRequest request = getBaseRequest();
request.setFileType(TFileType.FILE_STREAM);
request.setColumns("k1,k2, v1=hll_hash(k2)");
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request);
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request, null);
StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request);
scanNode.init(analyzer);
@ -374,7 +374,7 @@ public class StreamLoadScanNodeTest {
TStreamLoadPutRequest request = getBaseRequest();
request.setFileType(TFileType.FILE_LOCAL);
request.setColumns("k1,k2, v1=hll_hash1(k2)");
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request);
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request, null);
StreamLoadScanNode scanNode = getStreamLoadScanNode(dstDesc, request);
scanNode.init(analyzer);
@ -560,7 +560,7 @@ public class StreamLoadScanNodeTest {
TStreamLoadPutRequest request = getBaseRequest();
request.setColumns("k1,k2,v1, v2=k2");
request.setWhere("k1 1");
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request);
StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request, null);
StreamLoadScanNode scanNode = new StreamLoadScanNode(streamLoadTask.getId(), new PlanNodeId(1), dstDesc, dstTable,
streamLoadTask);

View File

@ -20,6 +20,7 @@ namespace java org.apache.doris.thrift
include "Status.thrift"
include "Types.thrift"
include "PlanNodes.thrift"
include "AgentService.thrift"
include "PaloInternalService.thrift"
include "DorisExternalService.thrift"
@ -59,6 +60,7 @@ struct TRoutineLoadTask {
11: optional i64 max_batch_size
12: optional TKafkaLoadInfo kafka_load_info
13: optional PaloInternalService.TExecPlanFragmentParams params
14: optional PlanNodes.TFileFormatType format
}
struct TKafkaMetaProxyRequest {

View File

@ -547,6 +547,8 @@ struct TStreamLoadPutRequest {
20: optional string timezone
21: optional i64 execMemLimit
22: optional bool isTempPartition
23: optional bool strip_outer_array
24: optional string jsonpaths
}
struct TStreamLoadPutResult {

View File

@ -103,7 +103,8 @@ enum TFileFormatType {
FORMAT_CSV_LZOP,
FORMAT_PARQUET,
FORMAT_CSV_DEFLATE,
FORMAT_ORC
FORMAT_ORC,
FORMAT_JSON
}
// One broker range information.
@ -125,6 +126,9 @@ struct TBrokerRangeDesc {
9: optional i32 num_of_columns_from_file
// columns parsed from file path should be after the columns read from file
10: optional list<string> columns_from_path
// it's usefull when format_type == FORMAT_JSON
11: optional bool strip_outer_array;
12: optional string jsonpaths;
}
struct TBrokerScanRangeParams {