You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2017/02/27 19:10:32 UTC

arrow git commit: ARROW-588: [C++] Fix some 32 bit compiler warnings

Repository: arrow
Updated Branches:
  refs/heads/master 01a67f3ff -> 2c3bd9311


ARROW-588: [C++] Fix some 32 bit compiler warnings

I also found that $CMAKE_CXX_FLAGS were not being passed to the gflags external project.

Author: Wes McKinney <we...@twosigma.com>

Closes #354 from wesm/32-bit-compiler-warnings and squashes the following commits:

8829a58 [Wes McKinney] Fix cast to wrong type
5a17654 [Wes McKinney] clang format
43687c5 [Wes McKinney] Fix some more compiler warnings
843479c [Wes McKinney] Fixes
9dbd619 [Wes McKinney] 32 bit fixes


Project: http://git-wip-us.apache.org/repos/asf/arrow/repo
Commit: http://git-wip-us.apache.org/repos/asf/arrow/commit/2c3bd931
Tree: http://git-wip-us.apache.org/repos/asf/arrow/tree/2c3bd931
Diff: http://git-wip-us.apache.org/repos/asf/arrow/diff/2c3bd931

Branch: refs/heads/master
Commit: 2c3bd9311b370a45bac3ff90ed2f772991f211e0
Parents: 01a67f3
Author: Wes McKinney <we...@twosigma.com>
Authored: Mon Feb 27 14:09:39 2017 -0500
Committer: Wes McKinney <we...@twosigma.com>
Committed: Mon Feb 27 14:09:39 2017 -0500

----------------------------------------------------------------------
 cpp/CMakeLists.txt                    |  6 +-----
 cpp/src/arrow/array-primitive-test.cc | 11 +++++------
 cpp/src/arrow/buffer.cc               | 15 ++++++++++++++-
 cpp/src/arrow/buffer.h                | 15 +++------------
 cpp/src/arrow/builder.cc              | 18 +++++++++++-------
 cpp/src/arrow/compare.cc              |  9 ++++++---
 cpp/src/arrow/io/file.cc              | 17 +++++++++--------
 cpp/src/arrow/io/test-common.h        |  4 ++--
 cpp/src/arrow/memory_pool.cc          |  8 +++++---
 cpp/src/arrow/test-util.h             |  4 ++--
 cpp/src/arrow/util/bit-util.cc        |  4 ++--
 11 files changed, 60 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/2c3bd931/cpp/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt
index f6dab78..7d1f9e1 100644
--- a/cpp/CMakeLists.txt
+++ b/cpp/CMakeLists.txt
@@ -499,11 +499,7 @@ if(ARROW_BUILD_TESTS)
 
   # gflags (formerly Googleflags) command line parsing
   if("$ENV{GFLAGS_HOME}" STREQUAL "")
-    if(APPLE)
-      set(GFLAGS_CMAKE_CXX_FLAGS "-fPIC -std=c++11 -stdlib=libc++")
-    else()
-      set(GFLAGS_CMAKE_CXX_FLAGS "-fPIC")
-    endif()
+    set(GFLAGS_CMAKE_CXX_FLAGS ${EP_CXX_FLAGS})
 
     set(GFLAGS_PREFIX "${CMAKE_CURRENT_BINARY_DIR}/gflags_ep-prefix/src/gflags_ep")
     set(GFLAGS_HOME "${GFLAGS_PREFIX}")

