- Install cmake using pip. - Fix compile command generation. - Clean up the tidy script and remove the need to load the yaml file. - Fix modernized type traits. - Fix span class. Polymorphism support is dropped
351 lines
11 KiB
C++
351 lines
11 KiB
C++
/**
|
|
* Copyright 2016-2024, XGBoost Contributors
|
|
*/
|
|
#include <gtest/gtest.h>
|
|
#include <xgboost/data.h>
|
|
|
|
#include <future>
|
|
#include <thread>
|
|
|
|
#include "../../../src/common/io.h"
|
|
#include "../../../src/data/adapter.h"
|
|
#include "../../../src/data/file_iterator.h"
|
|
#include "../../../src/data/simple_dmatrix.h"
|
|
#include "../../../src/data/sparse_page_dmatrix.h"
|
|
#include "../filesystem.h" // dmlc::TemporaryDirectory
|
|
#include "../helpers.h"
|
|
|
|
using namespace xgboost; // NOLINT
|
|
namespace {
|
|
std::string UriSVM(std::string name, std::string cache) {
|
|
return name + "?format=libsvm" + "#" + cache + ".cache";
|
|
}
|
|
} // namespace
|
|
|
|
template <typename Page>
|
|
void TestSparseDMatrixLoadFile(Context const* ctx) {
|
|
dmlc::TemporaryDirectory tmpdir;
|
|
auto opath = tmpdir.path + "/1-based.svm";
|
|
CreateBigTestData(opath, 3 * 64, false);
|
|
opath += "?indexing_mode=1&format=libsvm";
|
|
data::FileIterator iter{opath, 0, 1};
|
|
auto n_threads = 0;
|
|
data::SparsePageDMatrix m{&iter,
|
|
iter.Proxy(),
|
|
data::fileiter::Reset,
|
|
data::fileiter::Next,
|
|
std::numeric_limits<float>::quiet_NaN(),
|
|
n_threads,
|
|
tmpdir.path + "cache"};
|
|
ASSERT_EQ(AllThreadsForTest(), m.Ctx()->Threads());
|
|
ASSERT_EQ(m.Info().num_col_, 5);
|
|
ASSERT_EQ(m.Info().num_row_, 64);
|
|
|
|
std::unique_ptr<dmlc::Parser<uint32_t>> parser(
|
|
dmlc::Parser<uint32_t>::Create(opath.c_str(), 0, 1, "auto"));
|
|
auto adapter = data::FileAdapter{parser.get()};
|
|
|
|
data::SimpleDMatrix simple{&adapter, std::numeric_limits<float>::quiet_NaN(),
|
|
1};
|
|
Page out;
|
|
for (auto const &page : m.GetBatches<Page>(ctx)) {
|
|
if (std::is_same_v<Page, SparsePage>) {
|
|
out.Push(page);
|
|
} else {
|
|
out.PushCSC(page);
|
|
}
|
|
}
|
|
ASSERT_EQ(m.Info().num_col_, simple.Info().num_col_);
|
|
ASSERT_EQ(m.Info().num_row_, simple.Info().num_row_);
|
|
|
|
for (auto const& page : simple.GetBatches<Page>(ctx)) {
|
|
ASSERT_EQ(page.offset.HostVector(), out.offset.HostVector());
|
|
for (size_t i = 0; i < page.data.Size(); ++i) {
|
|
ASSERT_EQ(page.data.HostVector()[i].fvalue, out.data.HostVector()[i].fvalue);
|
|
}
|
|
}
|
|
}
|
|
|
|
TEST(SparsePageDMatrix, LoadFile) {
|
|
Context ctx;
|
|
TestSparseDMatrixLoadFile<SparsePage>(&ctx);
|
|
TestSparseDMatrixLoadFile<CSCPage>(&ctx);
|
|
TestSparseDMatrixLoadFile<SortedCSCPage>(&ctx);
|
|
}
|
|
|
|
// allow caller to retain pages so they can process multiple pages at the same time.
|
|
template <typename Page>
|
|
void TestRetainPage() {
|
|
std::size_t n_batches = 4;
|
|
auto p_fmat = RandomDataGenerator{1024, 128, 0.5f}.Batches(n_batches).GenerateSparsePageDMatrix(
|
|
"cache", true);
|
|
Context ctx;
|
|
auto batches = p_fmat->GetBatches<Page>(&ctx);
|
|
auto begin = batches.begin();
|
|
auto end = batches.end();
|
|
|
|
std::vector<Page> pages;
|
|
std::vector<std::shared_ptr<Page const>> iterators;
|
|
for (auto it = begin; it != end; ++it) {
|
|
iterators.push_back(it.Page());
|
|
pages.emplace_back(Page{});
|
|
if (std::is_same_v<Page, SparsePage>) {
|
|
pages.back().Push(*it);
|
|
} else {
|
|
pages.back().PushCSC(*it);
|
|
}
|
|
ASSERT_EQ(pages.back().Size(), (*it).Size());
|
|
}
|
|
ASSERT_GE(iterators.size(), n_batches);
|
|
|
|
for (size_t i = 0; i < iterators.size(); ++i) {
|
|
ASSERT_EQ((*iterators[i]).Size(), pages.at(i).Size());
|
|
ASSERT_EQ((*iterators[i]).data.HostVector(), pages.at(i).data.HostVector());
|
|
}
|
|
|
|
// make sure it's const and the caller can not modify the content of page.
|
|
for (auto &page : p_fmat->GetBatches<Page>({&ctx})) {
|
|
static_assert(std::is_const_v<std::remove_reference_t<decltype(page)>>);
|
|
}
|
|
}
|
|
|
|
TEST(SparsePageDMatrix, RetainSparsePage) {
|
|
TestRetainPage<SparsePage>();
|
|
TestRetainPage<CSCPage>();
|
|
TestRetainPage<SortedCSCPage>();
|
|
}
|
|
|
|
// Test GHistIndexMatrix can avoid loading sparse page after the initialization.
|
|
TEST(SparsePageDMatrix, GHistIndexSkipSparsePage) {
|
|
dmlc::TemporaryDirectory tmpdir;
|
|
std::size_t n_batches = 6;
|
|
auto Xy = RandomDataGenerator{180, 12, 0.0}.Batches(n_batches).GenerateSparsePageDMatrix(
|
|
tmpdir.path + "/", true);
|
|
Context ctx;
|
|
bst_bin_t n_bins{256};
|
|
double sparse_thresh{0.8};
|
|
BatchParam batch_param{n_bins, sparse_thresh};
|
|
|
|
auto check_ghist = [&] {
|
|
std::int32_t k = 0;
|
|
for (auto const &page : Xy->GetBatches<GHistIndexMatrix>(&ctx, batch_param)) {
|
|
ASSERT_EQ(page.Size(), 30);
|
|
ASSERT_EQ(k, page.base_rowid);
|
|
k += page.Size();
|
|
}
|
|
};
|
|
check_ghist();
|
|
|
|
auto casted = std::dynamic_pointer_cast<data::SparsePageDMatrix>(Xy);
|
|
CHECK(casted);
|
|
// Make the number of fetches don't change (no new fetch)
|
|
auto n_init_fetches = casted->SparsePageFetchCount();
|
|
|
|
std::vector<float> hess(Xy->Info().num_row_, 1.0f);
|
|
// Run multiple iterations to make sure fetches are consistent after reset.
|
|
for (std::int32_t i = 0; i < 4; ++i) {
|
|
auto n_fetches = casted->SparsePageFetchCount();
|
|
check_ghist();
|
|
ASSERT_EQ(casted->SparsePageFetchCount(), n_fetches);
|
|
if (i == 0) {
|
|
ASSERT_EQ(n_fetches, n_init_fetches);
|
|
}
|
|
// Make sure other page types don't interfere the GHist. This way, we can reuse the
|
|
// DMatrix for multiple purposes.
|
|
for ([[maybe_unused]] auto const &page : Xy->GetBatches<SparsePage>(&ctx)) {
|
|
}
|
|
for ([[maybe_unused]] auto const &page : Xy->GetBatches<SortedCSCPage>(&ctx)) {
|
|
}
|
|
for ([[maybe_unused]] auto const &page : Xy->GetBatches<GHistIndexMatrix>(&ctx, batch_param)) {
|
|
}
|
|
// Approx tree method pages
|
|
{
|
|
BatchParam regen{n_bins, common::Span{hess.data(), hess.size()}, false};
|
|
for ([[maybe_unused]] auto const &page : Xy->GetBatches<GHistIndexMatrix>(&ctx, regen)) {
|
|
}
|
|
}
|
|
{
|
|
BatchParam regen{n_bins, common::Span{hess.data(), hess.size()}, true};
|
|
for ([[maybe_unused]] auto const &page : Xy->GetBatches<GHistIndexMatrix>(&ctx, regen)) {
|
|
}
|
|
}
|
|
// Restore the batch parameter by passing it in again through check_ghist
|
|
check_ghist();
|
|
}
|
|
|
|
// half the pages
|
|
{
|
|
auto it = Xy->GetBatches<SparsePage>(&ctx).begin();
|
|
for (std::size_t i = 0; i < n_batches / 2; ++i) {
|
|
++it;
|
|
}
|
|
check_ghist();
|
|
}
|
|
{
|
|
auto it = Xy->GetBatches<GHistIndexMatrix>(&ctx, batch_param).begin();
|
|
for (std::size_t i = 0; i < n_batches / 2; ++i) {
|
|
++it;
|
|
}
|
|
check_ghist();
|
|
}
|
|
{
|
|
BatchParam regen{n_bins, common::Span{hess.data(), hess.size()}, true};
|
|
auto it = Xy->GetBatches<GHistIndexMatrix>(&ctx, regen).begin();
|
|
for (std::size_t i = 0; i < n_batches / 2; ++i) {
|
|
++it;
|
|
}
|
|
check_ghist();
|
|
}
|
|
}
|
|
|
|
TEST(SparsePageDMatrix, MetaInfo) {
|
|
dmlc::TemporaryDirectory tmpdir;
|
|
const std::string tmp_file = tmpdir.path + "/simple.libsvm";
|
|
size_t constexpr kEntries = 24;
|
|
CreateBigTestData(tmp_file, kEntries);
|
|
|
|
std::unique_ptr<DMatrix> dmat{xgboost::DMatrix::Load(UriSVM(tmp_file, tmp_file), false)};
|
|
|
|
// Test the metadata that was parsed
|
|
EXPECT_EQ(dmat->Info().num_row_, 8ul);
|
|
EXPECT_EQ(dmat->Info().num_col_, 5ul);
|
|
EXPECT_EQ(dmat->Info().num_nonzero_, kEntries);
|
|
EXPECT_EQ(dmat->Info().labels.Size(), dmat->Info().num_row_);
|
|
}
|
|
|
|
TEST(SparsePageDMatrix, RowAccess) {
|
|
auto dmat = RandomDataGenerator{12, 6, 0.8f}.Batches(2).GenerateSparsePageDMatrix("temp", false);
|
|
|
|
// Test the data read into the first row
|
|
auto &batch = *dmat->GetBatches<xgboost::SparsePage>().begin();
|
|
auto page = batch.GetView();
|
|
auto first_row = page[0];
|
|
ASSERT_EQ(first_row.size(), 1ul);
|
|
EXPECT_EQ(first_row[0].index, 5u);
|
|
EXPECT_NEAR(first_row[0].fvalue, 0.1805125, 1e-4);
|
|
}
|
|
|
|
TEST(SparsePageDMatrix, ColAccess) {
|
|
dmlc::TemporaryDirectory tempdir;
|
|
const std::string tmp_file = tempdir.path + "/simple.libsvm";
|
|
CreateSimpleTestData(tmp_file);
|
|
xgboost::DMatrix *dmat = xgboost::DMatrix::Load(UriSVM(tmp_file, tmp_file));
|
|
Context ctx;
|
|
|
|
// Loop over the batches and assert the data is as expected
|
|
size_t iter = 0;
|
|
for (auto const &col_batch : dmat->GetBatches<xgboost::SortedCSCPage>(&ctx)) {
|
|
auto col_page = col_batch.GetView();
|
|
ASSERT_EQ(col_page.Size(), dmat->Info().num_col_);
|
|
if (iter == 1) {
|
|
ASSERT_EQ(col_page[0][0].fvalue, 0.f);
|
|
ASSERT_EQ(col_page[3][0].fvalue, 30.f);
|
|
ASSERT_EQ(col_page[3][0].index, 1);
|
|
ASSERT_EQ(col_page[3].size(), 1);
|
|
} else {
|
|
ASSERT_EQ(col_page[1][0].fvalue, 10.0f);
|
|
ASSERT_EQ(col_page[1].size(), 1);
|
|
}
|
|
CHECK_LE(col_batch.base_rowid, dmat->Info().num_row_);
|
|
++iter;
|
|
}
|
|
|
|
// Loop over the batches and assert the data is as expected
|
|
iter = 0;
|
|
for (auto const &col_batch : dmat->GetBatches<xgboost::CSCPage>(&ctx)) {
|
|
auto col_page = col_batch.GetView();
|
|
EXPECT_EQ(col_page.Size(), dmat->Info().num_col_);
|
|
if (iter == 0) {
|
|
EXPECT_EQ(col_page[1][0].fvalue, 10.0f);
|
|
EXPECT_EQ(col_page[1].size(), 1);
|
|
} else {
|
|
EXPECT_EQ(col_page[3][0].fvalue, 30.f);
|
|
EXPECT_EQ(col_page[3].size(), 1);
|
|
}
|
|
iter++;
|
|
}
|
|
delete dmat;
|
|
}
|
|
|
|
TEST(SparsePageDMatrix, ThreadSafetyException) {
|
|
Context ctx;
|
|
|
|
auto dmat =
|
|
RandomDataGenerator{4096, 12, 0.0f}.Batches(8).GenerateSparsePageDMatrix("temp", true);
|
|
|
|
int threads = 1000;
|
|
|
|
std::vector<std::future<void>> waiting;
|
|
|
|
std::atomic<bool> exception {false};
|
|
|
|
for (int32_t i = 0; i < threads; ++i) {
|
|
waiting.emplace_back(std::async(std::launch::async, [&]() {
|
|
try {
|
|
auto iter = dmat->GetBatches<SparsePage>().begin();
|
|
++iter;
|
|
} catch (...) {
|
|
exception.store(true);
|
|
}
|
|
}));
|
|
}
|
|
|
|
using namespace std::chrono_literals;
|
|
|
|
while (std::any_of(waiting.cbegin(), waiting.cend(), [](auto const &f) {
|
|
return f.wait_for(0ms) != std::future_status::ready;
|
|
})) {
|
|
std::this_thread::sleep_for(50ms);
|
|
}
|
|
|
|
CHECK(exception);
|
|
}
|
|
|
|
// Multi-batches access
|
|
TEST(SparsePageDMatrix, ColAccessBatches) {
|
|
// Create multiple sparse pages
|
|
auto dmat =
|
|
RandomDataGenerator{1024, 32, 0.4f}.Batches(3).GenerateSparsePageDMatrix("temp", true);
|
|
ASSERT_EQ(dmat->Ctx()->Threads(), AllThreadsForTest());
|
|
Context ctx;
|
|
for (auto const &page : dmat->GetBatches<xgboost::CSCPage>(&ctx)) {
|
|
ASSERT_EQ(dmat->Info().num_col_, page.Size());
|
|
}
|
|
}
|
|
|
|
auto TestSparsePageDMatrixDeterminism(int32_t threads) {
|
|
std::vector<float> sparse_data;
|
|
std::vector<size_t> sparse_rptr;
|
|
std::vector<bst_feature_t> sparse_cids;
|
|
dmlc::TemporaryDirectory tempdir;
|
|
std::string filename = tempdir.path + "/simple.libsvm";
|
|
CreateBigTestData(filename, 1 << 16);
|
|
|
|
data::FileIterator iter(filename + "?format=libsvm", 0, 1);
|
|
std::unique_ptr<DMatrix> sparse{
|
|
new data::SparsePageDMatrix{&iter, iter.Proxy(), data::fileiter::Reset, data::fileiter::Next,
|
|
std::numeric_limits<float>::quiet_NaN(), threads, filename}};
|
|
CHECK(sparse->Ctx()->Threads() == threads || sparse->Ctx()->Threads() == AllThreadsForTest());
|
|
|
|
DMatrixToCSR(sparse.get(), &sparse_data, &sparse_rptr, &sparse_cids);
|
|
|
|
auto cache_name =
|
|
data::MakeId(filename, dynamic_cast<data::SparsePageDMatrix *>(sparse.get())) + ".row.page";
|
|
auto cache = common::LoadSequentialFile(cache_name);
|
|
return cache;
|
|
}
|
|
|
|
TEST(SparsePageDMatrix, Determinism) {
|
|
#if defined(_MSC_VER)
|
|
return;
|
|
#endif // defined(_MSC_VER)
|
|
std::vector<std::vector<char>> caches;
|
|
for (size_t i = 1; i < 18; i += 2) {
|
|
caches.emplace_back(TestSparsePageDMatrixDeterminism(i));
|
|
}
|
|
|
|
for (size_t i = 1; i < caches.size(); ++i) {
|
|
ASSERT_EQ(caches[i], caches.front());
|
|
}
|
|
}
|