Refactor DMatrix to return batches of different page types (#4686)

* Use explicit template parameter for specifying page type.
This commit is contained in:
Rong Ou 2019-08-03 12:10:34 -07:00 committed by Jiaming Yuan
parent e930a8e54f
commit 6edddd7966
41 changed files with 477 additions and 470 deletions

4
Jenkinsfile vendored
View File

@ -330,10 +330,8 @@ def TestR(args) {
def use_r35_flag = (args.use_r35) ? "1" : "0"
def docker_args = "--build-arg USE_R35=${use_r35_flag}"
sh """
${dockerRun} ${container_type} ${docker_binary} ${docker_args} tests/ci_build/build_test_rpkg.sh
${dockerRun} ${container_type} ${docker_binary} ${docker_args} tests/ci_build/build_test_rpkg.sh || tests/ci_build/print_r_stacktrace.sh
"""
// Save error log, if any
archiveArtifacts artifacts: "xgboost.Rcheck/00install.out", allowEmptyArchive: true
deleteDir()
}
}

View File

@ -38,7 +38,6 @@
#include "../src/predictor/cpu_predictor.cc"
#if DMLC_ENABLE_STD_THREAD
#include "../src/data/sparse_page_source.cc"
#include "../src/data/sparse_page_dmatrix.cc"
#include "../src/data/sparse_page_writer.cc"
#endif

View File

