Skip to content
Closed
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 cpp/src/arrow/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ set(ARROW_SRCS
util/hash.cc
util/io-util.cc
util/key_value_metadata.cc
util/thread-pool.cc
)

if ("${COMPILER_FAMILY}" STREQUAL "clang")
Expand Down
12 changes: 2 additions & 10 deletions cpp/src/arrow/io/io-memory-benchmark.cc
Original file line number Diff line number Diff line change
Expand Up @@ -60,16 +60,8 @@ static void BM_ParallelMemcopy(benchmark::State& state) { // NOLINT non-const r
state.SetBytesProcessed(int64_t(state.iterations()) * kTotalSize);
}

BENCHMARK(BM_SerialMemcopy)
->RangeMultiplier(4)
->Range(1, 1 << 13)
->MinTime(1.0)
->UseRealTime();
BENCHMARK(BM_SerialMemcopy)->MinTime(1.0)->Repetitions(2)->UseRealTime();

BENCHMARK(BM_ParallelMemcopy)
->RangeMultiplier(4)
->Range(1, 1 << 13)
->MinTime(1.0)
->UseRealTime();
BENCHMARK(BM_ParallelMemcopy)->MinTime(1.0)->Repetitions(2)->UseRealTime();

} // namespace arrow
11 changes: 10 additions & 1 deletion cpp/src/arrow/symbols.map
Original file line number Diff line number Diff line change
Expand Up @@ -16,12 +16,21 @@
# under the License.

