Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions be/src/io/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ set(IO_FILES
fs/hdfs_file_reader.cpp
fs/broker_file_system.cpp
fs/broker_file_reader.cpp
fs/stream_load_pipe_reader.cpp
cache/dummy_file_cache.cpp
cache/file_cache.cpp
cache/file_cache_manager.cpp
Expand Down
209 changes: 209 additions & 0 deletions be/src/io/fs/stream_load_pipe_reader.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,209 @@
// 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 "stream_load_pipe_reader.h"

#include <gen_cpp/internal_service.pb.h>

#include "runtime/thread_context.h"
#include "util/bit_util.h"

namespace doris {
namespace io {
StreamLoadPipeReader::StreamLoadPipeReader(size_t max_buffered_bytes, size_t min_chunk_size,
int64_t total_length, bool use_proto)
: _buffered_bytes(0),
_proto_buffered_bytes(0),
_max_buffered_bytes(max_buffered_bytes),
_min_chunk_size(min_chunk_size),
_use_proto(use_proto) {}

StreamLoadPipeReader::~StreamLoadPipeReader() {
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(ExecEnv::GetInstance()->orphan_mem_tracker());
while (!_buf_queue.empty()) {
_buf_queue.pop_front();
}
}

Status StreamLoadPipeReader::read_at(size_t /*offset*/, Slice result, const IOContext& /*io_ctx*/,
size_t* bytes_read) {
SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(ExecEnv::GetInstance()->orphan_mem_tracker());
*bytes_read = 0;
size_t bytes_req = result.size;
char* to = result.data;
if (UNLIKELY(bytes_req == 0)) {
return Status::OK();
}
while (*bytes_read < bytes_req) {
std::unique_lock<std::mutex> l(_lock);
while (!_cancelled && !_finished && _buf_queue.empty()) {
_get_cond.wait(l);
}
// cancelled
if (_cancelled) {
return Status::InternalError("cancelled: {}", _cancelled_reason);
}
// finished
if (_buf_queue.empty()) {
DCHECK(_finished);
// break the while loop
bytes_req = *bytes_read;
return Status::OK();
}
auto buf = _buf_queue.front();
int64_t copy_size = std::min(bytes_req - *bytes_read, buf->remaining());
buf->get_bytes(to + *bytes_read, copy_size);
*bytes_read += copy_size;
if (!buf->has_remaining()) {
_buf_queue.pop_front();
_buffered_bytes -= buf->limit;
_put_cond.notify_one();
}
}
DCHECK(*bytes_read == bytes_req)
<< "*bytes_read=" << *bytes_read << ", bytes_req=" << bytes_req;
return Status::OK();
}

Status StreamLoadPipeReader::append_and_flush(const char* data, size_t size,
size_t proto_byte_size) {
ByteBufferPtr buf = ByteBuffer::allocate(BitUtil::RoundUpToPowerOfTwo(size + 1));
buf->put_bytes(data, size);
buf->flip();
return _append(buf, proto_byte_size);
}

Status StreamLoadPipeReader::append(const char* data, size_t size) {
size_t pos = 0;
if (_write_buf != nullptr) {
if (size < _write_buf->remaining()) {
_write_buf->put_bytes(data, size);
return Status::OK();
} else {
pos = _write_buf->remaining();
_write_buf->put_bytes(data, pos);

_write_buf->flip();
RETURN_IF_ERROR(_append(_write_buf));
_write_buf.reset();
}
}
// 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);
_write_buf->put_bytes(data + pos, size - pos);
return Status::OK();
}

Status StreamLoadPipeReader::append(const ByteBufferPtr& buf) {
if (_write_buf != nullptr) {
_write_buf->flip();
RETURN_IF_ERROR(_append(_write_buf));
_write_buf.reset();
}
return _append(buf);
}

// read the next buffer from _buf_queue
Status StreamLoadPipeReader::_read_next_buffer(std::unique_ptr<uint8_t[]>* data, int64_t* length) {
std::unique_lock<std::mutex> l(_lock);
while (!_cancelled && !_finished && _buf_queue.empty()) {
_get_cond.wait(l);
}
// cancelled
if (_cancelled) {
return Status::InternalError("cancelled: {}", _cancelled_reason);
}
// finished
if (_buf_queue.empty()) {
DCHECK(_finished);
data->reset();
*length = 0;
return Status::OK();
}
auto buf = _buf_queue.front();
*length = buf->remaining();
data->reset(new uint8_t[*length]);
buf->get_bytes((char*)(data->get()), *length);
_buf_queue.pop_front();
_buffered_bytes -= buf->limit;
if (_use_proto) {
PDataRow** ptr = reinterpret_cast<PDataRow**>(data->get());
_proto_buffered_bytes -= (sizeof(PDataRow*) + (*ptr)->GetCachedSize());
}
_put_cond.notify_one();
return Status::OK();
}

Status StreamLoadPipeReader::_append(const ByteBufferPtr& buf, size_t proto_byte_size) {
{
std::unique_lock<std::mutex> l(_lock);
// if _buf_queue is empty, we append this buf without size check
if (_use_proto) {
while (!_cancelled && !_buf_queue.empty() &&
(_proto_buffered_bytes + proto_byte_size > _max_buffered_bytes)) {
_put_cond.wait(l);
}
} else {
while (!_cancelled && !_buf_queue.empty() &&
_buffered_bytes + buf->remaining() > _max_buffered_bytes) {
_put_cond.wait(l);
}
}
if (_cancelled) {
return Status::InternalError("cancelled: {}", _cancelled_reason);
}
_buf_queue.push_back(buf);
if (_use_proto) {
_proto_buffered_bytes += proto_byte_size;
} else {
_buffered_bytes += buf->remaining();
}
}
_get_cond.notify_one();
return Status::OK();
}

// called when producer finished
Status StreamLoadPipeReader::finish() {
if (_write_buf != nullptr) {
_write_buf->flip();
_append(_write_buf);
_write_buf.reset();
}
{
std::lock_guard<std::mutex> l(_lock);
_finished = true;
}
_get_cond.notify_all();
return Status::OK();
}

// called when producer/consumer failed
void StreamLoadPipeReader::cancel(const std::string& reason) {
{
std::lock_guard<std::mutex> l(_lock);
_cancelled = true;
_cancelled_reason = reason;
}
_get_cond.notify_all();
_put_cond.notify_all();
}

} // namespace io
} // namespace doris
97 changes: 97 additions & 0 deletions be/src/io/fs/stream_load_pipe_reader.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

#pragma once

#include <condition_variable>
#include <deque>

#include "io/fs/file_reader.h"
#include "runtime/message_body_sink.h"

namespace doris {
namespace io {

const size_t kMaxPipeBufferedBytes = 4 * 1024 * 1024;

class StreamLoadPipeReader : public MessageBodySink, public FileReader {
public:
StreamLoadPipeReader(size_t max_buffered_bytes = kMaxPipeBufferedBytes,
size_t min_chunk_size = 64 * 1024, int64_t total_length = -1,
bool use_proto = false);

~StreamLoadPipeReader() override;

Status append_and_flush(const char* data, size_t size, size_t proto_byte_size = 0);

Status append(const char* data, size_t size) override;

Status append(const ByteBufferPtr& buf) override;

Status read_at(size_t offset, Slice result, const IOContext& io_ctx,
size_t* bytes_read) override;

const Path& path() const override { return _path; }

size_t size() const override { return 0; }

// called when consumer finished
Status close() override {
cancel("closed");
return Status::OK();
}

bool closed() const override { return _cancelled; }

// called when producer finished
Status finish() override;

// called when producer/consumer failed
void cancel(const std::string& reason) override;

private:
// read the next buffer from _buf_queue
Status _read_next_buffer(std::unique_ptr<uint8_t[]>* data, int64_t* length);

Status _append(const ByteBufferPtr& buf, size_t proto_byte_size = 0);

// Blocking queue
std::mutex _lock;
size_t _buffered_bytes;
size_t _proto_buffered_bytes;
size_t _max_buffered_bytes;
size_t _min_chunk_size;
// The total amount of data expected to be read.
// In some scenarios, such as loading json format data through stream load,
// the data needs to be completely read before it can be parsed,
// so the total size of the data needs to be known.
// The default is -1, which means that the data arrives in a stream
// and the length is unknown.
// size_t is unsigned, so use int64_t
// int64_t _total_length = -1;
bool _use_proto = false;
std::deque<ByteBufferPtr> _buf_queue;
std::condition_variable _put_cond;
std::condition_variable _get_cond;

ByteBufferPtr _write_buf;

// no use, only for compatibility with the `Path` interface
Path _path = "";
};
} // namespace io
} // namespace doris
1 change: 1 addition & 0 deletions be/src/runtime/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,7 @@ set(RUNTIME_FILES
stream_load/stream_load_executor.cpp
stream_load/stream_load_recorder.cpp
stream_load/load_stream_mgr.cpp
stream_load/new_load_stream_mgr.cpp
routine_load/data_consumer.cpp
routine_load/data_consumer_group.cpp
routine_load/data_consumer_pool.cpp
Expand Down
3 changes: 3 additions & 0 deletions be/src/runtime/exec_env.h
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@ class FragmentMgr;
class ResultCache;
class LoadPathMgr;
class LoadStreamMgr;
class NewLoadStreamMgr;
class MemTrackerLimiter;
class MemTracker;
class StorageEngine;
Expand Down Expand Up @@ -168,6 +169,7 @@ class ExecEnv {
BufferPool* buffer_pool() { return _buffer_pool; }
LoadChannelMgr* load_channel_mgr() { return _load_channel_mgr; }
LoadStreamMgr* load_stream_mgr() { return _load_stream_mgr; }
NewLoadStreamMgr* new_load_stream_mgr() { return _new_load_stream_mgr; }
SmallFileMgr* small_file_mgr() { return _small_file_mgr; }
StoragePolicyMgr* storage_policy_mgr() { return _storage_policy_mgr; }

Expand Down Expand Up @@ -250,6 +252,7 @@ class ExecEnv {
BrokerMgr* _broker_mgr = nullptr;
LoadChannelMgr* _load_channel_mgr = nullptr;
LoadStreamMgr* _load_stream_mgr = nullptr;
NewLoadStreamMgr* _new_load_stream_mgr = nullptr;
BrpcClientCache<PBackendService_Stub>* _internal_client_cache = nullptr;
BrpcClientCache<PFunctionService_Stub>* _function_client_cache = nullptr;

Expand Down
2 changes: 2 additions & 0 deletions be/src/runtime/exec_env_init.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@
#include "runtime/routine_load/routine_load_task_executor.h"
#include "runtime/small_file_mgr.h"
#include "runtime/stream_load/load_stream_mgr.h"
#include "runtime/stream_load/new_load_stream_mgr.h"
#include "runtime/stream_load/stream_load_executor.h"
#include "runtime/thread_resource_mgr.h"
#include "runtime/tmp_file_mgr.h"
Expand Down Expand Up @@ -144,6 +145,7 @@ Status ExecEnv::_init(const std::vector<StorePath>& store_paths) {
_broker_mgr = new BrokerMgr(this);
_load_channel_mgr = new LoadChannelMgr();
_load_stream_mgr = new LoadStreamMgr();
_new_load_stream_mgr = new NewLoadStreamMgr();
_internal_client_cache = new BrpcClientCache<PBackendService_Stub>();
_function_client_cache = new BrpcClientCache<PFunctionService_Stub>();
_stream_load_executor = new StreamLoadExecutor(this);
Expand Down
Loading