@ -15,6 +15,7 @@
#include <numeric>
#include <algorithm>
#include <string>
#include <utility>
#include <vector>
#include "./base.h"
#include "../../src/common/span.h"
@ -270,20 +271,34 @@ class SparsePage {
size_t Size() { return offset.Size() - 1; }
};
class CSCPage: public SparsePage {
public:
CSCPage() : SparsePage() {}
explicit CSCPage(SparsePage page) : SparsePage(std::move(page)) {}
};
class SortedCSCPage : public SparsePage {
public:
SortedCSCPage() : SparsePage() {}
explicit SortedCSCPage(SparsePage page) : SparsePage(std::move(page)) {}
};
template<typename T>
class BatchIteratorImpl {
public:
virtual ~BatchIteratorImpl() {}
virtual BatchIteratorImpl* Clone() = 0;
virtual SparsePage& operator*() = 0;
virtual const SparsePage& operator*() const = 0;
virtual T& operator*() = 0;
virtual const T& operator*() const = 0;
virtual void operator++() = 0;
virtual bool AtEnd() const = 0;
};
template<typename T>
class BatchIterator {
public:
using iterator_category = std::forward_iterator_tag;
explicit BatchIterator(BatchIteratorImpl* impl) { impl_.reset(impl); }
explicit BatchIterator(BatchIteratorImpl<T>* impl) { impl_.reset(impl); }
BatchIterator(const BatchIterator& other) {
if (other.impl_) {
@ -298,12 +313,12 @@ class BatchIterator {
++(*impl_);
}
SparsePage& operator*() {
T& operator*() {
CHECK(impl_ != nullptr);
return *(*impl_);
}
const SparsePage& operator*() const {
const T& operator*() const {
CHECK(impl_ != nullptr);
return *(*impl_);
}
@ -319,17 +334,18 @@ class BatchIterator {
}
private:
std::unique_ptr<BatchIteratorImpl> impl_;
std::unique_ptr<BatchIteratorImpl<T>> impl_;
};
template<typename T>
class BatchSet {
public:
explicit BatchSet(BatchIterator begin_iter) : begin_iter_(begin_iter) {}
BatchIterator begin() { return begin_iter_; }
BatchIterator end() { return BatchIterator(nullptr); }
explicit BatchSet(BatchIterator<T> begin_iter) : begin_iter_(begin_iter) {}
BatchIterator<T> begin() { return begin_iter_; }
BatchIterator<T> end() { return BatchIterator<T>(nullptr); }
private:
BatchIterator begin_iter_;
BatchIterator<T> begin_iter_;
};
/*!
@ -339,7 +355,8 @@ class BatchSet {
*
* On distributed setting, usually an customized dmlc::Parser is needed instead.
*/
class DataSource : public dmlc::DataIter<SparsePage> {
template<typename T>
class DataSource : public dmlc::DataIter<T> {
public:
/*!
* \brief Meta information about the dataset
@ -367,11 +384,10 @@ class DMatrix {
/*! \brief meta information of the dataset */
virtual const MetaInfo& Info() const = 0;
/**
* \brief Gets row batches. Use range based for loop over BatchSet to access individual batches.
* \brief Gets batches. Use range based for loop over BatchSet to access individual batches.
*/
virtual BatchSet GetRowBatches() = 0;
virtual BatchSet GetSortedColumnBatches() = 0;
virtual BatchSet GetColumnBatches() = 0;
template<typename T>
BatchSet<T> GetBatches();
// the following are column meta data, should be able to answer them fast.
/*! \return Whether the data columns single column block. */
virtual bool SingleColBlock() const = 0;
@ -410,7 +426,7 @@ class DMatrix {
* This can be nullptr for common cases, and in-memory mode will be used.
* \return a Created DMatrix.
*/
static DMatrix* Create(std::unique_ptr<DataSource>&& source,
static DMatrix* Create(std::unique_ptr<DataSource<SparsePage>>&& source,
const std::string& cache_prefix = "");
/*!
* \brief Create a DMatrix by loading data from parser.
@ -431,7 +447,27 @@ class DMatrix {
/*! \brief page size 32 MB */
static const size_t kPageSize = 32UL << 20UL;
protected:
virtual BatchSet<SparsePage> GetRowBatches() = 0;
virtual BatchSet<CSCPage> GetColumnBatches() = 0;
virtual BatchSet<SortedCSCPage> GetSortedColumnBatches() = 0;
};
template<>
inline BatchSet<SparsePage> DMatrix::GetBatches() {
return GetRowBatches();
}
template<>
inline BatchSet<CSCPage> DMatrix::GetBatches() {
return GetColumnBatches();
}
template<>
inline BatchSet<SortedCSCPage> DMatrix::GetBatches() {
return GetSortedColumnBatches();
}
} // namespace xgboost
namespace dmlc {

View File

@ -165,7 +165,7 @@ void SparseCuts::Build(DMatrix* dmat, uint32_t const max_num_bins) {
sparse_cuts[i].reset(new SparseCuts(&cuts_containers[i]));
}
for (auto const& page : dmat->GetColumnBatches()) {
for (auto const& page : dmat->GetBatches<CSCPage>()) {
CHECK_LE(page.Size(), dmat->Info().num_col_);
monitor_.Start("Load balance");
std::vector<size_t> col_ptr = LoadBalance(page, nthreads);
@ -247,7 +247,7 @@ void DenseCuts::Build(DMatrix* p_fmat, uint32_t max_num_bins) {
// Use group index for weights?
bool const use_group = UseGroup(p_fmat);
for (const auto &batch : p_fmat->GetRowBatches()) {
for (const auto &batch : p_fmat->GetBatches<SparsePage>()) {
size_t group_ind = 0;
if (use_group) {
group_ind = this->SearchGroupIndFromRow(group_ptr, batch.base_rowid);
@ -336,7 +336,7 @@ void GHistIndexMatrix::Init(DMatrix* p_fmat, int max_num_bins) {
size_t new_size = 1;
for (const auto &batch : p_fmat->GetRowBatches()) {
for (const auto &batch : p_fmat->GetBatches<SparsePage>()) {
new_size += batch.Size();
}
@ -346,7 +346,7 @@ void GHistIndexMatrix::Init(DMatrix* p_fmat, int max_num_bins) {
size_t rbegin = 0;
size_t prev_sum = 0;
for (const auto &batch : p_fmat->GetRowBatches()) {
for (const auto &batch : p_fmat->GetBatches<SparsePage>()) {
// The number of threads is pegged to the batch size. If the OMP
// block is parallelized on anything other than the batch/block size,
// it should be reassigned

View File

@ -440,7 +440,7 @@ struct GPUSketcher {
row_stride_ = 0;
sketch_container_.reset(new SketchContainer(param_, dmat));
for (const auto &batch : dmat->GetRowBatches()) {
for (const auto &batch : dmat->GetBatches<SparsePage>()) {
this->SketchBatch(batch, info);
}

View File

@ -9,7 +9,6 @@
#include "./sparse_page_writer.h"
#include "./simple_dmatrix.h"
#include "./simple_csr_source.h"
#include "../common/common.h"
#include "../common/io.h"
#if DMLC_ENABLE_STD_THREAD
@ -146,7 +145,6 @@ void MetaInfo::SetInfo(const char* key, const void* dptr, DataType dtype, size_t
}
}
DMatrix* DMatrix::Load(const std::string& uri,
bool silent,
bool load_row_split,
@ -257,11 +255,11 @@ DMatrix* DMatrix::Create(dmlc::Parser<uint32_t>* parser,
return DMatrix::Create(std::move(source), cache_prefix);
} else {
#if DMLC_ENABLE_STD_THREAD
if (!data::SparsePageSource::CacheExist(cache_prefix, ".row.page")) {
data::SparsePageSource::CreateRowPage(parser, cache_prefix, page_size);
if (!data::SparsePageSource<SparsePage>::CacheExist(cache_prefix, ".row.page")) {
data::SparsePageSource<SparsePage>::CreateRowPage(parser, cache_prefix, page_size);
}
std::unique_ptr<data::SparsePageSource> source(
new data::SparsePageSource(cache_prefix, ".row.page"));
std::unique_ptr<data::SparsePageSource<SparsePage>> source(
new data::SparsePageSource<SparsePage>(cache_prefix, ".row.page"));
return DMatrix::Create(std::move(source), cache_prefix);
#else
LOG(FATAL) << "External memory is not enabled in mingw";
@ -277,7 +275,7 @@ void DMatrix::SaveToLocalFile(const std::string& fname) {
source.SaveBinary(fo.get());
}
DMatrix* DMatrix::Create(std::unique_ptr<DataSource>&& source,
DMatrix* DMatrix::Create(std::unique_ptr<DataSource<SparsePage>>&& source,
const std::string& cache_prefix) {
if (cache_prefix.length() == 0) {
return new data::SimpleDMatrix(std::move(source));

View File

@ -18,7 +18,7 @@ void SimpleCSRSource::Clear() {
void SimpleCSRSource::CopyFrom(DMatrix* src) {
this->Clear();
this->info = src->Info();
for (const auto &batch : src->GetRowBatches()) {
for (const auto &batch : src->GetBatches<SparsePage>()) {
page_.Push(batch);
}
}

View File

@ -25,7 +25,7 @@ namespace data {
* DMatrix* dmat = DMatrix::Create(std::move(source));
* \encode
*/
class SimpleCSRSource : public DataSource {
class SimpleCSRSource : public DataSource<SparsePage> {
public:
// public data members
// MetaInfo info; // inheritated from DataSource

View File

@ -18,10 +18,10 @@ float SimpleDMatrix::GetColDensity(size_t cidx) {
size_t column_size = 0;
// Use whatever version of column batches already exists
if (sorted_column_page_) {
auto batch = this->GetSortedColumnBatches();
auto batch = this->GetBatches<SortedCSCPage>();
column_size = (*batch.begin())[cidx].size();
} else {
auto batch = this->GetColumnBatches();
auto batch = this->GetBatches<CSCPage>();
column_size = (*batch.begin())[cidx].size();
}
@ -29,14 +29,15 @@ float SimpleDMatrix::GetColDensity(size_t cidx) {
return 1.0f - (static_cast<float>(nmiss)) / this->Info().num_row_;
}
class SimpleBatchIteratorImpl : public BatchIteratorImpl {
template<typename T>
class SimpleBatchIteratorImpl : public BatchIteratorImpl<T> {
public:
explicit SimpleBatchIteratorImpl(SparsePage* page) : page_(page) {}
SparsePage& operator*() override {
explicit SimpleBatchIteratorImpl(T* page) : page_(page) {}
T& operator*() override {
CHECK(page_ != nullptr);
return *page_;
}
const SparsePage& operator*() const override {
const T& operator*() const override {
CHECK(page_ != nullptr);
return *page_;
}
@ -47,38 +48,38 @@ class SimpleBatchIteratorImpl : public BatchIteratorImpl {
}
private:
SparsePage* page_{nullptr};
T* page_{nullptr};
};
BatchSet SimpleDMatrix::GetRowBatches() {
BatchSet<SparsePage> SimpleDMatrix::GetRowBatches() {
auto cast = dynamic_cast<SimpleCSRSource*>(source_.get());
auto begin_iter = BatchIterator(new SimpleBatchIteratorImpl(&(cast->page_)));
return BatchSet(begin_iter);
auto begin_iter = BatchIterator<SparsePage>(
new SimpleBatchIteratorImpl<SparsePage>(&(cast->page_)));
return BatchSet<SparsePage>(begin_iter);
}
BatchSet SimpleDMatrix::GetColumnBatches() {
BatchSet<CSCPage> SimpleDMatrix::GetColumnBatches() {
// column page doesn't exist, generate it
if (!column_page_) {
auto page = dynamic_cast<SimpleCSRSource*>(source_.get())->page_;
column_page_.reset(
new SparsePage(page.GetTranspose(source_->info.num_col_)));
column_page_.reset(new CSCPage(page.GetTranspose(source_->info.num_col_)));
}
auto begin_iter =
BatchIterator(new SimpleBatchIteratorImpl(column_page_.get()));
return BatchSet(begin_iter);
BatchIterator<CSCPage>(new SimpleBatchIteratorImpl<CSCPage>(column_page_.get()));
return BatchSet<CSCPage>(begin_iter);
}
BatchSet SimpleDMatrix::GetSortedColumnBatches() {
BatchSet<SortedCSCPage> SimpleDMatrix::GetSortedColumnBatches() {
// Sorted column page doesn't exist, generate it
if (!sorted_column_page_) {
auto page = dynamic_cast<SimpleCSRSource*>(source_.get())->page_;
sorted_column_page_.reset(
new SparsePage(page.GetTranspose(source_->info.num_col_)));
new SortedCSCPage(page.GetTranspose(source_->info.num_col_)));
sorted_column_page_->SortRows();
}
auto begin_iter =
BatchIterator(new SimpleBatchIteratorImpl(sorted_column_page_.get()));
return BatchSet(begin_iter);
auto begin_iter = BatchIterator<SortedCSCPage>(
new SimpleBatchIteratorImpl<SortedCSCPage>(sorted_column_page_.get()));
return BatchSet<SortedCSCPage>(begin_iter);
}
bool SimpleDMatrix::SingleColBlock() const { return true; }

View File

@ -23,7 +23,7 @@ namespace data {
// Used for single batch data.
class SimpleDMatrix : public DMatrix {
public:
explicit SimpleDMatrix(std::unique_ptr<DataSource>&& source)
explicit SimpleDMatrix(std::unique_ptr<DataSource<SparsePage>>&& source)
: source_(std::move(source)) {}
MetaInfo& Info() override;
@ -34,18 +34,16 @@ class SimpleDMatrix : public DMatrix {
bool SingleColBlock() const override;
BatchSet GetRowBatches() override;
BatchSet GetColumnBatches() override;
BatchSet GetSortedColumnBatches() override;
private:
// source data pointer.
std::unique_ptr<DataSource> source_;
BatchSet<SparsePage> GetRowBatches() override;
BatchSet<CSCPage> GetColumnBatches() override;
BatchSet<SortedCSCPage> GetSortedColumnBatches() override;
std::unique_ptr<SparsePage> sorted_column_page_;
std::unique_ptr<SparsePage> column_page_;
// source data pointer.
std::unique_ptr<DataSource<SparsePage>> source_;
std::unique_ptr<CSCPage> column_page_;
std::unique_ptr<SortedCSCPage> sorted_column_page_;
};
} // namespace data
} // namespace xgboost

View File

@ -6,12 +6,9 @@
*/
#include <dmlc/base.h>
#include <dmlc/timer.h>
#include <xgboost/logging.h>
#include <memory>
#if DMLC_ENABLE_STD_THREAD
#include "./sparse_page_dmatrix.h"
#include "../common/random.h"
namespace xgboost {
namespace data {
@ -24,13 +21,14 @@ const MetaInfo& SparsePageDMatrix::Info() const {
return row_source_->info;
}
class SparseBatchIteratorImpl : public BatchIteratorImpl {
template<typename T>
class SparseBatchIteratorImpl : public BatchIteratorImpl<T> {
public:
explicit SparseBatchIteratorImpl(SparsePageSource* source) : source_(source) {
explicit SparseBatchIteratorImpl(SparsePageSource<T>* source) : source_(source) {
CHECK(source_ != nullptr);
}
SparsePage& operator*() override { return source_->Value(); }
const SparsePage& operator*() const override { return source_->Value(); }
T& operator*() override { return source_->Value(); }
const T& operator*() const override { return source_->Value(); }
void operator++() override { at_end_ = !source_->Next(); }
bool AtEnd() const override { return at_end_; }
SparseBatchIteratorImpl* Clone() override {
@ -38,50 +36,50 @@ class SparseBatchIteratorImpl : public BatchIteratorImpl {
}
private:
SparsePageSource* source_{nullptr};
SparsePageSource<T>* source_{nullptr};
bool at_end_{ false };
};
BatchSet SparsePageDMatrix::GetRowBatches() {
auto cast = dynamic_cast<SparsePageSource*>(row_source_.get());
BatchSet<SparsePage> SparsePageDMatrix::GetRowBatches() {
auto cast = dynamic_cast<SparsePageSource<SparsePage>*>(row_source_.get());
cast->BeforeFirst();
cast->Next();
auto begin_iter = BatchIterator(new SparseBatchIteratorImpl(cast));
return BatchSet(begin_iter);
auto begin_iter = BatchIterator<SparsePage>(new SparseBatchIteratorImpl<SparsePage>(cast));
return BatchSet<SparsePage>(begin_iter);
}
BatchSet SparsePageDMatrix::GetSortedColumnBatches() {
// Lazily instantiate
if (!sorted_column_source_) {
SparsePageSource::CreateColumnPage(this, cache_info_, true);
sorted_column_source_.reset(
new SparsePageSource(cache_info_, ".sorted.col.page"));
}
sorted_column_source_->BeforeFirst();
sorted_column_source_->Next();
auto begin_iter =
BatchIterator(new SparseBatchIteratorImpl(sorted_column_source_.get()));
return BatchSet(begin_iter);
}
BatchSet SparsePageDMatrix::GetColumnBatches() {
BatchSet<CSCPage> SparsePageDMatrix::GetColumnBatches() {
// Lazily instantiate
if (!column_source_) {
SparsePageSource::CreateColumnPage(this, cache_info_, false);
column_source_.reset(new SparsePageSource(cache_info_, ".col.page"));
SparsePageSource<SparsePage>::CreateColumnPage(this, cache_info_, false);
column_source_.reset(new SparsePageSource<CSCPage>(cache_info_, ".col.page"));
}
column_source_->BeforeFirst();
column_source_->Next();
auto begin_iter =
BatchIterator(new SparseBatchIteratorImpl(column_source_.get()));
return BatchSet(begin_iter);
BatchIterator<CSCPage>(new SparseBatchIteratorImpl<CSCPage>(column_source_.get()));
return BatchSet<CSCPage>(begin_iter);
}
BatchSet<SortedCSCPage> SparsePageDMatrix::GetSortedColumnBatches() {
// Lazily instantiate
if (!sorted_column_source_) {
SparsePageSource<SparsePage>::CreateColumnPage(this, cache_info_, true);
sorted_column_source_.reset(
new SparsePageSource<SortedCSCPage>(cache_info_, ".sorted.col.page"));
}
sorted_column_source_->BeforeFirst();
sorted_column_source_->Next();
auto begin_iter = BatchIterator<SortedCSCPage>(
new SparseBatchIteratorImpl<SortedCSCPage>(sorted_column_source_.get()));
return BatchSet<SortedCSCPage>(begin_iter);
}
float SparsePageDMatrix::GetColDensity(size_t cidx) {
// Finds densities if we don't already have them
if (col_density_.empty()) {
std::vector<size_t> column_size(this->Info().num_col_);
for (const auto &batch : this->GetColumnBatches()) {
for (const auto &batch : this->GetBatches<CSCPage>()) {
for (auto i = 0u; i < batch.Size(); i++) {
column_size[i] += batch[i].size();
}

View File

@ -21,7 +21,7 @@ namespace data {
// Used for external memory.
class SparsePageDMatrix : public DMatrix {
public:
explicit SparsePageDMatrix(std::unique_ptr<DataSource>&& source,
explicit SparsePageDMatrix(std::unique_ptr<DataSource<SparsePage>>&& source,
std::string cache_info)
: row_source_(std::move(source)), cache_info_(std::move(cache_info)) {}
virtual ~SparsePageDMatrix() = default;
@ -30,21 +30,19 @@ class SparsePageDMatrix : public DMatrix {
const MetaInfo& Info() const override;
BatchSet GetRowBatches() override;
BatchSet GetSortedColumnBatches() override;
BatchSet GetColumnBatches() override;
float GetColDensity(size_t cidx) override;
bool SingleColBlock() const override;
private:
BatchSet<SparsePage> GetRowBatches() override;
BatchSet<CSCPage> GetColumnBatches() override;
BatchSet<SortedCSCPage> GetSortedColumnBatches() override;
// source data pointers.
std::unique_ptr<DataSource> row_source_;
std::unique_ptr<SparsePageSource> column_source_;
std::unique_ptr<SparsePageSource> sorted_column_source_;
std::unique_ptr<DataSource<SparsePage>> row_source_;
std::unique_ptr<SparsePageSource<CSCPage>> column_source_;
std::unique_ptr<SparsePageSource<SortedCSCPage>> sorted_column_source_;
// the cache prefix
std::string cache_info_;
// Store column densities to avoid recalculating

View File

@ -1,301 +0,0 @@
/*!
* Copyright 2015 by Contributors
* \file sparse_page_source.cc
*/
#include <dmlc/base.h>
#include <dmlc/timer.h>
#include <xgboost/logging.h>
#include <memory>
#include <vector>
#include <string>
#include <locale>
#if DMLC_ENABLE_STD_THREAD
#include "./sparse_page_source.h"
#include "../common/common.h"
namespace {
// Split a cache info string with delimiter ':'
// If cache info string contains drive letter (e.g. C:), exclude it before splitting
inline std::vector<std::string>
GetCacheShards(const std::string& cache_info) {
#if (defined _WIN32) || (defined __CYGWIN__)
if (cache_info.length() >= 2
&& std::isalpha(cache_info[0], std::locale::classic())
&& cache_info[1] == ':') {
std::vector<std::string> cache_shards
= xgboost::common::Split(cache_info.substr(2), ':');
cache_shards[0] = cache_info.substr(0, 2) + cache_shards[0];
return cache_shards;
}
#endif // (defined _WIN32) || (defined __CYGWIN__)
return xgboost::common::Split(cache_info, ':');
}
} // anonymous namespace
namespace xgboost {
namespace data {
SparsePageSource::SparsePageSource(const std::string& cache_info,
const std::string& page_type)
: base_rowid_(0), page_(nullptr), clock_ptr_(0) {
// read in the info files
std::vector<std::string> cache_shards = GetCacheShards(cache_info);
CHECK_NE(cache_shards.size(), 0U);
{
std::string name_info = cache_shards[0];
std::unique_ptr<dmlc::Stream> finfo(dmlc::Stream::Create(name_info.c_str(), "r"));
int tmagic;
CHECK_EQ(finfo->Read(&tmagic, sizeof(tmagic)), sizeof(tmagic));
this->info.LoadBinary(finfo.get());
}
files_.resize(cache_shards.size());
formats_.resize(cache_shards.size());
prefetchers_.resize(cache_shards.size());
// read in the cache files.
for (size_t i = 0; i < cache_shards.size(); ++i) {
std::string name_row = cache_shards[i] + page_type;
files_[i].reset(dmlc::SeekStream::CreateForRead(name_row.c_str()));
std::unique_ptr<dmlc::SeekStream>& fi = files_[i];
std::string format;
CHECK(fi->Read(&format)) << "Invalid page format";
formats_[i].reset(SparsePageFormat::Create(format));
std::unique_ptr<SparsePageFormat>& fmt = formats_[i];
size_t fbegin = fi->Tell();
prefetchers_[i].reset(new dmlc::ThreadedIter<SparsePage>(4));
prefetchers_[i]->Init([&fi, &fmt] (SparsePage** dptr) {
if (*dptr == nullptr) {
*dptr = new SparsePage();
}
return fmt->Read(*dptr, fi.get());
}, [&fi, fbegin] () { fi->Seek(fbegin); });
}
}
SparsePageSource::~SparsePageSource() {
delete page_;
}
bool SparsePageSource::Next() {
// doing clock rotation over shards.
if (page_ != nullptr) {
size_t n = prefetchers_.size();
prefetchers_[(clock_ptr_ + n - 1) % n]->Recycle(&page_);
}
if (prefetchers_[clock_ptr_]->Next(&page_)) {
page_->base_rowid = base_rowid_;
base_rowid_ += page_->Size();
// advance clock
clock_ptr_ = (clock_ptr_ + 1) % prefetchers_.size();
return true;
} else {
return false;
}
}
void SparsePageSource::BeforeFirst() {
base_rowid_ = 0;
clock_ptr_ = 0;
for (auto& p : prefetchers_) {
p->BeforeFirst();
}
}
SparsePage& SparsePageSource::Value() {
return *page_;
}
const SparsePage& SparsePageSource::Value() const {
return *page_;
}
bool SparsePageSource::CacheExist(const std::string& cache_info,
const std::string& page_type) {
std::vector<std::string> cache_shards = GetCacheShards(cache_info);
CHECK_NE(cache_shards.size(), 0U);
{
std::string name_info = cache_shards[0];
std::unique_ptr<dmlc::Stream> finfo(dmlc::Stream::Create(name_info.c_str(), "r", true));
if (finfo == nullptr) return false;
}
for (const std::string& prefix : cache_shards) {
std::string name_row = prefix + page_type;
std::unique_ptr<dmlc::Stream> frow(dmlc::Stream::Create(name_row.c_str(), "r", true));
if (frow == nullptr) return false;
}
return true;
}
void SparsePageSource::CreateRowPage(dmlc::Parser<uint32_t>* src,
const std::string& cache_info,
const size_t page_size) {
const std::string page_type = ".row.page";
std::vector<std::string> cache_shards = GetCacheShards(cache_info);
CHECK_NE(cache_shards.size(), 0U);
// read in the info files.
std::string name_info = cache_shards[0];
std::vector<std::string> name_shards, format_shards;
for (const std::string& prefix : cache_shards) {
name_shards.push_back(prefix + page_type);
format_shards.push_back(SparsePageFormat::DecideFormat(prefix).first);
}
{
SparsePageWriter writer(name_shards, format_shards, 6);
std::shared_ptr<SparsePage> page;
writer.Alloc(&page); page->Clear();
MetaInfo info;
size_t bytes_write = 0;
double tstart = dmlc::GetTime();
// print every 4 sec.
constexpr double kStep = 4.0;
size_t tick_expected = static_cast<double>(kStep);
const uint64_t default_max = std::numeric_limits<uint64_t>::max();
uint64_t last_group_id = default_max;
bst_uint group_size = 0;
while (src->Next()) {
const dmlc::RowBlock<uint32_t>& batch = src->Value();
if (batch.label != nullptr) {
auto& labels = info.labels_.HostVector();
labels.insert(labels.end(), batch.label, batch.label + batch.size);
}
if (batch.weight != nullptr) {
auto& weights = info.weights_.HostVector();
weights.insert(weights.end(), batch.weight, batch.weight + batch.size);
}
if (batch.qid != nullptr) {
info.qids_.insert(info.qids_.end(), batch.qid, batch.qid + batch.size);
// get group
for (size_t i = 0; i < batch.size; ++i) {
const uint64_t cur_group_id = batch.qid[i];
if (last_group_id == default_max || last_group_id != cur_group_id) {
info.group_ptr_.push_back(group_size);
}
last_group_id = cur_group_id;
++group_size;
}
}
info.num_row_ += batch.size;
info.num_nonzero_ += batch.offset[batch.size] - batch.offset[0];
for (size_t i = batch.offset[0]; i < batch.offset[batch.size]; ++i) {
uint32_t index = batch.index[i];
info.num_col_ = std::max(info.num_col_,
static_cast<uint64_t>(index + 1));
}
page->Push(batch);
if (page->MemCostBytes() >= page_size) {
bytes_write += page->MemCostBytes();
writer.PushWrite(std::move(page));
writer.Alloc(&page);
page->Clear();
double tdiff = dmlc::GetTime() - tstart;
if (tdiff >= tick_expected) {
LOG(CONSOLE) << "Writing " << page_type << " to " << cache_info
<< " in " << ((bytes_write >> 20UL) / tdiff) << " MB/s, "
<< (bytes_write >> 20UL) << " written";
tick_expected += static_cast<size_t>(kStep);
}
}
}
if (last_group_id != default_max) {
if (group_size > info.group_ptr_.back()) {
info.group_ptr_.push_back(group_size);
}
}
if (page->data.Size() != 0) {
writer.PushWrite(std::move(page));
}
std::unique_ptr<dmlc::Stream> fo(
dmlc::Stream::Create(name_info.c_str(), "w"));
int tmagic = kMagic;
fo->Write(&tmagic, sizeof(tmagic));
// Either every row has query ID or none at all
CHECK(info.qids_.empty() || info.qids_.size() == info.num_row_);
info.SaveBinary(fo.get());
}
LOG(INFO) << "SparsePageSource::CreateRowPage Finished writing to "
<< name_info;
}
void SparsePageSource::CreatePageFromDMatrix(DMatrix* src,
const std::string& cache_info,
const std::string& page_type,
const size_t page_size) {
std::vector<std::string> cache_shards = GetCacheShards(cache_info);
CHECK_NE(cache_shards.size(), 0U);
// read in the info files.
std::string name_info = cache_shards[0];
std::vector<std::string> name_shards, format_shards;
for (const std::string& prefix : cache_shards) {
name_shards.push_back(prefix + page_type);
format_shards.push_back(SparsePageFormat::DecideFormat(prefix).first);
}
{
SparsePageWriter writer(name_shards, format_shards, 6);
std::shared_ptr<SparsePage> page;
writer.Alloc(&page);
page->Clear();
MetaInfo info = src->Info();
size_t bytes_write = 0;
double tstart = dmlc::GetTime();
for (auto& batch : src->GetRowBatches()) {
if (page_type == ".row.page") {
page->Push(batch);
} else if (page_type == ".col.page") {
page->PushCSC(batch.GetTranspose(src->Info().num_col_));
} else if (page_type == ".sorted.col.page") {
SparsePage tmp = batch.GetTranspose(src->Info().num_col_);
page->PushCSC(tmp);
page->SortRows();
} else {
LOG(FATAL) << "Unknown page type: " << page_type;
}
if (page->MemCostBytes() >= page_size) {
bytes_write += page->MemCostBytes();
writer.PushWrite(std::move(page));
writer.Alloc(&page);
page->Clear();
double tdiff = dmlc::GetTime() - tstart;
LOG(INFO) << "Writing to " << cache_info << " in "
<< ((bytes_write >> 20UL) / tdiff) << " MB/s, "
<< (bytes_write >> 20UL) << " written";
}
}
if (page->data.Size() != 0) {
writer.PushWrite(std::move(page));
}
std::unique_ptr<dmlc::Stream> fo(
dmlc::Stream::Create(name_info.c_str(), "w"));
int tmagic = kMagic;
fo->Write(&tmagic, sizeof(tmagic));
info.SaveBinary(fo.get());
}
LOG(INFO) << "SparsePageSource: Finished writing to " << name_info;
}
void SparsePageSource::CreateRowPage(DMatrix* src,
const std::string& cache_info) {
const std::string page_type = ".row.page";
CreatePageFromDMatrix(src, cache_info, page_type);
}
void SparsePageSource::CreateColumnPage(DMatrix* src,
const std::string& cache_info,
bool sorted) {
const std::string page_type = sorted ? ".sorted.col.page" : ".col.page";
CreatePageFromDMatrix(src, cache_info, page_type);
}
} // namespace data
} // namespace xgboost
#endif // DMLC_ENABLE_STD_THREAD

View File

@ -10,14 +10,39 @@
#include <xgboost/base.h>
#include <xgboost/data.h>
#include <dmlc/threadediter.h>
#include <dmlc/timer.h>
#include <algorithm>
#include <limits>
#include <locale>
#include <memory>
#include <string>
#include <utility>
#include <vector>
#include "sparse_page_writer.h"
namespace {
// Split a cache info string with delimiter ':'
// If cache info string contains drive letter (e.g. C:), exclude it before splitting
inline std::vector<std::string>
GetCacheShards(const std::string& cache_info) {
#if (defined _WIN32) || (defined __CYGWIN__)
if (cache_info.length() >= 2
&& std::isalpha(cache_info[0], std::locale::classic())
&& cache_info[1] == ':') {
std::vector<std::string> cache_shards
= xgboost::common::Split(cache_info.substr(2), ':');
cache_shards[0] = cache_info.substr(0, 2) + cache_shards[0];
return cache_shards;
}
#endif // (defined _WIN32) || (defined __CYGWIN__)
return xgboost::common::Split(cache_info, ':');
}
} // anonymous namespace
namespace xgboost {
namespace data {
/*!
@ -28,23 +53,91 @@ namespace data {
* DMatrix* dmat = DMatrix::Create(std::move(source));
* \encode
*/
class SparsePageSource : public DataSource {
template<typename T>
class SparsePageSource : public DataSource<T> {
public:
/*!
* \brief Create source from cache files the cache_prefix.
* \param cache_prefix The prefix of cache we want to solve.
*/
explicit SparsePageSource(const std::string& cache_prefix,
const std::string& page_type) noexcept(false);
explicit SparsePageSource(const std::string& cache_info,
const std::string& page_type) noexcept(false)
: base_rowid_(0), page_(nullptr), clock_ptr_(0) {
// read in the info files
std::vector<std::string> cache_shards = GetCacheShards(cache_info);
CHECK_NE(cache_shards.size(), 0U);
{
std::string name_info = cache_shards[0];
std::unique_ptr<dmlc::Stream> finfo(dmlc::Stream::Create(name_info.c_str(), "r"));
int tmagic;
CHECK_EQ(finfo->Read(&tmagic, sizeof(tmagic)), sizeof(tmagic));
this->info.LoadBinary(finfo.get());
}
files_.resize(cache_shards.size());
formats_.resize(cache_shards.size());
prefetchers_.resize(cache_shards.size());
// read in the cache files.
for (size_t i = 0; i < cache_shards.size(); ++i) {
std::string name_row = cache_shards[i] + page_type;
files_[i].reset(dmlc::SeekStream::CreateForRead(name_row.c_str()));
std::unique_ptr<dmlc::SeekStream>& fi = files_[i];
std::string format;
CHECK(fi->Read(&format)) << "Invalid page format";
formats_[i].reset(SparsePageFormat::Create(format));
std::unique_ptr<SparsePageFormat>& fmt = formats_[i];
size_t fbegin = fi->Tell();
prefetchers_[i].reset(new dmlc::ThreadedIter<T>(4));
prefetchers_[i]->Init([&fi, &fmt] (T** dptr) {
if (*dptr == nullptr) {
*dptr = new T();
}
return fmt->Read(*dptr, fi.get());
}, [&fi, fbegin] () { fi->Seek(fbegin); });
}
}
/*! \brief destructor */
~SparsePageSource() override;
~SparsePageSource() override {
delete page_;
}
// implement Next
bool Next() override;
bool Next() override {
// doing clock rotation over shards.
if (page_ != nullptr) {
size_t n = prefetchers_.size();
prefetchers_[(clock_ptr_ + n - 1) % n]->Recycle(&page_);
}
if (prefetchers_[clock_ptr_]->Next(&page_)) {
page_->base_rowid = base_rowid_;
base_rowid_ += page_->Size();
// advance clock
clock_ptr_ = (clock_ptr_ + 1) % prefetchers_.size();
return true;
} else {
return false;
}
}
// implement BeforeFirst
void BeforeFirst() override;
void BeforeFirst() override {
base_rowid_ = 0;
clock_ptr_ = 0;
for (auto& p : prefetchers_) {
p->BeforeFirst();
}
}
// implement Value
SparsePage& Value();
const SparsePage& Value() const override;
T& Value() {
return *page_;
}
const T& Value() const override {
return *page_;
}
/*!
* \brief Create source by taking data from parser.
* \param src source parser.
@ -53,13 +146,109 @@ class SparsePageSource : public DataSource {
*/
static void CreateRowPage(dmlc::Parser<uint32_t>* src,
const std::string& cache_info,
const size_t page_size = DMatrix::kPageSize);
const size_t page_size = DMatrix::kPageSize) {
const std::string page_type = ".row.page";
std::vector<std::string> cache_shards = GetCacheShards(cache_info);
CHECK_NE(cache_shards.size(), 0U);
// read in the info files.
std::string name_info = cache_shards[0];
std::vector<std::string> name_shards, format_shards;
for (const std::string& prefix : cache_shards) {
name_shards.push_back(prefix + page_type);
format_shards.push_back(SparsePageFormat::DecideFormat(prefix).first);
}
{
SparsePageWriter writer(name_shards, format_shards, 6);
std::shared_ptr<SparsePage> page;
writer.Alloc(&page); page->Clear();
MetaInfo info;
size_t bytes_write = 0;
double tstart = dmlc::GetTime();
// print every 4 sec.
constexpr double kStep = 4.0;
size_t tick_expected = static_cast<double>(kStep);
const uint64_t default_max = std::numeric_limits<uint64_t>::max();
uint64_t last_group_id = default_max;
bst_uint group_size = 0;
while (src->Next()) {
const dmlc::RowBlock<uint32_t>& batch = src->Value();
if (batch.label != nullptr) {
auto& labels = info.labels_.HostVector();
labels.insert(labels.end(), batch.label, batch.label + batch.size);
}
if (batch.weight != nullptr) {
auto& weights = info.weights_.HostVector();
weights.insert(weights.end(), batch.weight, batch.weight + batch.size);
}
if (batch.qid != nullptr) {
info.qids_.insert(info.qids_.end(), batch.qid, batch.qid + batch.size);
// get group
for (size_t i = 0; i < batch.size; ++i) {
const uint64_t cur_group_id = batch.qid[i];
if (last_group_id == default_max || last_group_id != cur_group_id) {
info.group_ptr_.push_back(group_size);
}
last_group_id = cur_group_id;
++group_size;
}
}
info.num_row_ += batch.size;
info.num_nonzero_ += batch.offset[batch.size] - batch.offset[0];
for (size_t i = batch.offset[0]; i < batch.offset[batch.size]; ++i) {
uint32_t index = batch.index[i];
info.num_col_ = std::max(info.num_col_,
static_cast<uint64_t>(index + 1));
}
page->Push(batch);
if (page->MemCostBytes() >= page_size) {
bytes_write += page->MemCostBytes();
writer.PushWrite(std::move(page));
writer.Alloc(&page);
page->Clear();
double tdiff = dmlc::GetTime() - tstart;
if (tdiff >= tick_expected) {
LOG(CONSOLE) << "Writing " << page_type << " to " << cache_info
<< " in " << ((bytes_write >> 20UL) / tdiff) << " MB/s, "
<< (bytes_write >> 20UL) << " written";
tick_expected += static_cast<size_t>(kStep);
}
}
}
if (last_group_id != default_max) {
if (group_size > info.group_ptr_.back()) {
info.group_ptr_.push_back(group_size);
}
}
if (page->data.Size() != 0) {
writer.PushWrite(std::move(page));
}
std::unique_ptr<dmlc::Stream> fo(
dmlc::Stream::Create(name_info.c_str(), "w"));
int tmagic = kMagic;
fo->Write(&tmagic, sizeof(tmagic));
// Either every row has query ID or none at all
CHECK(info.qids_.empty() || info.qids_.size() == info.num_row_);
info.SaveBinary(fo.get());
}
LOG(INFO) << "SparsePageSource::CreateRowPage Finished writing to "
<< name_info;
}
/*!
* \brief Create source cache by copy content from DMatrix.
* \param cache_info The cache_info of cache file location.
*/
static void CreateRowPage(DMatrix* src,
const std::string& cache_info);
const std::string& cache_info) {
const std::string page_type = ".row.page";
CreatePageFromDMatrix(src, cache_info, page_type);
}
/*!
* \brief Create source cache by copy content from DMatrix. Creates transposed column page, may be sorted or not.
@ -67,7 +256,11 @@ class SparsePageSource : public DataSource {
* \param sorted Whether columns should be pre-sorted
*/
static void CreateColumnPage(DMatrix* src,
const std::string& cache_info, bool sorted);
const std::string& cache_info, bool sorted) {
const std::string page_type = sorted ? ".sorted.col.page" : ".col.page";
CreatePageFromDMatrix(src, cache_info, page_type);
}
/*!
* \brief Check if the cache file already exists.
* \param cache_info The cache prefix of files.
@ -75,18 +268,88 @@ class SparsePageSource : public DataSource {
* \return Whether cache file already exists.
*/
static bool CacheExist(const std::string& cache_info,
const std::string& page_type);
const std::string& page_type) {
std::vector<std::string> cache_shards = GetCacheShards(cache_info);
CHECK_NE(cache_shards.size(), 0U);
{
std::string name_info = cache_shards[0];
std::unique_ptr<dmlc::Stream> finfo(dmlc::Stream::Create(name_info.c_str(), "r", true));
if (finfo == nullptr) return false;
}
for (const std::string& prefix : cache_shards) {
std::string name_row = prefix + page_type;
std::unique_ptr<dmlc::Stream> frow(dmlc::Stream::Create(name_row.c_str(), "r", true));
if (frow == nullptr) return false;
}
return true;
}
/*! \brief magic number used to identify Page */
static const int kMagic = 0xffffab02;
private:
static void CreatePageFromDMatrix(DMatrix* src, const std::string& cache_info,
const std::string& page_type,
const size_t page_size = DMatrix::kPageSize);
const size_t page_size = DMatrix::kPageSize) {
std::vector<std::string> cache_shards = GetCacheShards(cache_info);
CHECK_NE(cache_shards.size(), 0U);
// read in the info files.
std::string name_info = cache_shards[0];
std::vector<std::string> name_shards, format_shards;
for (const std::string& prefix : cache_shards) {
name_shards.push_back(prefix + page_type);
format_shards.push_back(SparsePageFormat::DecideFormat(prefix).first);
}
{
SparsePageWriter writer(name_shards, format_shards, 6);
std::shared_ptr<SparsePage> page;
writer.Alloc(&page);
page->Clear();
MetaInfo info = src->Info();
size_t bytes_write = 0;
double tstart = dmlc::GetTime();
for (auto& batch : src->GetBatches<SparsePage>()) {
if (page_type == ".row.page") {
page->Push(batch);
} else if (page_type == ".col.page") {
page->PushCSC(batch.GetTranspose(src->Info().num_col_));
} else if (page_type == ".sorted.col.page") {
SparsePage tmp = batch.GetTranspose(src->Info().num_col_);
page->PushCSC(tmp);
page->SortRows();
} else {
LOG(FATAL) << "Unknown page type: " << page_type;
}
if (page->MemCostBytes() >= page_size) {
bytes_write += page->MemCostBytes();
writer.PushWrite(std::move(page));
writer.Alloc(&page);
page->Clear();
double tdiff = dmlc::GetTime() - tstart;
LOG(INFO) << "Writing to " << cache_info << " in "
<< ((bytes_write >> 20UL) / tdiff) << " MB/s, "
<< (bytes_write >> 20UL) << " written";
}
}
if (page->data.Size() != 0) {
writer.PushWrite(std::move(page));
}
std::unique_ptr<dmlc::Stream> fo(
dmlc::Stream::Create(name_info.c_str(), "w"));
int tmagic = kMagic;
fo->Write(&tmagic, sizeof(tmagic));
info.SaveBinary(fo.get());
}
LOG(INFO) << "SparsePageSource: Finished writing to " << name_info;
}
/*! \brief number of rows */
size_t base_rowid_;
/*! \brief page currently on hold. */
SparsePage *page_;
T *page_;
/*! \brief internal clock ptr */
size_t clock_ptr_;
/*! \brief file pointer to the row blob file. */
@ -94,7 +357,7 @@ class SparsePageSource : public DataSource {
/*! \brief Sparse page format file. */
std::vector<std::unique_ptr<SparsePageFormat> > formats_;
/*! \brief internal prefetcher. */
std::vector<std::unique_ptr<dmlc::ThreadedIter<SparsePage> > > prefetchers_;
std::vector<std::unique_ptr<dmlc::ThreadedIter<T> > > prefetchers_;
};
} // namespace data
} // namespace xgboost

View File

@ -140,7 +140,7 @@ class GBLinear : public GradientBooster {
// make sure contributions is zeroed, we could be reusing a previously allocated one
std::fill(contribs.begin(), contribs.end(), 0);
// start collecting the contributions
for (const auto &batch : p_fmat->GetRowBatches()) {
for (const auto &batch : p_fmat->GetBatches<SparsePage>()) {
// parallel over local batch
const auto nsize = static_cast<bst_omp_uint>(batch.Size());
#pragma omp parallel for schedule(static)
@ -198,7 +198,7 @@ class GBLinear : public GradientBooster {
// start collecting the prediction
const int ngroup = model_.param.num_output_group;
preds.resize(p_fmat->Info().num_row_ * ngroup);
for (const auto &batch : p_fmat->GetRowBatches()) {
for (const auto &batch : p_fmat->GetBatches<SparsePage>()) {
// output convention: nrow * k, where nrow is number of rows
// k is number of group
// parallel over local batch

View File

@ -371,7 +371,7 @@ class Dart : public GBTree {
CHECK_EQ(preds.size(), p_fmat->Info().num_row_ * num_group);
// start collecting the prediction
auto* self = static_cast<Derived*>(this);
for (const auto &batch : p_fmat->GetRowBatches()) {
for (const auto &batch : p_fmat->GetBatches<SparsePage>()) {
constexpr int kUnroll = 8;
const auto nsize = static_cast<bst_omp_uint>(batch.Size());
const bst_omp_uint rest = nsize % kUnroll;

View File

@ -80,7 +80,7 @@ inline std::pair<double, double> GetGradient(int group_idx, int num_group, int f
const std::vector<GradientPair> &gpair,
DMatrix *p_fmat) {
double sum_grad = 0.0, sum_hess = 0.0;
for (const auto &batch : p_fmat->GetColumnBatches()) {
for (const auto &batch : p_fmat->GetBatches<CSCPage>()) {
auto col = batch[fidx];
const auto ndata = static_cast<bst_omp_uint>(col.size());
for (bst_omp_uint j = 0; j < ndata; ++j) {
@ -109,7 +109,7 @@ inline std::pair<double, double> GetGradientParallel(int group_idx, int num_grou
const std::vector<GradientPair> &gpair,
DMatrix *p_fmat) {
double sum_grad = 0.0, sum_hess = 0.0;
for (const auto &batch : p_fmat->GetColumnBatches()) {
for (const auto &batch : p_fmat->GetBatches<CSCPage>()) {
auto col = batch[fidx];
const auto ndata = static_cast<bst_omp_uint>(col.size());
#pragma omp parallel for schedule(static) reduction(+ : sum_grad, sum_hess)
@ -164,7 +164,7 @@ inline void UpdateResidualParallel(int fidx, int group_idx, int num_group,
float dw, std::vector<GradientPair> *in_gpair,
DMatrix *p_fmat) {
if (dw == 0.0f) return;
for (const auto &batch : p_fmat->GetColumnBatches()) {
for (const auto &batch : p_fmat->GetBatches<CSCPage>()) {
auto col = batch[fidx];
// update grad value
const auto num_row = static_cast<bst_omp_uint>(col.size());
@ -332,7 +332,7 @@ class GreedyFeatureSelector : public FeatureSelector {
const bst_omp_uint nfeat = model.param.num_feature;
// Calculate univariate gradient sums
std::fill(gpair_sums_.begin(), gpair_sums_.end(), std::make_pair(0., 0.));
for (const auto &batch : p_fmat->GetColumnBatches()) {
for (const auto &batch : p_fmat->GetBatches<CSCPage>()) {
#pragma omp parallel for schedule(static)
for (bst_omp_uint i = 0; i < nfeat; ++i) {
const auto col = batch[i];
@ -397,7 +397,7 @@ class ThriftyFeatureSelector : public FeatureSelector {
}
// Calculate univariate gradient sums
std::fill(gpair_sums_.begin(), gpair_sums_.end(), std::make_pair(0., 0.));
for (const auto &batch : p_fmat->GetColumnBatches()) {
for (const auto &batch : p_fmat->GetBatches<CSCPage>()) {
// column-parallel is usually faster than row-parallel
#pragma omp parallel for schedule(static)
for (bst_omp_uint i = 0; i < nfeat; ++i) {

View File

@ -186,7 +186,7 @@ class GPUCoordinateUpdater : public LinearUpdater {
}
CHECK(p_fmat->SingleColBlock());
SparsePage const& batch = *(p_fmat->GetColumnBatches().begin());
SparsePage const& batch = *(p_fmat->GetBatches<CSCPage>().begin());
shards_.resize(n_devices);
// Create device shards

View File

@ -42,7 +42,7 @@ class ShotgunUpdater : public LinearUpdater {
// lock-free parallel updates of weights
selector_->Setup(*model, in_gpair->ConstHostVector(), p_fmat,
param_.reg_alpha_denorm, param_.reg_lambda_denorm, 0);
for (const auto &batch : p_fmat->GetColumnBatches()) {
for (const auto &batch : p_fmat->GetBatches<CSCPage>()) {
const auto nfeat = static_cast<bst_omp_uint>(batch.Size());
#pragma omp parallel for schedule(static)
for (bst_omp_uint i = 0; i < nfeat; ++i) {

View File

@ -53,7 +53,7 @@ class CPUPredictor : public Predictor {
<< "size_leaf_vector is enforced to 0 so far";
CHECK_EQ(preds.size(), p_fmat->Info().num_row_ * num_group);
// start collecting the prediction
for (const auto &batch : p_fmat->GetRowBatches()) {
for (const auto &batch : p_fmat->GetBatches<SparsePage>()) {
// parallel over local batch
constexpr int kUnroll = 8;
const auto nsize = static_cast<bst_omp_uint>(batch.Size());
@ -230,7 +230,7 @@ class CPUPredictor : public Predictor {
std::vector<bst_float>& preds = *out_preds;
preds.resize(info.num_row_ * ntree_limit);
// start collecting the prediction
for (const auto &batch : p_fmat->GetRowBatches()) {
for (const auto &batch : p_fmat->GetBatches<SparsePage>()) {
// parallel over local batch
const auto nsize = static_cast<bst_omp_uint>(batch.Size());
#pragma omp parallel for schedule(static)
@ -276,7 +276,7 @@ class CPUPredictor : public Predictor {
}
const std::vector<bst_float>& base_margin = info.base_margin_.HostVector();
// start collecting the contributions
for (const auto &batch : p_fmat->GetRowBatches()) {
for (const auto &batch : p_fmat->GetBatches<SparsePage>()) {
// parallel over local batch
const auto nsize = static_cast<bst_omp_uint>(batch.Size());
#pragma omp parallel for schedule(static)

View File

@ -352,7 +352,7 @@ class GPUPredictor : public xgboost::Predictor {
InitModel(model, tree_begin, tree_end);
size_t batch_offset = 0;
for (auto &batch : dmat->GetRowBatches()) {
for (auto &batch : dmat->GetBatches<SparsePage>()) {
bool is_external_memory = batch.Size() < dmat->Info().num_row_;
if (is_external_memory) {
std::vector<size_t> out_preds_offsets;

View File

@ -45,7 +45,7 @@ class BaseMaker: public TreeUpdater {
std::fill(fminmax_.begin(), fminmax_.end(),
-std::numeric_limits<bst_float>::max());
// start accumulating statistics
for (const auto &batch : p_fmat->GetSortedColumnBatches()) {
for (const auto &batch : p_fmat->GetBatches<SortedCSCPage>()) {
for (bst_uint fid = 0; fid < batch.Size(); ++fid) {
auto c = batch[fid];
if (c.size() != 0) {
@ -302,7 +302,7 @@ class BaseMaker: public TreeUpdater {
const RegTree &tree) {
std::vector<unsigned> fsplits;
this->GetSplitSet(nodes, tree, &fsplits);
for (const auto &batch : p_fmat->GetSortedColumnBatches()) {
for (const auto &batch : p_fmat->GetBatches<SortedCSCPage>()) {
for (auto fid : fsplits) {
auto col = batch[fid];
const auto ndata = static_cast<bst_omp_uint>(col.size());

View File

@ -637,7 +637,7 @@ class ColMaker: public TreeUpdater {
DMatrix *p_fmat,
RegTree *p_tree) {
auto feat_set = column_sampler_.GetFeatureSet(depth);
for (const auto &batch : p_fmat->GetSortedColumnBatches()) {
for (const auto &batch : p_fmat->GetBatches<SortedCSCPage>()) {
this->UpdateSolution(batch, feat_set->HostVector(), gpair, p_fmat);
}
// after this each thread's stemp will get the best candidates, aggregate results
@ -716,7 +716,7 @@ class ColMaker: public TreeUpdater {
}
std::sort(fsplits.begin(), fsplits.end());
fsplits.resize(std::unique(fsplits.begin(), fsplits.end()) - fsplits.begin());
for (const auto &batch : p_fmat->GetSortedColumnBatches()) {
for (const auto &batch : p_fmat->GetBatches<SortedCSCPage>()) {
for (auto fid : fsplits) {
auto col = batch[fid];
const auto ndata = static_cast<bst_omp_uint>(col.size());
@ -846,7 +846,7 @@ class DistColMaker : public ColMaker {
boolmap_[j] = 0;
}
}
for (const auto &batch : p_fmat->GetSortedColumnBatches()) {
for (const auto &batch : p_fmat->GetBatches<SortedCSCPage>()) {
for (auto fid : fsplits) {
auto col = batch[fid];
const auto ndata = static_cast<bst_omp_uint>(col.size());

View File

@ -734,7 +734,7 @@ class GPUMaker : public TreeUpdater {
fId->reserve(n_cols_ * n_rows_);
// in case you end up with a DMatrix having no column access
// then make sure to enable that before copying the data!
for (const auto& batch : dmat->GetSortedColumnBatches()) {
for (const auto& batch : dmat->GetBatches<SortedCSCPage>()) {
for (int i = 0; i < batch.Size(); i++) {
auto col = batch[i];
for (const Entry& e : col) {

View File

@ -1382,7 +1382,7 @@ class GPUHistMakerSpecialised {
monitor_.StartCuda("BinningCompression");
DeviceHistogramBuilderState hist_builder_row_state(shards_);
for (const auto &batch : dmat->GetRowBatches()) {
for (const auto &batch : dmat->GetBatches<SparsePage>()) {
hist_builder_row_state.BeginBatch(batch);
dh::ExecuteIndexShards(

View File

@ -351,7 +351,7 @@ class CQHistMaker: public HistMaker {
auto lazy_get_hist = [&]() {
thread_hist_.resize(omp_get_max_threads());
// start accumulating statistics
for (const auto &batch : p_fmat->GetSortedColumnBatches()) {
for (const auto &batch : p_fmat->GetBatches<SortedCSCPage>()) {
// start enumeration
const auto nsize = static_cast<bst_omp_uint>(fset.size());
#pragma omp parallel for schedule(dynamic, 1)
@ -425,7 +425,7 @@ class CQHistMaker: public HistMaker {
work_set_.resize(std::unique(work_set_.begin(), work_set_.end()) - work_set_.begin());
// start accumulating statistics
for (const auto &batch : p_fmat->GetSortedColumnBatches()) {
for (const auto &batch : p_fmat->GetBatches<SortedCSCPage>()) {
// TWOPASS: use the real set + split set in the column iteration.
this->CorrectNonDefaultPositionByBatch(batch, fsplit_set_, tree);
@ -707,7 +707,7 @@ class GlobalProposalHistMaker: public CQHistMaker {
std::unique(this->work_set_.begin(), this->work_set_.end()) - this->work_set_.begin());
// start accumulating statistics
for (const auto &batch : p_fmat->GetSortedColumnBatches()) {
for (const auto &batch : p_fmat->GetBatches<SortedCSCPage>()) {
// TWOPASS: use the real set + split set in the column iteration.
this->CorrectNonDefaultPositionByBatch(batch, this->fsplit_set_, tree);

View File

@ -56,7 +56,7 @@ class TreeRefresher: public TreeUpdater {
auto lazy_get_stats = [&]() {
const MetaInfo &info = p_fmat->Info();
// start accumulating statistics
for (const auto &batch : p_fmat->GetRowBatches()) {
for (const auto &batch : p_fmat->GetBatches<SparsePage>()) {
CHECK_LT(batch.Size(), std::numeric_limits<unsigned>::max());
const auto nbatch = static_cast<bst_omp_uint>(batch.Size());
#pragma omp parallel for schedule(static)

View File

@ -135,7 +135,7 @@ class SketchMaker: public BaseMaker {
// number of rows in
const size_t nrows = p_fmat->Info().num_row_;
// start accumulating statistics
for (const auto &batch : p_fmat->GetSortedColumnBatches()) {
for (const auto &batch : p_fmat->GetBatches<SortedCSCPage>()) {
// start enumeration
const auto nsize = static_cast<bst_omp_uint>(batch.Size());
#pragma omp parallel for schedule(dynamic, 1)

View File

@ -0,0 +1,19 @@
#!/bin/bash
# To be called when R package tests have failed
set -e
if [ -f "xgboost/xgboost.Rcheck/00install.out" ]; then
echo "===== xgboost/xgboost.Rcheck/00install.out ===="
cat xgboost/xgboost.Rcheck/00install.out
fi
if [ -f "xgboost/xgboost.Rcheck/00check.log" ]; then
printf "\n\n===== xgboost/xgboost.Rcheck/00check.log ====\n"
cat xgboost/xgboost.Rcheck/00check.log
fi
# Produce error code to interrupt Jenkins pipeline
exit 1

View File

@ -20,7 +20,7 @@ TEST(c_api, XGDMatrixCreateFromMatDT) {
ASSERT_EQ(info.num_row_, 3);
ASSERT_EQ(info.num_nonzero_, 6);
for (const auto &batch : (*dmat)->GetRowBatches()) {
for (const auto &batch : (*dmat)->GetBatches<xgboost::SparsePage>()) {
ASSERT_EQ(batch[0][0].fvalue, 0.0f);
ASSERT_EQ(batch[0][1].fvalue, -4.0f);
ASSERT_EQ(batch[2][0].fvalue, 3.0f);
@ -52,7 +52,7 @@ TEST(c_api, XGDMatrixCreateFromMat_omp) {
ASSERT_EQ(info.num_row_, row);
ASSERT_EQ(info.num_nonzero_, num_cols * row - num_missing);
for (const auto &batch : (*dmat)->GetRowBatches()) {
for (const auto &batch : (*dmat)->GetBatches<xgboost::SparsePage>()) {
for (size_t i = 0; i < batch.Size(); i++) {
auto inst = batch[i];
for (auto e : inst) {

View File

@ -58,7 +58,7 @@ void TestDeviceSketch(const GPUSet& devices, bool use_external_memory) {
// compare the row stride with the one obtained from the dmatrix
size_t expected_row_stride = 0;
for (const auto &batch : dmat->get()->GetRowBatches()) {
for (const auto &batch : dmat->get()->GetBatches<xgboost::SparsePage>()) {
const auto &offset_vec = batch.offset.ConstHostVector();
for (int i = 1; i <= offset_vec.size() -1; ++i) {
expected_row_stride = std::max(expected_row_stride, offset_vec[i] - offset_vec[i-1]);

View File

@ -61,7 +61,7 @@ TEST(SparseCuts, SingleThreadedBuild) {
HistogramCuts cuts;
SparseCuts indices(&cuts);
auto const& page = *(p_fmat->GetColumnBatches().begin());
auto const& page = *(p_fmat->GetBatches<xgboost::CSCPage>().begin());
indices.SingleThreadBuild(page, p_fmat->Info(), kBins, false, 0, page.Size(), 0);
ASSERT_EQ(hmat.cut.Ptrs().size(), cuts.Ptrs().size());
@ -92,7 +92,7 @@ TEST(SparseCuts, MultiThreadedBuild) {
HistogramCuts container;
SparseCuts indices(&container);
auto const& page = *(p_fmat->GetColumnBatches().begin());
auto const& page = *(p_fmat->GetBatches<xgboost::CSCPage>().begin());
indices.SingleThreadBuild(page, p_fmat->Info(), kBins, false, 0, page.Size(), 0);
ASSERT_EQ(container.Ptrs().size(), threaded_container.Ptrs().size());

View File

@ -63,7 +63,7 @@ TEST(SparsePage, PushCSCAfterTranspose) {
CreateSparsePageDMatrix(n_entries, 64UL, filename);
const int ncols = dmat->Info().num_col_;
SparsePage page; // Consolidated sparse page
for (const auto &batch : dmat->GetRowBatches()) {
for (const auto &batch : dmat->GetBatches<xgboost::SparsePage>()) {
// Transpose each batch and push
SparsePage tmp = batch.GetTranspose(ncols);
page.PushCSC(tmp);

View File

@ -122,7 +122,7 @@ TEST(MetaInfo, LoadQid) {
xgboost::Entry(2, 0), xgboost::Entry(3, 0), xgboost::Entry(4, 0.4),
xgboost::Entry(5, 1), xgboost::Entry(1, 0), xgboost::Entry(2, 1),
xgboost::Entry(3, 1), xgboost::Entry(4, 0.5), {5, 0}};
for (const auto &batch : dmat->GetRowBatches()) {
for (const auto &batch : dmat->GetBatches<xgboost::SparsePage>()) {
CHECK_EQ(batch.base_rowid, 0);
CHECK(batch.offset.HostVector() == expected_offset);
CHECK(batch.data.HostVector() == expected_data);

View File

@ -20,10 +20,10 @@ TEST(SimpleCSRSource, SaveLoadBinary) {
EXPECT_EQ(dmat->Info().num_row_, dmat_read->Info().num_row_);
// Test we have non-empty batch
EXPECT_EQ(dmat->GetRowBatches().begin().AtEnd(), false);
EXPECT_EQ(dmat->GetBatches<xgboost::SparsePage>().begin().AtEnd(), false);
auto row_iter = dmat->GetRowBatches().begin();
auto row_iter_read = dmat_read->GetRowBatches().begin();
auto row_iter = dmat->GetBatches<xgboost::SparsePage>().begin();
auto row_iter_read = dmat_read->GetBatches<xgboost::SparsePage>().begin();
// Test the data read into the first row
auto first_row = (*row_iter)[0];
auto first_row_read = (*row_iter_read)[0];

View File

@ -28,12 +28,12 @@ TEST(SimpleDMatrix, RowAccess) {
// Loop over the batches and count the records
int64_t row_count = 0;
for (auto &batch : dmat->GetRowBatches()) {
for (auto &batch : dmat->GetBatches<xgboost::SparsePage>()) {
row_count += batch.Size();
}
EXPECT_EQ(row_count, dmat->Info().num_row_);
// Test the data read into the first row
auto &batch = *dmat->GetRowBatches().begin();
auto &batch = *dmat->GetBatches<xgboost::SparsePage>().begin();
auto first_row = batch[0];
ASSERT_EQ(first_row.size(), 3);
EXPECT_EQ(first_row[2].index, 2);
@ -55,7 +55,7 @@ TEST(SimpleDMatrix, ColAccessWithoutBatches) {
// Loop over the batches and assert the data is as expected
int64_t num_col_batch = 0;
for (const auto &batch : dmat->GetSortedColumnBatches()) {
for (const auto &batch : dmat->GetBatches<xgboost::SortedCSCPage>()) {
num_col_batch += 1;
EXPECT_EQ(batch.Size(), dmat->Info().num_col_)
<< "Expected batch size = number of cells as #batches is 1.";

View File

@ -33,7 +33,7 @@ TEST(SparsePageDMatrix, RowAccess) {
xgboost::CreateSparsePageDMatrix(12, 64, filename);
// Test the data read into the first row
auto &batch = *dmat->GetRowBatches().begin();
auto &batch = *dmat->GetBatches<xgboost::SparsePage>().begin();
auto first_row = batch[0];
ASSERT_EQ(first_row.size(), 3);
EXPECT_EQ(first_row[2].index, 2);
@ -51,14 +51,14 @@ TEST(SparsePageDMatrix, ColAccess) {
EXPECT_EQ(dmat->GetColDensity(1), 0.5);
// Loop over the batches and assert the data is as expected
for (auto col_batch : dmat->GetSortedColumnBatches()) {
for (auto col_batch : dmat->GetBatches<xgboost::SortedCSCPage>()) {
EXPECT_EQ(col_batch.Size(), dmat->Info().num_col_);
EXPECT_EQ(col_batch[1][0].fvalue, 10.0f);
EXPECT_EQ(col_batch[1].size(), 1);
}
// Loop over the batches and assert the data is as expected
for (auto col_batch : dmat->GetColumnBatches()) {
for (auto col_batch : dmat->GetBatches<xgboost::CSCPage>()) {
EXPECT_EQ(col_batch.Size(), dmat->Info().num_col_);
EXPECT_EQ(col_batch[1][0].fvalue, 10.0f);
EXPECT_EQ(col_batch[1].size(), 1);
@ -82,7 +82,7 @@ TEST(SparsePageDMatrix, ColAccessBatches) {
};
auto n_threads = omp_get_max_threads();
omp_set_num_threads(16);
for (auto const& page : dmat->GetColumnBatches()) {
for (auto const& page : dmat->GetBatches<xgboost::CSCPage>()) {
ASSERT_EQ(dmat->Info().num_col_, page.Size());
}
omp_set_num_threads(n_threads);

View File

@ -157,7 +157,7 @@ std::unique_ptr<DMatrix> CreateSparsePageDMatrix(
// Loop over the batches and count the records
int64_t batch_count = 0;
int64_t row_count = 0;
for (const auto &batch : dmat->GetRowBatches()) {
for (const auto &batch : dmat->GetBatches<xgboost::SparsePage>()) {
batch_count++;
row_count += batch.Size();
}

View File

@ -26,7 +26,7 @@ TEST(cpu_predictor, Test) {
}
// Test predict instance
auto &batch = *(*dmat)->GetRowBatches().begin();
auto &batch = *(*dmat)->GetBatches<xgboost::SparsePage>().begin();
for (size_t i = 0; i < batch.Size(); i++) {
std::vector<float> instance_out_predictions;
cpu_predictor->PredictInstance(batch[i], &instance_out_predictions, model);

View File

@ -76,7 +76,7 @@ template <typename GradientSumT>
void BuildGidx(DeviceShard<GradientSumT>* shard, int n_rows, int n_cols,
bst_float sparsity=0) {
auto dmat = CreateDMatrix(n_rows, n_cols, sparsity, 3);
const SparsePage& batch = *(*dmat)->GetRowBatches().begin();
const SparsePage& batch = *(*dmat)->GetBatches<xgboost::SparsePage>().begin();
HistogramCutsWrapper cmat;
cmat.SetPtrs({0, 3, 6, 9, 12, 15, 18, 21, 24});

View File

@ -65,7 +65,7 @@ class QuantileHistMock : public QuantileHistMaker {
ASSERT_EQ(gmat.row_ptr.size(), num_row + 1);
ASSERT_LT(*std::max_element(gmat.index.begin(), gmat.index.end()),
gmat.cut.Ptrs().back());
for (const auto& batch : p_fmat->GetRowBatches()) {
for (const auto& batch : p_fmat->GetBatches<xgboost::SparsePage>()) {
for (size_t i = 0; i < batch.Size(); ++i) {
const size_t rid = batch.base_rowid + i;
ASSERT_LT(rid, num_row);