http://git-wip-us.apache.org/repos/asf/arrow/blob/2c3bd931/cpp/src/arrow/array-primitive-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array-primitive-test.cc b/cpp/src/arrow/array-primitive-test.cc
index 7b36275..dfa37a8 100644
--- a/cpp/src/arrow/array-primitive-test.cc
+++ b/cpp/src/arrow/array-primitive-test.cc
@@ -100,7 +100,7 @@ class TestPrimitiveBuilder : public TestBuilder {
   void RandomData(int64_t N, double pct_null = 0.1) {
     Attrs::draw(N, &draws_);
 
-    valid_bytes_.resize(N);
+    valid_bytes_.resize(static_cast<size_t>(N));
     test::random_null_bytes(N, pct_null, valid_bytes_.data());
   }
 
@@ -192,8 +192,8 @@ struct PBoolean {
 
 template <>
 void TestPrimitiveBuilder<PBoolean>::RandomData(int64_t N, double pct_null) {
-  draws_.resize(N);
-  valid_bytes_.resize(N);
+  draws_.resize(static_cast<size_t>(N));
+  valid_bytes_.resize(static_cast<size_t>(N));
 
   test::random_null_bytes(N, 0.5, draws_.data());
   test::random_null_bytes(N, pct_null, valid_bytes_.data());
@@ -394,10 +394,9 @@ TYPED_TEST(TestPrimitiveBuilder, TestAppendScalar) {
   this->builder_->Reserve(1000);
   this->builder_nn_->Reserve(1000);
 
-  int64_t i;
   int64_t null_count = 0;
   // Append the first 1000
-  for (i = 0; i < 1000; ++i) {
+  for (size_t i = 0; i < 1000; ++i) {
     if (valid_bytes[i] > 0) {
       this->builder_->Append(draws[i]);
     } else {
@@ -419,7 +418,7 @@ TYPED_TEST(TestPrimitiveBuilder, TestAppendScalar) {
   this->builder_nn_->Reserve(size - 1000);
 
   // Append the next 9000
-  for (i = 1000; i < size; ++i) {
+  for (size_t i = 1000; i < size; ++i) {
     if (valid_bytes[i] > 0) {
       this->builder_->Append(draws[i]);
     } else {

http://git-wip-us.apache.org/repos/asf/arrow/blob/2c3bd931/cpp/src/arrow/buffer.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/buffer.cc b/cpp/src/arrow/buffer.cc
index 18e9ed2..a0b78ac 100644
--- a/cpp/src/arrow/buffer.cc
+++ b/cpp/src/arrow/buffer.cc
@@ -45,7 +45,7 @@ Status Buffer::Copy(
   auto new_buffer = std::make_shared<PoolBuffer>(pool);
   RETURN_NOT_OK(new_buffer->Resize(nbytes));
 
-  std::memcpy(new_buffer->mutable_data(), data() + start, nbytes);
+  std::memcpy(new_buffer->mutable_data(), data() + start, static_cast<size_t>(nbytes));
 
   *out = new_buffer;
   return Status::OK();
@@ -55,6 +55,19 @@ Status Buffer::Copy(int64_t start, int64_t nbytes, std::shared_ptr<Buffer>* out)
   return Copy(start, nbytes, default_memory_pool(), out);
 }
 
+bool Buffer::Equals(const Buffer& other, int64_t nbytes) const {
+  return this == &other ||
+         (size_ >= nbytes && other.size_ >= nbytes &&
+             (data_ == other.data_ ||
+                 !memcmp(data_, other.data_, static_cast<size_t>(nbytes))));
+}
+
+bool Buffer::Equals(const Buffer& other) const {
+  return this == &other || (size_ == other.size_ && (data_ == other.data_ ||
+                                                        !memcmp(data_, other.data_,
+                                                            static_cast<size_t>(size_))));
+}
+
 std::shared_ptr<Buffer> SliceBuffer(
     const std::shared_ptr<Buffer>& buffer, int64_t offset, int64_t length) {
   DCHECK_LE(offset, buffer->size());

http://git-wip-us.apache.org/repos/asf/arrow/blob/2c3bd931/cpp/src/arrow/buffer.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/buffer.h b/cpp/src/arrow/buffer.h
index be91af3..0724385 100644
--- a/cpp/src/arrow/buffer.h
+++ b/cpp/src/arrow/buffer.h
@@ -64,17 +64,8 @@ class ARROW_EXPORT Buffer : public std::enable_shared_from_this<Buffer> {
 
   /// Return true if both buffers are the same size and contain the same bytes
   /// up to the number of compared bytes
-  bool Equals(const Buffer& other, int64_t nbytes) const {
-    return this == &other ||
-           (size_ >= nbytes && other.size_ >= nbytes &&
-               (data_ == other.data_ || !memcmp(data_, other.data_, nbytes)));
-  }
-
-  bool Equals(const Buffer& other) const {
-    return this == &other ||
-           (size_ == other.size_ &&
-               (data_ == other.data_ || !memcmp(data_, other.data_, size_)));
-  }
+  bool Equals(const Buffer& other, int64_t nbytes) const;
+  bool Equals(const Buffer& other) const;
 
   /// Copy a section of the buffer into a new Buffer.
   Status Copy(int64_t start, int64_t nbytes, MemoryPool* pool,
@@ -196,7 +187,7 @@ class ARROW_EXPORT BufferBuilder {
 
   // Unsafe methods don't check existing size
   void UnsafeAppend(const uint8_t* data, int64_t length) {
-    memcpy(data_ + size_, data, length);
+    memcpy(data_ + size_, data, static_cast<size_t>(length));
     size_ += length;
   }
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/2c3bd931/cpp/src/arrow/builder.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/builder.cc b/cpp/src/arrow/builder.cc
index 63e083e..9086598 100644
--- a/cpp/src/arrow/builder.cc
+++ b/cpp/src/arrow/builder.cc
@@ -58,7 +58,7 @@ Status ArrayBuilder::Init(int64_t capacity) {
   const int64_t byte_capacity = null_bitmap_->capacity();
   capacity_ = capacity;
   null_bitmap_data_ = null_bitmap_->mutable_data();
-  memset(null_bitmap_data_, 0, byte_capacity);
+  memset(null_bitmap_data_, 0, static_cast<size_t>(byte_capacity));
   return Status::OK();
 }
 
@@ -72,7 +72,8 @@ Status ArrayBuilder::Resize(int64_t new_bits) {
   const int64_t byte_capacity = null_bitmap_->capacity();
   capacity_ = new_bits;
   if (old_bytes < new_bytes) {
-    memset(null_bitmap_data_ + old_bytes, 0, byte_capacity - old_bytes);
+    memset(
+        null_bitmap_data_ + old_bytes, 0, static_cast<size_t>(byte_capacity - old_bytes));
   }
   return Status::OK();
 }
@@ -152,7 +153,8 @@ void ArrayBuilder::UnsafeSetNotNull(int64_t length) {
 
   // Fast bitsetting
   int64_t fast_length = (length - pad_to_byte) / 8;
-  memset(null_bitmap_data_ + ((length_ + pad_to_byte) / 8), 255, fast_length);
+  memset(null_bitmap_data_ + ((length_ + pad_to_byte) / 8), 255,
+      static_cast<size_t>(fast_length));
 
   // Trailing bytes
   for (int64_t i = length_ + pad_to_byte + (fast_length * 8); i < new_length; ++i) {
@@ -170,7 +172,7 @@ Status PrimitiveBuilder<T>::Init(int64_t capacity) {
   int64_t nbytes = TypeTraits<T>::bytes_required(capacity);
   RETURN_NOT_OK(data_->Resize(nbytes));
   // TODO(emkornfield) valgrind complains without this
-  memset(data_->mutable_data(), 0, nbytes);
+  memset(data_->mutable_data(), 0, static_cast<size_t>(nbytes));
 
   raw_data_ = reinterpret_cast<value_type*>(data_->mutable_data());
   return Status::OK();
@@ -190,7 +192,8 @@ Status PrimitiveBuilder<T>::Resize(int64_t capacity) {
     RETURN_NOT_OK(data_->Resize(new_bytes));
     raw_data_ = reinterpret_cast<value_type*>(data_->mutable_data());
     // TODO(emkornfield) valgrind complains without this
-    memset(data_->mutable_data() + old_bytes, 0, new_bytes - old_bytes);
+    memset(
+        data_->mutable_data() + old_bytes, 0, static_cast<size_t>(new_bytes - old_bytes));
   }
   return Status::OK();
 }
@@ -256,7 +259,7 @@ Status BooleanBuilder::Init(int64_t capacity) {
   int64_t nbytes = BitUtil::BytesForBits(capacity);
   RETURN_NOT_OK(data_->Resize(nbytes));
   // TODO(emkornfield) valgrind complains without this
-  memset(data_->mutable_data(), 0, nbytes);
+  memset(data_->mutable_data(), 0, static_cast<size_t>(nbytes));
 
   raw_data_ = reinterpret_cast<uint8_t*>(data_->mutable_data());
   return Status::OK();
@@ -275,7 +278,8 @@ Status BooleanBuilder::Resize(int64_t capacity) {
 
     RETURN_NOT_OK(data_->Resize(new_bytes));
     raw_data_ = reinterpret_cast<uint8_t*>(data_->mutable_data());
-    memset(data_->mutable_data() + old_bytes, 0, new_bytes - old_bytes);
+    memset(
+        data_->mutable_data() + old_bytes, 0, static_cast<size_t>(new_bytes - old_bytes));
   }
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/arrow/blob/2c3bd931/cpp/src/arrow/compare.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/compare.cc b/cpp/src/arrow/compare.cc
index e94fa74..f38f8d6 100644
--- a/cpp/src/arrow/compare.cc
+++ b/cpp/src/arrow/compare.cc
@@ -87,7 +87,8 @@ class RangeEqualsVisitor : public ArrayVisitor {
 
       if (end_offset - begin_offset > 0 &&
           std::memcmp(left.data()->data() + begin_offset,
-              right.data()->data() + right_begin_offset, end_offset - begin_offset)) {
+              right.data()->data() + right_begin_offset,
+              static_cast<size_t>(end_offset - begin_offset))) {
         return false;
       }
     }
@@ -348,7 +349,8 @@ class ArrayEqualsVisitor : public RangeEqualsVisitor {
       }
       return true;
     } else {
-      return memcmp(left_data, right_data, value_byte_size * left.length()) == 0;
+      return memcmp(left_data, right_data,
+                 static_cast<size_t>(value_byte_size * left.length())) == 0;
     }
   }
 
@@ -431,7 +433,8 @@ class ArrayEqualsVisitor : public RangeEqualsVisitor {
         const int64_t total_bytes =
             left.value_offset(left.length()) - left.value_offset(0);
         return std::memcmp(left_data + left.value_offset(0),
-                   right_data + right.value_offset(0), total_bytes) == 0;
+                   right_data + right.value_offset(0),
+                   static_cast<size_t>(total_bytes)) == 0;
       }
     } else {
       // ARROW-537: Only compare data in non-null slots

http://git-wip-us.apache.org/repos/asf/arrow/blob/2c3bd931/cpp/src/arrow/io/file.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/io/file.cc b/cpp/src/arrow/io/file.cc
index 230c7fe..7c14238 100644
--- a/cpp/src/arrow/io/file.cc
+++ b/cpp/src/arrow/io/file.cc
@@ -244,9 +244,9 @@ static inline Status FileRead(
     int fd, uint8_t* buffer, int64_t nbytes, int64_t* bytes_read) {
 #if defined(_MSC_VER)
   if (nbytes > INT32_MAX) { return Status::IOError("Unable to read > 2GB blocks yet"); }
-  *bytes_read = _read(fd, buffer, static_cast<unsigned int>(nbytes));
+  *bytes_read = _read(fd, buffer, static_cast<size_t>(nbytes));
 #else
-  *bytes_read = read(fd, buffer, nbytes);
+  *bytes_read = read(fd, buffer, static_cast<size_t>(nbytes));
 #endif
 
   if (*bytes_read == -1) {
@@ -263,9 +263,9 @@ static inline Status FileWrite(int fd, const uint8_t* buffer, int64_t nbytes) {
   if (nbytes > INT32_MAX) {
     return Status::IOError("Unable to write > 2GB blocks to file yet");
   }
-  ret = static_cast<int>(_write(fd, buffer, static_cast<unsigned int>(nbytes)));
+  ret = static_cast<int>(_write(fd, buffer, static_cast<size_t>(nbytes)));
 #else
-  ret = static_cast<int>(write(fd, buffer, nbytes));
+  ret = static_cast<int>(write(fd, buffer, static_cast<size_t>(nbytes)));
 #endif
 
   if (ret == -1) {
@@ -526,7 +526,7 @@ class MemoryMappedFile::MemoryMap : public MutableBuffer {
 
   ~MemoryMap() {
     if (file_->is_open()) {
-      munmap(mutable_data_, size_);
+      munmap(mutable_data_, static_cast<size_t>(size_));
       file_->Close();
     }
   }
@@ -554,7 +554,8 @@ class MemoryMappedFile::MemoryMap : public MutableBuffer {
       is_mutable_ = false;
     }
 
-    void* result = mmap(nullptr, file_->size(), prot_flags, map_mode, file_->fd(), 0);
+    void* result = mmap(nullptr, static_cast<size_t>(file_->size()), prot_flags, map_mode,
+        file_->fd(), 0);
     if (result == MAP_FAILED) {
       std::stringstream ss;
       ss << "Memory mapping file failed, errno: " << errno;
@@ -630,7 +631,7 @@ Status MemoryMappedFile::Close() {
 Status MemoryMappedFile::Read(int64_t nbytes, int64_t* bytes_read, uint8_t* out) {
   nbytes = std::max<int64_t>(
       0, std::min(nbytes, memory_map_->size() - memory_map_->position()));
-  if (nbytes > 0) { std::memcpy(out, memory_map_->head(), nbytes); }
+  if (nbytes > 0) { std::memcpy(out, memory_map_->head(), static_cast<size_t>(nbytes)); }
   *bytes_read = nbytes;
   memory_map_->advance(nbytes);
   return Status::OK();
@@ -677,7 +678,7 @@ Status MemoryMappedFile::Write(const uint8_t* data, int64_t nbytes) {
 }
 
 Status MemoryMappedFile::WriteInternal(const uint8_t* data, int64_t nbytes) {
-  memcpy(memory_map_->head(), data, nbytes);
+  memcpy(memory_map_->head(), data, static_cast<size_t>(nbytes));
   memory_map_->advance(nbytes);
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/arrow/blob/2c3bd931/cpp/src/arrow/io/test-common.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/io/test-common.h b/cpp/src/arrow/io/test-common.h
index 6e91713..8355714 100644
--- a/cpp/src/arrow/io/test-common.h
+++ b/cpp/src/arrow/io/test-common.h
@@ -53,9 +53,9 @@ class MemoryMapFixture {
     FILE* file = fopen(path.c_str(), "w");
     if (file != nullptr) { tmp_files_.push_back(path); }
 #ifdef _MSC_VER
-    _chsize(fileno(file), size);
+    _chsize(fileno(file), static_cast<size_t>(size));
 #else
-    ftruncate(fileno(file), size);
+    ftruncate(fileno(file), static_cast<size_t>(size));
 #endif
     fclose(file);
   }

http://git-wip-us.apache.org/repos/asf/arrow/blob/2c3bd931/cpp/src/arrow/memory_pool.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/memory_pool.cc b/cpp/src/arrow/memory_pool.cc
index 8d85a08..5a63027 100644
--- a/cpp/src/arrow/memory_pool.cc
+++ b/cpp/src/arrow/memory_pool.cc
@@ -36,14 +36,16 @@ Status AllocateAligned(int64_t size, uint8_t** out) {
   constexpr size_t kAlignment = 64;
 #ifdef _MSC_VER
   // Special code path for MSVC
-  *out = reinterpret_cast<uint8_t*>(_aligned_malloc(size, kAlignment));
+  *out =
+      reinterpret_cast<uint8_t*>(_aligned_malloc(static_cast<size_t>(size), kAlignment));
   if (!*out) {
     std::stringstream ss;
     ss << "malloc of size " << size << " failed";
     return Status::OutOfMemory(ss.str());
   }
 #else
-  const int result = posix_memalign(reinterpret_cast<void**>(out), kAlignment, size);
+  const int result = posix_memalign(
+      reinterpret_cast<void**>(out), kAlignment, static_cast<size_t>(size));
   if (result == ENOMEM) {
     std::stringstream ss;
     ss << "malloc of size " << size << " failed";
@@ -90,7 +92,7 @@ Status DefaultMemoryPool::Reallocate(int64_t old_size, int64_t new_size, uint8_t
   uint8_t* out;
   RETURN_NOT_OK(AllocateAligned(new_size, &out));
   // Copy contents and release old memory chunk
-  memcpy(out, *ptr, std::min(new_size, old_size));
+  memcpy(out, *ptr, static_cast<size_t>(std::min(new_size, old_size)));
 #ifdef _MSC_VER
   _aligned_free(*ptr);
 #else

http://git-wip-us.apache.org/repos/asf/arrow/blob/2c3bd931/cpp/src/arrow/test-util.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/test-util.h b/cpp/src/arrow/test-util.h
index 5c7d04d..11ce50a 100644
--- a/cpp/src/arrow/test-util.h
+++ b/cpp/src/arrow/test-util.h
@@ -109,13 +109,13 @@ inline Status CopyBufferFromVector(
 
 static inline Status GetBitmapFromBoolVector(
     const std::vector<bool>& is_valid, std::shared_ptr<Buffer>* result) {
-  int64_t length = static_cast<int64_t>(is_valid.size());
+  size_t length = is_valid.size();
 
   std::shared_ptr<MutableBuffer> buffer;
   RETURN_NOT_OK(GetEmptyBitmap(default_memory_pool(), length, &buffer));
 
   uint8_t* bitmap = buffer->mutable_data();
-  for (int64_t i = 0; i < length; ++i) {
+  for (size_t i = 0; i < static_cast<size_t>(length); ++i) {
     if (is_valid[i]) { BitUtil::SetBit(bitmap, i); }
   }
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/2c3bd931/cpp/src/arrow/util/bit-util.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/bit-util.cc b/cpp/src/arrow/util/bit-util.cc
index 1bbd238..3767ba9 100644
--- a/cpp/src/arrow/util/bit-util.cc
+++ b/cpp/src/arrow/util/bit-util.cc
@@ -47,7 +47,7 @@ Status BitUtil::BytesToBits(
   std::shared_ptr<MutableBuffer> buffer;
   RETURN_NOT_OK(AllocateBuffer(default_memory_pool(), bit_length, &buffer));
 
-  memset(buffer->mutable_data(), 0, bit_length);
+  memset(buffer->mutable_data(), 0, static_cast<size_t>(bit_length));
   BytesToBits(bytes, buffer->mutable_data());
 
   *out = buffer;
@@ -94,7 +94,7 @@ int64_t CountSetBits(const uint8_t* data, int64_t bit_offset, int64_t length) {
 Status GetEmptyBitmap(
     MemoryPool* pool, int64_t length, std::shared_ptr<MutableBuffer>* result) {
   RETURN_NOT_OK(AllocateBuffer(pool, BitUtil::BytesForBits(length), result));
-  memset((*result)->mutable_data(), 0, (*result)->size());
+  memset((*result)->mutable_data(), 0, static_cast<size_t>((*result)->size()));
   return Status::OK();
 }