Skip to content

Commit

Permalink
ARROW-4178: [C++] Fix TSan and UBSan errors
Browse files Browse the repository at this point in the history
Author: Antoine Pitrou <[email protected]>

Closes #3334 from pitrou/ARROW-4178-tsan-ubsan-fixes and squashes the following commits:

b836f73 <Antoine Pitrou> ARROW-4178:  Fix TSan and UBSan errors
  • Loading branch information
pitrou authored and xhochy committed Jan 8, 2019
1 parent af07f75 commit 4f2f533
Show file tree
Hide file tree
Showing 23 changed files with 213 additions and 55 deletions.
4 changes: 4 additions & 0 deletions cpp/build-support/run-test.sh
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,10 @@ function setup_sanitizers() {
TSAN_OPTIONS="$TSAN_OPTIONS history_size=7"
export TSAN_OPTIONS

UBSAN_OPTIONS="$UBSAN_OPTIONS print_stacktrace=1"
UBSAN_OPTIONS="$UBSAN_OPTIONS suppressions=$ROOT/build-support/ubsan-suppressions.txt"
export UBSAN_OPTIONS

# Enable leak detection even under LLVM 3.4, where it was disabled by default.
# This flag only takes effect when running an ASAN build.
# ASAN_OPTIONS="$ASAN_OPTIONS detect_leaks=1"
Expand Down
19 changes: 19 additions & 0 deletions cpp/build-support/tsan-suppressions.txt
Original file line number Diff line number Diff line change
@@ -0,0 +1,19 @@
# 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.

# Thread leak in CUDA
thread:libcuda.so
16 changes: 16 additions & 0 deletions cpp/build-support/ubsan-suppressions.txt
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
# 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.
10 changes: 9 additions & 1 deletion cpp/src/arrow/compare.cc
Original file line number Diff line number Diff line change
Expand Up @@ -324,7 +324,15 @@ static bool IsEqualPrimitive(const PrimitiveArray& left, const PrimitiveArray& r
right_data = right.values()->data() + right.offset() * byte_width;
}

if (left.null_count() > 0) {
if (byte_width == 0) {
// Special case 0-width data, as the data pointers may be null
for (int64_t i = 0; i < left.length(); ++i) {
if (left.IsNull(i) != right.IsNull(i)) {
return false;
}
}
return true;
} else if (left.null_count() > 0) {
for (int64_t i = 0; i < left.length(); ++i) {
const bool left_null = left.IsNull(i);
const bool right_null = right.IsNull(i);
Expand Down
1 change: 1 addition & 0 deletions cpp/src/arrow/compute/kernels/cast.cc
Original file line number Diff line number Diff line change
Expand Up @@ -404,6 +404,7 @@ struct is_float_truncate<

template <typename O, typename I>
struct CastFunctor<O, I, typename std::enable_if<is_float_truncate<O, I>::value>::type> {
ARROW_DISABLE_UBSAN("float-cast-overflow")
void operator()(FunctionContext* ctx, const CastOptions& options,
const ArrayData& input, ArrayData* output) {
using in_type = typename I::c_type;
Expand Down
4 changes: 2 additions & 2 deletions cpp/src/arrow/csv/column-builder.cc
Original file line number Diff line number Diff line change
Expand Up @@ -305,12 +305,12 @@ Status InferringColumnBuilder::TryConvertChunk(size_t chunk_index) {

void InferringColumnBuilder::Insert(int64_t block_index,
const std::shared_ptr<BlockParser>& parser) {
DCHECK_NE(converter_, nullptr);

// Create a slot for the new chunk and spawn a task to convert it
size_t chunk_index = static_cast<size_t>(block_index);
{
std::lock_guard<std::mutex> lock(mutex_);

DCHECK_NE(converter_, nullptr);
if (chunks_.size() <= chunk_index) {
chunks_.resize(chunk_index + 1);
}
Expand Down
4 changes: 2 additions & 2 deletions cpp/src/arrow/io/file-test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -468,10 +468,10 @@ class MyMemoryPool : public MemoryPool {

int64_t bytes_allocated() const override { return -1; }

int64_t num_allocations() const { return num_allocations_; }
int64_t num_allocations() const { return num_allocations_.load(); }

private:
int64_t num_allocations_;
std::atomic<int64_t> num_allocations_;
};

TEST_F(TestReadableFile, CustomMemoryPool) {
Expand Down
56 changes: 51 additions & 5 deletions cpp/src/arrow/io/readahead-test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
#include <cstring>
#include <functional>
#include <memory>
#include <mutex>
#include <set>
#include <string>
#include <thread>
Expand All @@ -45,6 +46,51 @@ using internal::checked_cast;
namespace io {
namespace internal {

class LockedInputStream : public InputStream {
public:
explicit LockedInputStream(const std::shared_ptr<InputStream>& stream)
: stream_(stream) {}

Status Close() override {
std::lock_guard<std::mutex> lock(mutex_);
return stream_->Close();
}

bool closed() const override {
std::lock_guard<std::mutex> lock(mutex_);
return stream_->closed();
}

Status Tell(int64_t* position) const override {
std::lock_guard<std::mutex> lock(mutex_);
return stream_->Tell(position);
}

Status Read(int64_t nbytes, int64_t* bytes_read, void* buffer) override {
std::lock_guard<std::mutex> lock(mutex_);
return stream_->Read(nbytes, bytes_read, buffer);
}

Status Read(int64_t nbytes, std::shared_ptr<Buffer>* out) override {
std::lock_guard<std::mutex> lock(mutex_);
return stream_->Read(nbytes, out);
}

bool supports_zero_copy() const override {
std::lock_guard<std::mutex> lock(mutex_);
return stream_->supports_zero_copy();
}

util::string_view Peek(int64_t nbytes) const override {
std::lock_guard<std::mutex> lock(mutex_);
return stream_->Peek(nbytes);
}

protected:
std::shared_ptr<InputStream> stream_;
mutable std::mutex mutex_;
};

static void sleep_for(double seconds) {
std::this_thread::sleep_for(
std::chrono::nanoseconds(static_cast<int64_t>(seconds * 1e9)));
Expand All @@ -57,13 +103,13 @@ static void busy_wait(double seconds, std::function<bool()> predicate) {
}
}

std::shared_ptr<BufferReader> DataReader(const std::string& data) {
std::shared_ptr<InputStream> DataReader(const std::string& data) {
std::shared_ptr<Buffer> buffer;
ABORT_NOT_OK(Buffer::FromString(data, &buffer));
return std::make_shared<BufferReader>(buffer);
return std::make_shared<LockedInputStream>(std::make_shared<BufferReader>(buffer));
}

static int64_t WaitForPosition(const RandomAccessFile& file, int64_t expected,
static int64_t WaitForPosition(const FileInterface& file, int64_t expected,
double seconds = 0.2) {
int64_t pos = -1;
busy_wait(seconds, [&]() -> bool {
Expand All @@ -73,12 +119,12 @@ static int64_t WaitForPosition(const RandomAccessFile& file, int64_t expected,
return pos;
}

static void AssertEventualPosition(const RandomAccessFile& file, int64_t expected) {
static void AssertEventualPosition(const FileInterface& file, int64_t expected) {
int64_t pos = WaitForPosition(file, expected);
ASSERT_EQ(pos, expected) << "File didn't reach expected position";
}

static void AssertPosition(const RandomAccessFile& file, int64_t expected) {
static void AssertPosition(const FileInterface& file, int64_t expected) {
int64_t pos = -1;
ABORT_NOT_OK(file.Tell(&pos));
ASSERT_EQ(pos, expected) << "File didn't reach expected position";
Expand Down
3 changes: 2 additions & 1 deletion cpp/src/arrow/util/bit-stream-utils.h
Original file line number Diff line number Diff line change
Expand Up @@ -397,7 +397,8 @@ inline bool BitReader::GetVlqInt(int32_t* v) {
}

inline bool BitWriter::PutZigZagVlqInt(int32_t v) {
uint32_t u = (v << 1) ^ (v >> 31);
// Note negative left shift is undefined
uint32_t u = (static_cast<uint32_t>(v) << 1) ^ (v >> 31);
return PutVlqInt(u);
}

Expand Down
2 changes: 2 additions & 0 deletions cpp/src/arrow/util/bit-util-test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -756,7 +756,9 @@ static void TestZigZag(int32_t v) {
TEST(BitStreamUtil, ZigZag) {
TestZigZag(0);
TestZigZag(1);
TestZigZag(1234);
TestZigZag(-1);
TestZigZag(-1234);
TestZigZag(std::numeric_limits<int32_t>::max());
TestZigZag(-std::numeric_limits<int32_t>::max());
}
Expand Down
4 changes: 2 additions & 2 deletions cpp/src/arrow/util/decimal-test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -417,8 +417,8 @@ TEST(Decimal128Test, TestFromBigEndian) {
auto negated = -value;
little_endian = negated.ToBytes();
std::reverse(little_endian.begin(), little_endian.end());
// Convert all of the bytes since we have to include the sign bit
ASSERT_OK(Decimal128::FromBigEndian(little_endian.data(), 16, &out));
// The sign bit is looked up in the MSB
ASSERT_OK(Decimal128::FromBigEndian(little_endian.data() + 15 - ii, ii + 1, &out));
ASSERT_EQ(negated, out);

// Take the complement and convert to big endian
Expand Down
16 changes: 10 additions & 6 deletions cpp/src/arrow/util/decimal.cc
Original file line number Diff line number Diff line change
Expand Up @@ -29,11 +29,15 @@
#include "arrow/status.h"
#include "arrow/util/bit-util.h"
#include "arrow/util/decimal.h"
#include "arrow/util/int-util.h"
#include "arrow/util/logging.h"
#include "arrow/util/macros.h"

namespace arrow {

using internal::SafeLeftShift;
using internal::SafeSignedAdd;

static const Decimal128 ScaleMultipliers[] = {
Decimal128(0LL),
Decimal128(10LL),
Expand Down Expand Up @@ -405,7 +409,7 @@ Decimal128& Decimal128::Negate() {
low_bits_ = ~low_bits_ + 1;
high_bits_ = ~high_bits_;
if (low_bits_ == 0) {
++high_bits_;
high_bits_ = SafeSignedAdd<int64_t>(high_bits_, 1);
}
return *this;
}
Expand All @@ -414,9 +418,9 @@ Decimal128& Decimal128::Abs() { return *this < 0 ? Negate() : *this; }

Decimal128& Decimal128::operator+=(const Decimal128& right) {
const uint64_t sum = low_bits_ + right.low_bits_;
high_bits_ += right.high_bits_;
high_bits_ = SafeSignedAdd<int64_t>(high_bits_, right.high_bits_);
if (sum < low_bits_) {
++high_bits_;
high_bits_ = SafeSignedAdd<int64_t>(high_bits_, 1);
}
low_bits_ = sum;
return *this;
Expand Down Expand Up @@ -454,7 +458,7 @@ Decimal128& Decimal128::operator&=(const Decimal128& right) {
Decimal128& Decimal128::operator<<=(uint32_t bits) {
if (bits != 0) {
if (bits < 64) {
high_bits_ <<= bits;
high_bits_ = SafeLeftShift(high_bits_, bits);
high_bits_ |= (low_bits_ >> (64 - bits));
low_bits_ <<= bits;
} else if (bits < 128) {
Expand Down Expand Up @@ -925,7 +929,7 @@ Status Decimal128::FromBigEndian(const uint8_t* bytes, int32_t length, Decimal12
} else {
high = -1 * (is_negative && length < kMaxDecimalBytes);
// Shift left enough bits to make room for the incoming int64_t
high <<= high_bits_offset * CHAR_BIT;
high = SafeLeftShift(high, high_bits_offset * CHAR_BIT);
// Preserve the upper bits by inplace OR-ing the int64_t
high |= high_bits;
}
Expand All @@ -943,7 +947,7 @@ Status Decimal128::FromBigEndian(const uint8_t* bytes, int32_t length, Decimal12
// Sign extend the low bits if necessary
low = -1 * (is_negative && length < 8);
// Shift left enough bits to make room for the incoming int64_t
low <<= low_bits_offset * CHAR_BIT;
low = SafeLeftShift(low, low_bits_offset * CHAR_BIT);
// Preserve the upper bits by inplace OR-ing the int64_t
low |= low_bits;
}
Expand Down
16 changes: 16 additions & 0 deletions cpp/src/arrow/util/int-util.h
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
#define ARROW_UTIL_INT_UTIL_H

#include <cstdint>
#include <type_traits>

#include "arrow/util/visibility.h"

Expand Down Expand Up @@ -67,6 +68,21 @@ template <typename InputInt, typename OutputInt>
ARROW_EXPORT void TransposeInts(const InputInt* source, OutputInt* dest, int64_t length,
const int32_t* transpose_map);

/// Signed addition with well-defined behaviour on overflow (as unsigned)
template <typename SignedInt>
SignedInt SafeSignedAdd(SignedInt u, SignedInt v) {
using UnsignedInt = typename std::make_unsigned<SignedInt>::type;
return static_cast<SignedInt>(static_cast<UnsignedInt>(u) +
static_cast<UnsignedInt>(v));
}

/// Signed left shift with well-defined behaviour on negative numbers or overflow
template <typename SignedInt, typename Shift>
SignedInt SafeLeftShift(SignedInt u, Shift shift) {
using UnsignedInt = typename std::make_unsigned<SignedInt>::type;
return static_cast<SignedInt>(static_cast<UnsignedInt>(u) << shift);
}

} // namespace internal
} // namespace arrow

Expand Down
9 changes: 9 additions & 0 deletions cpp/src/arrow/util/macros.h
Original file line number Diff line number Diff line change
Expand Up @@ -113,6 +113,15 @@
#endif
#endif // !defined(MANUALLY_ALIGNED_STRUCT)

// ----------------------------------------------------------------------
// Convenience macro disabling a particular UBSan check in a function

#if defined(__clang__)
#define ARROW_DISABLE_UBSAN(feature) __attribute__((no_sanitize(feature)))
#else
#define ARROW_DISABLE_UBSAN(feature)
#endif

// ----------------------------------------------------------------------
// From googletest
// (also in parquet-cpp)
Expand Down
5 changes: 4 additions & 1 deletion cpp/src/arrow/util/parsing.h
Original file line number Diff line number Diff line change
Expand Up @@ -335,7 +335,10 @@ class StringToSignedIntConverterMixin {
if (ARROW_PREDICT_FALSE(unsigned_value > max_negative)) {
return false;
}
*out = static_cast<value_type>(-static_cast<value_type>(unsigned_value));
// To avoid both compiler warnings (with unsigned negation)
// and undefined behaviour (with signed negation overflow),
// use the expanded formula for 2's complement negation.
*out = static_cast<value_type>(~unsigned_value + 1);
} else {
if (ARROW_PREDICT_FALSE(unsigned_value > max_positive)) {
return false;
Expand Down
3 changes: 2 additions & 1 deletion cpp/src/arrow/util/thread-pool-test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -298,7 +298,8 @@ TEST_F(TestThreadPool, Submit) {

// Test fork safety on Unix

#if !(defined(_WIN32) || defined(ARROW_VALGRIND) || defined(ADDRESS_SANITIZER))
#if !(defined(_WIN32) || defined(ARROW_VALGRIND) || defined(ADDRESS_SANITIZER) || \
defined(THREAD_SANITIZER))
TEST_F(TestThreadPool, ForkSafety) {
pid_t child_pid;
int child_status;
Expand Down
Loading

0 comments on commit 4f2f533

Please sign in to comment.