{
global:
extern "C++" {
# Export pthread_once-related symbols so that two SO files
# (e.g. libarrow.so and libplasma.so) don't use separate copies of
# those symbols.
# See https://github.com/apache/arrow/pull/1953#issuecomment-386057063
std::__once*;
};

# Symbols marked as 'local' are not exported by the DSO and thus may not
# be used by client applications.
local:
# devtoolset / static-libstdc++ symbols
__cxa_*;

__once_proxy;

# Static libraries that are linked in e.g. the manylinux1 build
# Brotli compression library
Expand Down
1 change: 1 addition & 0 deletions cpp/src/arrow/util/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,7 @@ ADD_ARROW_TEST(decimal-test)
ADD_ARROW_TEST(key-value-metadata-test)
ADD_ARROW_TEST(rle-encoding-test)
ADD_ARROW_TEST(stl-util-test)
ADD_ARROW_TEST(thread-pool-test)

ADD_ARROW_BENCHMARK(bit-util-benchmark)
ADD_ARROW_BENCHMARK(decimal-benchmark)
Expand Down
23 changes: 13 additions & 10 deletions cpp/src/arrow/util/memory.h
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,8 @@
#include <thread>
#include <vector>

#include "arrow/util/thread-pool.h"

namespace arrow {
namespace internal {

Expand All @@ -33,7 +35,9 @@ uint8_t* pointer_logical_and(const uint8_t* address, uintptr_t bits) {
// to saturate the memory bandwidth of modern cpus.
void parallel_memcopy(uint8_t* dst, const uint8_t* src, int64_t nbytes,
uintptr_t block_size, int num_threads) {
std::vector<std::thread> threadpool(num_threads);
// XXX This function is really using `num_threads + 1` threads.
auto pool = CPUThreadPool();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I commented in the latest PR about the name for this function -- I would either call it GetCpuThreadPool or cpu_thread_pool

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok, will do. Should I open a separate PR for the fixes here or do them as part of the latest PR?


uint8_t* left = pointer_logical_and(src + block_size - 1, ~(block_size - 1));
uint8_t* right = pointer_logical_and(src + nbytes, ~(block_size - 1));
int64_t num_blocks = (right - left) / block_size;
Expand All @@ -42,7 +46,7 @@ void parallel_memcopy(uint8_t* dst, const uint8_t* src, int64_t nbytes,
right = right - (num_blocks % num_threads) * block_size;

// Now we divide these blocks between available threads. The remainder is
// handled on the main thread.
// handled separately.
int64_t chunk_size = (right - left) / num_threads;
int64_t prefix = left - src;
int64_t suffix = src + nbytes - right;
Expand All @@ -51,19 +55,18 @@ void parallel_memcopy(uint8_t* dst, const uint8_t* src, int64_t nbytes,
// | prefix | num_threads * chunk_size | suffix |.
// Each thread gets a "chunk" of k blocks.

// Start all threads first and handle leftovers while threads run.
// Start all parallel memcpy tasks and handle leftovers while threads run.
std::vector<std::future<void*>> futures;

for (int i = 0; i < num_threads; i++) {
threadpool[i] = std::thread(memcpy, dst + prefix + i * chunk_size,
left + i * chunk_size, chunk_size);
futures.push_back(pool->Submit(memcpy, dst + prefix + i * chunk_size,
left + i * chunk_size, chunk_size));
}

memcpy(dst, src, prefix);
memcpy(dst + prefix + num_threads * chunk_size, right, suffix);

for (auto& t : threadpool) {
if (t.joinable()) {
t.join();
}
for (auto& fut : futures) {
fut.get();
}
}

Expand Down
275 changes: 275 additions & 0 deletions cpp/src/arrow/util/thread-pool-test.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,275 @@
// 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 <algorithm>
#include <chrono>
#include <functional>
#include <thread>
#include <vector>

#include "arrow/test-util.h"
#include "arrow/util/macros.h"
#include "arrow/util/thread-pool.h"

namespace arrow {
namespace internal {

static void sleep_for(double seconds) {
std::this_thread::sleep_for(
std::chrono::nanoseconds(static_cast<int64_t>(seconds * 1e9)));
}

static void busy_wait(double seconds, std::function<bool()> predicate) {
const double period = 0.001;
for (int i = 0; !predicate() && i * period < seconds; ++i) {
sleep_for(period);
}
}

template <typename T>
static void task_add(T x, T y, T* out) {
*out = x + y;
}

template <typename T>
static void task_slow_add(double seconds, T x, T y, T* out) {
sleep_for(seconds);
*out = x + y;
}

typedef std::function<void(int, int, int*)> AddTaskFunc;

template <typename T>
static T add(T x, T y) {
return x + y;
}

template <typename T>
static T slow_add(double seconds, T x, T y) {
sleep_for(seconds);
return x + y;
}

template <typename T>
static T inplace_add(T& x, T y) {
return x += y;
}

// A class to spawn "add" tasks to a pool and check the results when done

class AddTester {
public:
explicit AddTester(int nadds) : nadds(nadds), xs(nadds), ys(nadds), outs(nadds, -1) {
int x = 0, y = 0;
std::generate(xs.begin(), xs.end(), [&] {
++x;
return x;
});
std::generate(ys.begin(), ys.end(), [&] {
y += 10;
return y;
});
}

AddTester(AddTester&&) = default;

void SpawnTasks(ThreadPool* pool, AddTaskFunc add_func) {
for (int i = 0; i < nadds; ++i) {
ASSERT_OK(pool->Spawn([=] { add_func(xs[i], ys[i], &outs[i]); }));
}
}

void CheckResults() {
for (int i = 0; i < nadds; ++i) {
ASSERT_EQ(outs[i], (i + 1) * 11);
}
}

void CheckNotAllComputed() {
for (int i = 0; i < nadds; ++i) {
if (outs[i] == -1) {
return;
}
}
ASSERT_TRUE(0) << "all values were computed";
}

private:
ARROW_DISALLOW_COPY_AND_ASSIGN(AddTester);

int nadds;
std::vector<int> xs;
std::vector<int> ys;
std::vector<int> outs;
};

class TestThreadPool : public ::testing::Test {
public:
void TearDown() {
fflush(stdout);
fflush(stderr);
}

std::shared_ptr<ThreadPool> MakeThreadPool() { return MakeThreadPool(4); }

std::shared_ptr<ThreadPool> MakeThreadPool(size_t threads) {
std::shared_ptr<ThreadPool> pool;
Status st = ThreadPool::Make(threads, &pool);
return pool;
}

void SpawnAdds(ThreadPool* pool, int nadds, AddTaskFunc add_func) {
AddTester add_tester(nadds);
add_tester.SpawnTasks(pool, add_func);
ASSERT_OK(pool->Shutdown());
add_tester.CheckResults();
}

void SpawnAddsThreaded(ThreadPool* pool, int nthreads, int nadds,
AddTaskFunc add_func) {
// Same as SpawnAdds, but do the task spawning from multiple threads
std::vector<AddTester> add_testers;
std::vector<std::thread> threads;
for (int i = 0; i < nthreads; ++i) {
add_testers.emplace_back(nadds);
}
for (auto& add_tester : add_testers) {
threads.emplace_back([&] { add_tester.SpawnTasks(pool, add_func); });
}
for (auto& thread : threads) {
thread.join();
}
ASSERT_OK(pool->Shutdown());
for (auto& add_tester : add_testers) {
add_tester.CheckResults();
}
}
};

TEST_F(TestThreadPool, ConstructDestruct) {
// Stress shutdown-at-destruction logic
for (size_t threads : {1, 2, 3, 8, 32, 70}) {
auto pool = this->MakeThreadPool(threads);
}
}

// Correctness and stress tests using Spawn() and Shutdown()

TEST_F(TestThreadPool, Spawn) {
auto pool = this->MakeThreadPool(3);
SpawnAdds(pool.get(), 7, task_add<int>);
}

TEST_F(TestThreadPool, StressSpawn) {
auto pool = this->MakeThreadPool(30);
SpawnAdds(pool.get(), 1000, task_add<int>);
}

TEST_F(TestThreadPool, StressSpawnThreaded) {
auto pool = this->MakeThreadPool(30);
SpawnAddsThreaded(pool.get(), 20, 100, task_add<int>);
}

TEST_F(TestThreadPool, SpawnSlow) {
// This checks that Shutdown() waits for all tasks to finish
auto pool = this->MakeThreadPool(2);
SpawnAdds(pool.get(), 7, [](int x, int y, int* out) {
return task_slow_add(0.02 /* seconds */, x, y, out);
});
}

TEST_F(TestThreadPool, StressSpawnSlow) {
auto pool = this->MakeThreadPool(30);
SpawnAdds(pool.get(), 1000, [](int x, int y, int* out) {
return task_slow_add(0.002 /* seconds */, x, y, out);
});
}

TEST_F(TestThreadPool, StressSpawnSlowThreaded) {
auto pool = this->MakeThreadPool(30);
SpawnAddsThreaded(pool.get(), 20, 100, [](int x, int y, int* out) {
return task_slow_add(0.002 /* seconds */, x, y, out);
});
}

TEST_F(TestThreadPool, QuickShutdown) {
AddTester add_tester(100);
{
auto pool = this->MakeThreadPool(3);
add_tester.SpawnTasks(pool.get(), [](int x, int y, int* out) {
return task_slow_add(0.02 /* seconds */, x, y, out);
});
ASSERT_OK(pool->Shutdown(false /* wait */));
add_tester.CheckNotAllComputed();
}
add_tester.CheckNotAllComputed();
}

TEST_F(TestThreadPool, SetCapacity) {
auto pool = this->MakeThreadPool(3);
ASSERT_EQ(pool->GetCapacity(), 3);
ASSERT_OK(pool->SetCapacity(5));
ASSERT_EQ(pool->GetCapacity(), 5);
ASSERT_OK(pool->SetCapacity(2));
// Wait for workers to wake up and secede
busy_wait(0.5, [&] { return pool->GetCapacity() == 2; });
ASSERT_EQ(pool->GetCapacity(), 2);
ASSERT_OK(pool->SetCapacity(5));
ASSERT_EQ(pool->GetCapacity(), 5);
// Downsize while tasks are pending
for (int i = 0; i < 10; ++i) {
ASSERT_OK(pool->Spawn(std::bind(sleep_for, 0.01 /* seconds */)));
}
ASSERT_OK(pool->SetCapacity(2));
busy_wait(0.5, [&] { return pool->GetCapacity() == 2; });
ASSERT_EQ(pool->GetCapacity(), 2);
// Ensure nothing got stuck
ASSERT_OK(pool->Shutdown());
}

// Test Submit() functionality

TEST_F(TestThreadPool, Submit) {
auto pool = this->MakeThreadPool(3);
{
auto fut = pool->Submit(add<int>, 4, 5);
ASSERT_EQ(fut.get(), 9);
}
{
auto fut = pool->Submit(add<std::string>, "foo", "bar");
ASSERT_EQ(fut.get(), "foobar");
}
{
auto fut = pool->Submit(slow_add<int>, 0.01 /* seconds */, 4, 5);
ASSERT_EQ(fut.get(), 9);
}
{
// Reference passing
std::string s = "foo";
auto fut = pool->Submit(inplace_add<std::string>, std::ref(s), "bar");
ASSERT_EQ(fut.get(), "foobar");
ASSERT_EQ(s, "foobar");
}
{
// `void` return type
auto fut = pool->Submit(sleep_for, 0.001);
fut.get();
}
}

} // namespace internal
} // namespace arrow
Loading