[feature-wip][multi-catalog]Support prefetch for orc file format (#11292)

Refactor the prefetch code in parquet and support prefetch for orc file format
This commit is contained in:
huangzhaowei
2022-08-02 11:01:15 +08:00
committed by GitHub
parent bd6e3cf132
commit 0ac5228c05
6 changed files with 120 additions and 109 deletions

View File

@ -48,6 +48,11 @@ ArrowReaderWrap::ArrowReaderWrap(FileReader* file_reader, int64_t batch_size,
ArrowReaderWrap::~ArrowReaderWrap() {
close();
_closed = true;
_queue_writer_cond.notify_one();
if (_thread.joinable()) {
_thread.join();
}
}
void ArrowReaderWrap::close() {
@ -76,6 +81,62 @@ Status ArrowReaderWrap::column_indices(const std::vector<SlotDescriptor*>& tuple
return Status::OK();
}
Status ArrowReaderWrap::next_batch(std::shared_ptr<arrow::RecordBatch>* batch, bool* eof) {
std::unique_lock<std::mutex> lock(_mtx);
while (!_closed && _queue.empty()) {
if (_batch_eof) {
_include_column_ids.clear();
*eof = true;
_batch_eof = false;
return Status::OK();
}
_queue_reader_cond.wait_for(lock, std::chrono::seconds(1));
}
if (UNLIKELY(_closed)) {
return Status::InternalError(_status.message());
}
*batch = _queue.front();
_queue.pop_front();
_queue_writer_cond.notify_one();
return Status::OK();
}
void ArrowReaderWrap::prefetch_batch() {
auto insert_batch = [this](const auto& batch) {
std::unique_lock<std::mutex> lock(_mtx);
while (!_closed && _queue.size() == _max_queue_size) {
_queue_writer_cond.wait_for(lock, std::chrono::seconds(1));
}
if (UNLIKELY(_closed)) {
return;
}
_queue.push_back(batch);
_queue_reader_cond.notify_one();
};
int current_group = _current_group;
int total_groups = _total_groups;
while (true) {
if (_closed || current_group >= total_groups) {
_batch_eof = true;
_queue_reader_cond.notify_one();
return;
}
if (filter_row_group(current_group)) {
current_group++;
continue;
}
arrow::RecordBatchVector batches;
read_batches(batches, current_group);
if (!_status.ok()) {
_closed = true;
return;
}
std::for_each(batches.begin(), batches.end(), insert_batch);
current_group++;
}
}
ArrowFile::ArrowFile(FileReader* file) : _file(file) {}
ArrowFile::~ArrowFile() {