You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by ju...@apache.org on 2016/03/01 01:14:37 UTC

parquet-cpp git commit: PARQUET-518: Remove -Wno-sign-compare and scrub integer signedness

Repository: parquet-cpp
Updated Branches:
  refs/heads/master ebb45b1e7 -> 5b3e9c103


PARQUET-518: Remove -Wno-sign-compare and scrub integer signedness

This patch removes compiler warning suppresses, fixes signed-unsigned integer comparisons, and scrubs most usages of `size_t` from the codebase in favor of signed integer types.

Author: Wes McKinney <we...@apache.org>

Closes #63 from wesm/PARQUET-518 and squashes the following commits:

ba74e14 [Wes McKinney] Fix unsigned int comparison after rebase
b6adc51 [Wes McKinney] Scrub more usages of size_t
242ca3f [Wes McKinney] Disable -Wno-sign-compare and do some scrubbing


Project: http://git-wip-us.apache.org/repos/asf/parquet-cpp/repo
Commit: http://git-wip-us.apache.org/repos/asf/parquet-cpp/commit/5b3e9c10
Tree: http://git-wip-us.apache.org/repos/asf/parquet-cpp/tree/5b3e9c10
Diff: http://git-wip-us.apache.org/repos/asf/parquet-cpp/diff/5b3e9c10

Branch: refs/heads/master
Commit: 5b3e9c103ae041688c625d75fb771c8607ce9859
Parents: ebb45b1
Author: Wes McKinney <we...@apache.org>
Authored: Mon Feb 29 16:14:33 2016 -0800
Committer: Julien Le Dem <ju...@dremio.com>
Committed: Mon Feb 29 16:14:33 2016 -0800

----------------------------------------------------------------------
 CMakeLists.txt                                  |  2 +-
 example/decode_benchmark.cc                     |  8 +++---
 src/parquet/column/column-reader-test.cc        |  8 +++---
 src/parquet/column/levels.h                     | 22 +++++++--------
 src/parquet/column/reader.cc                    | 10 +++----
 src/parquet/column/reader.h                     | 28 ++++++++++----------
 src/parquet/column/scanner-test.cc              | 13 ++++-----
 src/parquet/column/scanner.h                    | 20 +++++++-------
 src/parquet/column/test-util.h                  | 10 +++----
 src/parquet/encodings/delta-bit-pack-encoding.h |  2 +-
 src/parquet/encodings/dictionary-encoding.h     | 16 +++++------
 src/parquet/encodings/plain-encoding.h          | 12 ++++-----
 src/parquet/file/file-deserialize-test.cc       |  6 ++---
 src/parquet/file/reader-internal.cc             |  8 +++---
 src/parquet/file/reader.cc                      |  4 +--
 src/parquet/reader-test.cc                      |  6 ++---
 src/parquet/schema/descriptor.cc                |  6 ++---
 src/parquet/schema/descriptor.h                 |  4 +--
 src/parquet/schema/schema-converter-test.cc     |  2 +-
 src/parquet/schema/schema-descriptor-test.cc    |  4 +--
 src/parquet/schema/test-util.h                  |  2 +-
 src/parquet/types.h                             | 18 ++++++-------
 src/parquet/util/bit-util.h                     |  4 +--
 src/parquet/util/mem-pool.cc                    |  6 ++---
 src/parquet/util/rle-encoding.h                 |  2 +-
 src/parquet/util/rle-test.cc                    | 14 +++++-----
 src/parquet/util/test-common.h                  | 19 ++++++-------
 27 files changed, 127 insertions(+), 129 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 5ff9e6c..0076449 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -244,7 +244,7 @@ message(STATUS "Build Type: ${CMAKE_BUILD_TYPE}")
 
 # Build with C++11 and SSE3 by default
 # TODO(wesm): These compiler warning suppressions should be removed one by one
-SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++11 -msse3 -Wall -Wno-unused-value -Wno-unused-variable -Wno-sign-compare -Wno-unknown-pragmas")
+SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++11 -msse3 -Wall -Wno-unused-value -Wno-unused-variable")
 
 
 if (APPLE)

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/example/decode_benchmark.cc
----------------------------------------------------------------------
diff --git a/example/decode_benchmark.cc b/example/decode_benchmark.cc
index ce16588..a4fd697 100644
--- a/example/decode_benchmark.cc
+++ b/example/decode_benchmark.cc
@@ -228,7 +228,7 @@ uint64_t TestBinaryPackedEncoding(const char* name, const vector<int64_t>& value
   }
   DeltaBitPackDecoder<Type::INT64> decoder(nullptr);
   DeltaBitPackEncoder encoder(mini_block_size);
-  for (int i = 0; i < values.size(); ++i) {
+  for (size_t i = 0; i < values.size(); ++i) {
     encoder.Add(values[i]);
   }
 
@@ -262,7 +262,7 @@ uint64_t TestBinaryPackedEncoding(const char* name, const vector<int64_t>& value
     sw.Start();\
     for (int k = 0; k < benchmark_iters; ++k) {
       decoder.SetData(encoder.num_values(), buffer, len);
-      for (int i = 0; i < values.size();) {
+      for (size_t i = 0; i < values.size();) {
         int n = decoder.Decode(buf, benchmark_batch_size);
         for (int j = 0; j < n; ++j) {
           result += buf[j];
@@ -363,7 +363,7 @@ void TestDeltaLengthByteArray() {
   values.push_back("Foobar");
   values.push_back("ABCDEF");
 
-  for (int i = 0; i < values.size(); ++i) {
+  for (size_t i = 0; i < values.size(); ++i) {
     encoder.Add(values[i]);
   }
 
@@ -401,7 +401,7 @@ void TestDeltaByteArray() {
   values.push_back("nacarat");
   values.push_back("nacelle");
 
-  for (int i = 0; i < values.size(); ++i) {
+  for (size_t i = 0; i < values.size(); ++i) {
     encoder.Add(values[i]);
   }
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/column/column-reader-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/column-reader-test.cc b/src/parquet/column/column-reader-test.cc
index e64ef28..a5b918f 100644
--- a/src/parquet/column/column-reader-test.cc
+++ b/src/parquet/column/column-reader-test.cc
@@ -91,13 +91,13 @@ class TestPrimitiveReader : public ::testing::Test {
     vector<int32_t> vresult(num_values_, -1);
     vector<int16_t> dresult(num_levels_, -1);
     vector<int16_t> rresult(num_levels_, -1);
-    size_t values_read = 0;
-    size_t total_values_read = 0;
-    size_t batch_actual = 0;
+    int64_t values_read = 0;
+    int total_values_read = 0;
+    int batch_actual = 0;
 
     Int32Reader* reader = static_cast<Int32Reader*>(reader_.get());
     int32_t batch_size = 8;
-    size_t batch = 0;
+    int batch = 0;
     // This will cover both the cases
     // 1) batch_size < page_size (multiple ReadBatch from a single page)
     // 2) batch_size > page_size (BatchRead limits to a single page)

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/column/levels.h
----------------------------------------------------------------------
diff --git a/src/parquet/column/levels.h b/src/parquet/column/levels.h
index a026604..55f36ad 100644
--- a/src/parquet/column/levels.h
+++ b/src/parquet/column/levels.h
@@ -52,14 +52,14 @@ class LevelEncoder {
   }
 
   // Encodes a batch of levels from an array and returns the number of levels encoded
-  size_t Encode(size_t batch_size, const int16_t* levels) {
-    size_t num_encoded = 0;
+  int Encode(int batch_size, const int16_t* levels) {
+    int num_encoded = 0;
     if (!rle_encoder_ && !bit_packed_encoder_) {
       throw ParquetException("Level encoders are not initialized.");
     }
 
     if (encoding_ == Encoding::RLE) {
-      for (size_t i = 0; i < batch_size; ++i) {
+      for (int i = 0; i < batch_size; ++i) {
         if (!rle_encoder_->Put(*(levels + i))) {
           break;
         }
@@ -68,7 +68,7 @@ class LevelEncoder {
       rle_encoder_->Flush();
       rle_length_ = rle_encoder_->len();
     } else {
-      for (size_t i = 0; i < batch_size; ++i) {
+      for (int i = 0; i < batch_size; ++i) {
         if (!bit_packed_encoder_->PutValue(*(levels + i), bit_width_)) {
           break;
         }
@@ -101,7 +101,7 @@ class LevelDecoder {
 
   // Initialize the LevelDecoder state with new data
   // and return the number of bytes consumed
-  size_t SetData(Encoding::type encoding, int16_t max_level,
+  int SetData(Encoding::type encoding, int16_t max_level,
       int num_buffered_values, const uint8_t* data) {
     uint32_t num_bytes = 0;
     uint32_t total_bytes = 0;
@@ -135,19 +135,19 @@ class LevelDecoder {
   }
 
   // Decodes a batch of levels into an array and returns the number of levels decoded
-  size_t Decode(size_t batch_size, int16_t* levels) {
-    size_t num_decoded = 0;
+  int Decode(int batch_size, int16_t* levels) {
+    int num_decoded = 0;
 
-    size_t num_values = std::min(num_values_remaining_, batch_size);
+    int num_values = std::min(num_values_remaining_, batch_size);
     if (encoding_ == Encoding::RLE) {
-      for (size_t i = 0; i < num_values; ++i) {
+      for (int i = 0; i < num_values; ++i) {
         if (!rle_decoder_->Get(levels + i)) {
           break;
         }
         ++num_decoded;
       }
     } else {
-      for (size_t i = 0; i < num_values; ++i) {
+      for (int i = 0; i < num_values; ++i) {
         if (!bit_packed_decoder_->GetValue(bit_width_, levels + i)) {
           break;
         }
@@ -160,7 +160,7 @@ class LevelDecoder {
 
  private:
   int bit_width_;
-  size_t num_values_remaining_;
+  int num_values_remaining_;
   Encoding::type encoding_;
   std::unique_ptr<RleDecoder> rle_decoder_;
   std::unique_ptr<BitReader> bit_packed_decoder_;

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/column/reader.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/reader.cc b/src/parquet/column/reader.cc
index 4cff810..2885ebe 100644
--- a/src/parquet/column/reader.cc
+++ b/src/parquet/column/reader.cc
@@ -96,13 +96,13 @@ bool TypedColumnReader<TYPE>::ReadNewPage() {
       // If the data page includes repetition and definition levels, we
       // initialize the level decoder and subtract the encoded level bytes from
       // the page size to determine the number of bytes in the encoded data.
-      size_t data_size = page->size();
+      int64_t data_size = page->size();
 
       //Data page Layout: Repetition Levels - Definition Levels - encoded values.
       //Levels are encoded as rle or bit-packed.
       //Init repetition levels
       if (descr_->max_repetition_level() > 0) {
-        size_t rep_levels_bytes = repetition_level_decoder_.SetData(
+        int64_t rep_levels_bytes = repetition_level_decoder_.SetData(
             page->repetition_level_encoding(), descr_->max_repetition_level(),
             num_buffered_values_, buffer);
         buffer += rep_levels_bytes;
@@ -113,7 +113,7 @@ bool TypedColumnReader<TYPE>::ReadNewPage() {
 
       //Init definition levels
       if (descr_->max_definition_level() > 0) {
-        size_t def_levels_bytes = definition_level_decoder_.SetData(
+        int64_t def_levels_bytes = definition_level_decoder_.SetData(
             page->definition_level_encoding(), descr_->max_definition_level(),
             num_buffered_values_, buffer);
         buffer += def_levels_bytes;
@@ -165,14 +165,14 @@ bool TypedColumnReader<TYPE>::ReadNewPage() {
 // ----------------------------------------------------------------------
 // Batch read APIs
 
-size_t ColumnReader::ReadDefinitionLevels(size_t batch_size, int16_t* levels) {
+int64_t ColumnReader::ReadDefinitionLevels(int64_t batch_size, int16_t* levels) {
   if (descr_->max_definition_level() == 0) {
     return 0;
   }
   return definition_level_decoder_.Decode(batch_size, levels);
 }
 
-size_t ColumnReader::ReadRepetitionLevels(size_t batch_size, int16_t* levels) {
+int64_t ColumnReader::ReadRepetitionLevels(int64_t batch_size, int16_t* levels) {
   if (descr_->max_repetition_level() == 0) {
     return 0;
   }

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/column/reader.h
----------------------------------------------------------------------
diff --git a/src/parquet/column/reader.h b/src/parquet/column/reader.h
index dc23dd9..f6bf100 100644
--- a/src/parquet/column/reader.h
+++ b/src/parquet/column/reader.h
@@ -66,12 +66,12 @@ class ColumnReader {
   // Read multiple definition levels into preallocated memory
   //
   // Returns the number of decoded definition levels
-  size_t ReadDefinitionLevels(size_t batch_size, int16_t* levels);
+  int64_t ReadDefinitionLevels(int64_t batch_size, int16_t* levels);
 
   // Read multiple repetition levels into preallocated memory
   //
   // Returns the number of decoded repetition levels
-  size_t ReadRepetitionLevels(size_t batch_size, int16_t* levels);
+  int64_t ReadRepetitionLevels(int64_t batch_size, int16_t* levels);
 
   const ColumnDescriptor* descr_;
 
@@ -122,8 +122,8 @@ class TypedColumnReader : public ColumnReader {
   // This API is the same for both V1 and V2 of the DataPage
   //
   // @returns: actual number of levels read (see values_read for number of values read)
-  size_t ReadBatch(int32_t batch_size, int16_t* def_levels, int16_t* rep_levels,
-      T* values, size_t* values_read);
+  int64_t ReadBatch(int32_t batch_size, int16_t* def_levels, int16_t* rep_levels,
+      T* values, int64_t* values_read);
 
  private:
   typedef Decoder<TYPE> DecoderType;
@@ -135,7 +135,7 @@ class TypedColumnReader : public ColumnReader {
   // pre-allocated memory T*
   //
   // @returns: the number of values read into the out buffer
-  size_t ReadValues(size_t batch_size, T* out);
+  int64_t ReadValues(int64_t batch_size, T* out);
 
   // Map of encoding type to the respective decoder object. For example, a
   // column chunk's data pages may include both dictionary-encoded and
@@ -149,14 +149,14 @@ class TypedColumnReader : public ColumnReader {
 
 
 template <int TYPE>
-inline size_t TypedColumnReader<TYPE>::ReadValues(size_t batch_size, T* out) {
-  size_t num_decoded = current_decoder_->Decode(out, batch_size);
+inline int64_t TypedColumnReader<TYPE>::ReadValues(int64_t batch_size, T* out) {
+  int64_t num_decoded = current_decoder_->Decode(out, batch_size);
   return num_decoded;
 }
 
 template <int TYPE>
-inline size_t TypedColumnReader<TYPE>::ReadBatch(int batch_size, int16_t* def_levels,
-    int16_t* rep_levels, T* values, size_t* values_read) {
+inline int64_t TypedColumnReader<TYPE>::ReadBatch(int batch_size, int16_t* def_levels,
+    int16_t* rep_levels, T* values, int64_t* values_read) {
   // HasNext invokes ReadNewPage
   if (!HasNext()) {
     *values_read = 0;
@@ -167,17 +167,17 @@ inline size_t TypedColumnReader<TYPE>::ReadBatch(int batch_size, int16_t* def_le
   // row group is finished
   batch_size = std::min(batch_size, num_buffered_values_);
 
-  size_t num_def_levels = 0;
-  size_t num_rep_levels = 0;
+  int64_t num_def_levels = 0;
+  int64_t num_rep_levels = 0;
 
-  size_t values_to_read = 0;
+  int64_t values_to_read = 0;
 
   // If the field is required and non-repeated, there are no definition levels
   if (descr_->max_definition_level() > 0) {
     num_def_levels = ReadDefinitionLevels(batch_size, def_levels);
     // TODO(wesm): this tallying of values-to-decode can be performed with better
     // cache-efficiency if fused with the level decoding.
-    for (size_t i = 0; i < num_def_levels; ++i) {
+    for (int64_t i = 0; i < num_def_levels; ++i) {
       if (def_levels[i] == descr_->max_definition_level()) {
         ++values_to_read;
       }
@@ -196,7 +196,7 @@ inline size_t TypedColumnReader<TYPE>::ReadBatch(int batch_size, int16_t* def_le
   }
 
   *values_read = ReadValues(values_to_read, values);
-  size_t total_values = std::max(num_def_levels, *values_read);
+  int64_t total_values = std::max(num_def_levels, *values_read);
   num_decoded_values_ += total_values;
 
   return total_values;

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/column/scanner-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/scanner-test.cc b/src/parquet/column/scanner-test.cc
index fcaf65e..32c1ea5 100644
--- a/src/parquet/column/scanner-test.cc
+++ b/src/parquet/column/scanner-test.cc
@@ -106,9 +106,9 @@ class TestFlatScanner : public ::testing::Test {
     bool is_null = false;
     int16_t def_level;
     int16_t rep_level;
-    size_t j = 0;
+    int j = 0;
     scanner->SetBatchSize(batch_size);
-    for (size_t i = 0; i < num_levels_; i++) {
+    for (int i = 0; i < num_levels_; i++) {
       ASSERT_TRUE(scanner->Next(&val, &def_level, &rep_level, &is_null)) << i << j;
       if (!is_null) {
         ASSERT_EQ(values_[j++], val) << i <<"V"<< j;
@@ -193,7 +193,7 @@ template<>
 void TestFlatScanner<ByteArrayType>::InitValues() {
   int max_byte_array_len = 12;
   int num_bytes = max_byte_array_len + sizeof(uint32_t);
-  size_t nbytes = num_values_ * num_bytes;
+  int nbytes = num_values_ * num_bytes;
   data_buffer_.resize(nbytes);
   random_byte_array(num_values_, 0, data_buffer_.data(), values_.data(),
       max_byte_array_len);
@@ -201,7 +201,7 @@ void TestFlatScanner<ByteArrayType>::InitValues() {
 
 template<>
 void TestFlatScanner<FLBAType>::InitValues() {
-  size_t nbytes = num_values_ * FLBA_LENGTH;
+  int nbytes = num_values_ * FLBA_LENGTH;
   data_buffer_.resize(nbytes);
   random_fixed_byte_array(num_values_, 0, data_buffer_.data(), FLBA_LENGTH,
       values_.data());
@@ -259,10 +259,9 @@ TEST_F(TestFlatFLBAScanner, TestFLBAPrinterNext) {
   InitScanner(&d);
   TypedScanner<FLBAType::type_num>* scanner =
     reinterpret_cast<TypedScanner<FLBAType::type_num>* >(scanner_.get());
-  size_t j = 0;
   scanner->SetBatchSize(batch_size);
   std::stringstream ss_fail;
-  for (size_t i = 0; i < num_levels_; i++) {
+  for (int i = 0; i < num_levels_; i++) {
     std::stringstream ss;
     scanner->PrintNext(ss, 17);
     std::string result = ss.str();
@@ -271,7 +270,5 @@ TEST_F(TestFlatFLBAScanner, TestFLBAPrinterNext) {
   ASSERT_THROW(scanner->PrintNext(ss_fail, 17), ParquetException);
 }
 
-//Test for GroupNode
-
 } // namespace test
 } // namespace parquet_cpp

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/column/scanner.h
----------------------------------------------------------------------
diff --git a/src/parquet/column/scanner.h b/src/parquet/column/scanner.h
index 8569a94..f83cd81 100644
--- a/src/parquet/column/scanner.h
+++ b/src/parquet/column/scanner.h
@@ -75,12 +75,12 @@ class Scanner {
 
   std::vector<int16_t> def_levels_;
   std::vector<int16_t> rep_levels_;
-  size_t level_offset_;
-  size_t levels_buffered_;
+  int level_offset_;
+  int levels_buffered_;
 
   std::vector<uint8_t> value_buffer_;
-  size_t value_offset_;
-  size_t values_buffered_;
+  int value_offset_;
+  int64_t values_buffered_;
 
  private:
   std::shared_ptr<ColumnReader> reader_;
@@ -96,7 +96,7 @@ class TypedScanner : public Scanner {
       int64_t batch_size = DEFAULT_SCANNER_BATCH_SIZE) :
       Scanner(reader, batch_size) {
     typed_reader_ = static_cast<TypedColumnReader<TYPE>*>(reader.get());
-    size_t value_byte_size = type_traits<TYPE>::value_byte_size;
+    int value_byte_size = type_traits<TYPE>::value_byte_size;
     value_buffer_.resize(batch_size_ * value_byte_size);
     values_ = reinterpret_cast<T*>(&value_buffer_[0]);
   }
@@ -190,7 +190,7 @@ class TypedScanner : public Scanner {
   // The ownership of this object is expressed through the reader_ variable in the base
   TypedColumnReader<TYPE>* typed_reader_;
 
-  inline void FormatValue(void* val, char* buffer, size_t bufsize, size_t width);
+  inline void FormatValue(void* val, char* buffer, int bufsize, int width);
 
   T* values_;
 };
@@ -198,14 +198,14 @@ class TypedScanner : public Scanner {
 
 template <int TYPE>
 inline void TypedScanner<TYPE>::FormatValue(void* val, char* buffer,
-    size_t bufsize, size_t width) {
+    int bufsize, int width) {
   std::string fmt = format_fwf<TYPE>(width);
   snprintf(buffer, bufsize, fmt.c_str(), *reinterpret_cast<T*>(val));
 }
 
 template <>
 inline void TypedScanner<Type::INT96>::FormatValue(
-    void* val, char* buffer, size_t bufsize, size_t width) {
+    void* val, char* buffer, int bufsize, int width) {
   std::string fmt = format_fwf<Type::INT96>(width);
   std::string result = Int96ToString(*reinterpret_cast<Int96*>(val));
   snprintf(buffer, bufsize, fmt.c_str(), result.c_str());
@@ -213,7 +213,7 @@ inline void TypedScanner<Type::INT96>::FormatValue(
 
 template <>
 inline void TypedScanner<Type::BYTE_ARRAY>::FormatValue(
-    void* val, char* buffer, size_t bufsize, size_t width) {
+    void* val, char* buffer, int bufsize, int width) {
   std::string fmt = format_fwf<Type::BYTE_ARRAY>(width);
   std::string result = ByteArrayToString(*reinterpret_cast<ByteArray*>(val));
   snprintf(buffer, bufsize, fmt.c_str(), result.c_str());
@@ -221,7 +221,7 @@ inline void TypedScanner<Type::BYTE_ARRAY>::FormatValue(
 
 template <>
 inline void TypedScanner<Type::FIXED_LEN_BYTE_ARRAY>::FormatValue(
-    void* val, char* buffer, size_t bufsize, size_t width) {
+    void* val, char* buffer, int bufsize, int width) {
   std::string fmt = format_fwf<Type::FIXED_LEN_BYTE_ARRAY>(width);
   std::string result = FixedLenByteArrayToString(
       *reinterpret_cast<FixedLenByteArray*>(val),

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/column/test-util.h
----------------------------------------------------------------------
diff --git a/src/parquet/column/test-util.h b/src/parquet/column/test-util.h
index 1854ebb..a2cd77f 100644
--- a/src/parquet/column/test-util.h
+++ b/src/parquet/column/test-util.h
@@ -47,7 +47,7 @@ class MockPageReader : public PageReader {
 
   // Implement the PageReader interface
   virtual std::shared_ptr<Page> NextPage() {
-    if (page_index_ == pages_.size()) {
+    if (page_index_ == static_cast<int>(pages_.size())) {
       // EOS to consumer
       return std::shared_ptr<Page>(nullptr);
     }
@@ -56,7 +56,7 @@ class MockPageReader : public PageReader {
 
  private:
   std::vector<std::shared_ptr<Page> > pages_;
-  size_t page_index_;
+  int page_index_;
 };
 
 // TODO(wesm): this is only used for testing for now. Refactor to form part of
@@ -102,7 +102,7 @@ class DataPageBuilder {
     if (encoding != Encoding::PLAIN) {
       ParquetException::NYI("only plain encoding currently implemented");
     }
-    size_t bytes_to_encode = values.size() * sizeof(T);
+    int bytes_to_encode = values.size() * sizeof(T);
 
     PlainEncoder<TYPE> encoder(d);
     encoder.Encode(&values[0], values.size(), sink_);
@@ -171,7 +171,7 @@ void DataPageBuilder<Type::BOOLEAN>::AppendValues(const ColumnDescriptor *d,
   if (encoding != Encoding::PLAIN) {
     ParquetException::NYI("only plain encoding currently implemented");
   }
-  size_t bytes_to_encode = values.size() * sizeof(bool);
+  int bytes_to_encode = values.size() * sizeof(bool);
 
   PlainEncoder<Type::BOOLEAN> encoder(d);
   encoder.Encode(values, values.size(), sink_);
@@ -186,7 +186,7 @@ static std::shared_ptr<DataPage> MakeDataPage(const ColumnDescriptor *d,
     const std::vector<T>& values,
     const std::vector<int16_t>& def_levels, int16_t max_def_level,
     const std::vector<int16_t>& rep_levels, int16_t max_rep_level) {
-  size_t num_values = values.size();
+  int num_values = values.size();
 
   InMemoryOutputStream page_stream;
   test::DataPageBuilder<TYPE> page_builder(&page_stream);

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/encodings/delta-bit-pack-encoding.h
----------------------------------------------------------------------
diff --git a/src/parquet/encodings/delta-bit-pack-encoding.h b/src/parquet/encodings/delta-bit-pack-encoding.h
index d512db9..3e36af6 100644
--- a/src/parquet/encodings/delta-bit-pack-encoding.h
+++ b/src/parquet/encodings/delta-bit-pack-encoding.h
@@ -110,7 +110,7 @@ class DeltaBitPackDecoder : public Decoder<TYPE> {
   uint64_t values_current_mini_block_;
 
   int32_t min_delta_;
-  int mini_block_idx_;
+  size_t mini_block_idx_;
   std::vector<uint8_t> delta_bit_widths_;
   int delta_bit_width_;
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/encodings/dictionary-encoding.h
----------------------------------------------------------------------
diff --git a/src/parquet/encodings/dictionary-encoding.h b/src/parquet/encodings/dictionary-encoding.h
index eed0659..19ef1ea 100644
--- a/src/parquet/encodings/dictionary-encoding.h
+++ b/src/parquet/encodings/dictionary-encoding.h
@@ -263,24 +263,24 @@ class DictEncoder : public DictEncoderBase {
   int type_length_;
 
   /// Hash function for mapping a value to a bucket.
-  inline uint32_t Hash(const T& value) const;
+  inline int Hash(const T& value) const;
 
   /// Adds value to the hash table and updates dict_encoded_size_
   void AddDictKey(const T& value);
 };
 
 template<typename T>
-inline uint32_t DictEncoder<T>::Hash(const T& value) const {
+inline int DictEncoder<T>::Hash(const T& value) const {
   return HashUtil::Hash(&value, sizeof(value), 0);
 }
 
 template<>
-inline uint32_t DictEncoder<ByteArray>::Hash(const ByteArray& value) const {
+inline int DictEncoder<ByteArray>::Hash(const ByteArray& value) const {
   return HashUtil::Hash(value.ptr, value.len, 0);
 }
 
 template<>
-inline uint32_t DictEncoder<FixedLenByteArray>::Hash(
+inline int DictEncoder<FixedLenByteArray>::Hash(
     const FixedLenByteArray& value) const {
   return HashUtil::Hash(value.ptr, type_length_, 0);
 }
@@ -298,7 +298,7 @@ inline bool DictEncoder<FixedLenByteArray>::SlotDifferent(
 
 template <typename T>
 inline void DictEncoder<T>::Put(const T& v) {
-  uint32_t j = Hash(v) & mod_bitmask_;
+  int j = Hash(v) & mod_bitmask_;
   hash_slot_t index = hash_slots_[j];
 
   // Find an empty slot
@@ -316,8 +316,8 @@ inline void DictEncoder<T>::Put(const T& v) {
     hash_slots_[j] = index;
     AddDictKey(v);
 
-    if (UNLIKELY(uniques_.size() >
-            static_cast<size_t>(hash_table_size_ * MAX_HASH_LOAD))) {
+    if (UNLIKELY(static_cast<int>(uniques_.size()) >
+            hash_table_size_ * MAX_HASH_LOAD)) {
       DoubleTableSize();
     }
   }
@@ -330,7 +330,7 @@ inline void DictEncoder<T>::DoubleTableSize() {
   int new_size = hash_table_size_ * 2;
   std::vector<hash_slot_t> new_hash_slots(new_size, HASH_SLOT_EMPTY);
   hash_slot_t index, slot;
-  uint32_t j;
+  int j;
   for (int i = 0; i < hash_table_size_; ++i) {
     index = hash_slots_[i];
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/encodings/plain-encoding.h
----------------------------------------------------------------------
diff --git a/src/parquet/encodings/plain-encoding.h b/src/parquet/encodings/plain-encoding.h
index 9adabdf..95c353c 100644
--- a/src/parquet/encodings/plain-encoding.h
+++ b/src/parquet/encodings/plain-encoding.h
@@ -185,11 +185,11 @@ class PlainEncoder<Type::BOOLEAN> : public Encoder<Type::BOOLEAN> {
       Encoder<Type::BOOLEAN>(descr, Encoding::PLAIN) {}
 
   virtual void Encode(const bool* src, int num_values, OutputStream* dst) {
-    size_t bytes_required = BitUtil::Ceil(num_values, 8);
+    int bytes_required = BitUtil::Ceil(num_values, 8);
     std::vector<uint8_t> tmp_buffer(bytes_required);
 
     BitWriter bit_writer(&tmp_buffer[0], bytes_required);
-    for (size_t i = 0; i < num_values; ++i) {
+    for (int i = 0; i < num_values; ++i) {
       bit_writer.PutValue(src[i], 1);
     }
     bit_writer.Flush();
@@ -199,7 +199,7 @@ class PlainEncoder<Type::BOOLEAN> : public Encoder<Type::BOOLEAN> {
   }
 
   void Encode(const std::vector<bool>& src, int num_values, OutputStream* dst) {
-    size_t bytes_required = BitUtil::Ceil(num_values, 8);
+    int bytes_required = BitUtil::Ceil(num_values, 8);
 
     // TODO(wesm)
     // Use a temporary buffer for now and copy, because the BitWriter is not
@@ -208,7 +208,7 @@ class PlainEncoder<Type::BOOLEAN> : public Encoder<Type::BOOLEAN> {
     std::vector<uint8_t> tmp_buffer(bytes_required);
 
     BitWriter bit_writer(&tmp_buffer[0], bytes_required);
-    for (size_t i = 0; i < num_values; ++i) {
+    for (int i = 0; i < num_values; ++i) {
       bit_writer.PutValue(src[i], 1);
     }
     bit_writer.Flush();
@@ -227,7 +227,7 @@ inline void PlainEncoder<TYPE>::Encode(const T* buffer, int num_values,
 template <>
 inline void PlainEncoder<Type::BYTE_ARRAY>::Encode(const ByteArray* src,
     int num_values, OutputStream* dst) {
-  for (size_t i = 0; i < num_values; ++i) {
+  for (int i = 0; i < num_values; ++i) {
     // Write the result to the output stream
     dst->Write(reinterpret_cast<const uint8_t*>(&src[i].len), sizeof(uint32_t));
     dst->Write(reinterpret_cast<const uint8_t*>(src[i].ptr), src[i].len);
@@ -237,7 +237,7 @@ inline void PlainEncoder<Type::BYTE_ARRAY>::Encode(const ByteArray* src,
 template <>
 inline void PlainEncoder<Type::FIXED_LEN_BYTE_ARRAY>::Encode(
     const FixedLenByteArray* src, int num_values, OutputStream* dst) {
-  for (size_t i = 0; i < num_values; ++i) {
+  for (int i = 0; i < num_values; ++i) {
     // Write the result to the output stream
     dst->Write(reinterpret_cast<const uint8_t*>(src[i].ptr), descr_->type_length());
   }

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/file/file-deserialize-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/file/file-deserialize-test.cc b/src/parquet/file/file-deserialize-test.cc
index 3ce6084..1cbdaed 100644
--- a/src/parquet/file/file-deserialize-test.cc
+++ b/src/parquet/file/file-deserialize-test.cc
@@ -42,7 +42,7 @@ namespace parquet_cpp {
 
 // Adds page statistics occupying a certain amount of bytes (for testing very
 // large page headers)
-static inline void AddDummyStats(size_t stat_size,
+static inline void AddDummyStats(int stat_size,
     parquet::DataPageHeader& data_page) {
 
   std::vector<uint8_t> stat_bytes(stat_size);
@@ -199,7 +199,7 @@ TEST_F(TestPageSerde, Compression) {
     std::vector<uint8_t> buffer;
     for (int i = 0; i < num_pages; ++i) {
       const uint8_t* data = faux_data[i].data();
-      size_t data_size = faux_data[i].size();
+      int data_size = faux_data[i].size();
 
       int64_t max_compressed_size = codec->MaxCompressedLen(data_size, data);
       buffer.resize(max_compressed_size);
@@ -216,7 +216,7 @@ TEST_F(TestPageSerde, Compression) {
     std::shared_ptr<Page> page;
     const DataPage* data_page;
     for (int i = 0; i < num_pages; ++i) {
-      size_t data_size = faux_data[i].size();
+      int data_size = faux_data[i].size();
       page = page_reader_->NextPage();
       data_page = static_cast<const DataPage*>(page.get());
       ASSERT_EQ(data_size, data_page->size());

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/file/reader-internal.cc
----------------------------------------------------------------------
diff --git a/src/parquet/file/reader-internal.cc b/src/parquet/file/reader-internal.cc
index 24a8a8a..3d8c373 100644
--- a/src/parquet/file/reader-internal.cc
+++ b/src/parquet/file/reader-internal.cc
@@ -96,7 +96,7 @@ std::shared_ptr<Page> SerializedPageReader::NextPage() {
     // Uncompress it if we need to
     if (decompressor_ != NULL) {
       // Grow the uncompressed buffer if we need to.
-      if (uncompressed_len > decompression_buffer_.size()) {
+      if (uncompressed_len > static_cast<int>(decompression_buffer_.size())) {
         decompression_buffer_.resize(uncompressed_len);
       }
       decompressor_->Decompress(compressed_len, buffer, uncompressed_len,
@@ -239,7 +239,7 @@ SerializedFile::SerializedFile(std::unique_ptr<RandomAccessSource> source) :
 
 
 void SerializedFile::ParseMetaData() {
-  size_t filesize = source_->Size();
+  int64_t filesize = source_->Size();
 
   if (filesize < FOOTER_SIZE) {
     throw ParquetException("Corrupted file, smaller than file footer");
@@ -247,14 +247,14 @@ void SerializedFile::ParseMetaData() {
 
   uint8_t footer_buffer[FOOTER_SIZE];
   source_->Seek(filesize - FOOTER_SIZE);
-  size_t bytes_read = source_->Read(FOOTER_SIZE, footer_buffer);
+  int64_t bytes_read = source_->Read(FOOTER_SIZE, footer_buffer);
   if (bytes_read != FOOTER_SIZE ||
       memcmp(footer_buffer + 4, PARQUET_MAGIC, 4) != 0) {
     throw ParquetException("Invalid parquet file. Corrupt footer.");
   }
 
   uint32_t metadata_len = *reinterpret_cast<uint32_t*>(footer_buffer);
-  size_t metadata_start = filesize - FOOTER_SIZE - metadata_len;
+  int64_t metadata_start = filesize - FOOTER_SIZE - metadata_len;
   if (FOOTER_SIZE + metadata_len > filesize) {
     throw ParquetException("Invalid parquet file. File is less than "
         "file metadata size.");

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/file/reader.cc
----------------------------------------------------------------------
diff --git a/src/parquet/file/reader.cc b/src/parquet/file/reader.cc
index a2885a7..4901471 100644
--- a/src/parquet/file/reader.cc
+++ b/src/parquet/file/reader.cc
@@ -137,7 +137,7 @@ void ParquetFileReader::DebugPrint(std::ostream& stream, bool print_values) {
     auto group_reader = RowGroup(r);
 
     // Print column metadata
-    size_t num_columns = group_reader->num_columns();
+    int num_columns = group_reader->num_columns();
 
     for (int i = 0; i < num_columns; ++i) {
       RowGroupStatistics stats = group_reader->GetColumnStats(i);
@@ -153,7 +153,7 @@ void ParquetFileReader::DebugPrint(std::ostream& stream, bool print_values) {
       continue;
     }
 
-    static constexpr size_t bufsize = 25;
+    static constexpr int bufsize = 25;
     char buffer[bufsize];
 
     // Create readers for all columns and print contents

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/reader-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/reader-test.cc b/src/parquet/reader-test.cc
index e99140c..3ac1525 100644
--- a/src/parquet/reader-test.cc
+++ b/src/parquet/reader-test.cc
@@ -68,8 +68,8 @@ TEST_F(TestAllTypesPlain, TestBatchRead) {
   // This file only has 8 rows
 
   ASSERT_TRUE(col->HasNext());
-  size_t values_read;
-  size_t levels_read = col->ReadBatch(4, def_levels, rep_levels, values, &values_read);
+  int64_t values_read;
+  int levels_read = col->ReadBatch(4, def_levels, rep_levels, values, &values_read);
   ASSERT_EQ(4, levels_read);
   ASSERT_EQ(4, values_read);
 
@@ -89,7 +89,7 @@ TEST_F(TestAllTypesPlain, TestFlatScannerInt32) {
   std::shared_ptr<Int32Scanner> scanner(new Int32Scanner(group->Column(0)));
   int32_t val;
   bool is_null;
-  for (size_t i = 0; i < 8; ++i) {
+  for (int i = 0; i < 8; ++i) {
     ASSERT_TRUE(scanner->HasNext());
     ASSERT_TRUE(scanner->NextValue(&val, &is_null));
     ASSERT_FALSE(is_null);

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/schema/descriptor.cc
----------------------------------------------------------------------
diff --git a/src/parquet/schema/descriptor.cc b/src/parquet/schema/descriptor.cc
index b3fefee..1246f84 100644
--- a/src/parquet/schema/descriptor.cc
+++ b/src/parquet/schema/descriptor.cc
@@ -39,7 +39,7 @@ void SchemaDescriptor::Init(const NodePtr& schema) {
   group_ = static_cast<const GroupNode*>(schema_.get());
   leaves_.clear();
 
-  for (size_t i = 0; i < group_->field_count(); ++i) {
+  for (int i = 0; i < group_->field_count(); ++i) {
     BuildTree(group_->field(i), 0, 0);
   }
 }
@@ -58,7 +58,7 @@ void SchemaDescriptor::BuildTree(const NodePtr& node, int16_t max_def_level,
   // Now, walk the schema and create a ColumnDescriptor for each leaf node
   if (node->is_group()) {
     const GroupNode* group = static_cast<const GroupNode*>(node.get());
-    for (size_t i = 0; i < group->field_count(); ++i) {
+    for (int i = 0; i < group->field_count(); ++i) {
       BuildTree(group->field(i), max_def_level, max_rep_level);
     }
   } else {
@@ -80,7 +80,7 @@ ColumnDescriptor::ColumnDescriptor(const schema::NodePtr& node,
   primitive_node_ = static_cast<const PrimitiveNode*>(node_.get());
 }
 
-const ColumnDescriptor* SchemaDescriptor::Column(size_t i) const {
+const ColumnDescriptor* SchemaDescriptor::Column(int i) const {
   return &leaves_[i];
 }
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/schema/descriptor.h
----------------------------------------------------------------------
diff --git a/src/parquet/schema/descriptor.h b/src/parquet/schema/descriptor.h
index 4c6f50d..3fad182 100644
--- a/src/parquet/schema/descriptor.h
+++ b/src/parquet/schema/descriptor.h
@@ -100,10 +100,10 @@ class SchemaDescriptor {
   void Init(std::unique_ptr<schema::Node> schema);
   void Init(const schema::NodePtr& schema);
 
-  const ColumnDescriptor* Column(size_t i) const;
+  const ColumnDescriptor* Column(int i) const;
 
   // The number of physical columns appearing in the file
-  size_t num_columns() const {
+  int num_columns() const {
     return leaves_.size();
   }
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/schema/schema-converter-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/schema/schema-converter-test.cc b/src/parquet/schema/schema-converter-test.cc
index 64ca817..f749b40 100644
--- a/src/parquet/schema/schema-converter-test.cc
+++ b/src/parquet/schema/schema-converter-test.cc
@@ -49,7 +49,7 @@ class TestSchemaConverter : public ::testing::Test {
     name_ = "parquet_cpp_schema";
   }
 
-  void Convert(const parquet::SchemaElement* elements, size_t length) {
+  void Convert(const parquet::SchemaElement* elements, int length) {
     FlatSchemaConverter converter(elements, length);
     node_ = converter.Convert();
     ASSERT_TRUE(node_->is_group());

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/schema/schema-descriptor-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/schema/schema-descriptor-test.cc b/src/parquet/schema/schema-descriptor-test.cc
index 519d968..eda33a9 100644
--- a/src/parquet/schema/schema-descriptor-test.cc
+++ b/src/parquet/schema/schema-descriptor-test.cc
@@ -94,7 +94,7 @@ TEST_F(TestSchemaDescriptor, BuildTree) {
 
   descr_.Init(schema);
 
-  size_t nleaves = 6;
+  int nleaves = 6;
 
   // 6 leaves
   ASSERT_EQ(nleaves, descr_.num_columns());
@@ -111,7 +111,7 @@ TEST_F(TestSchemaDescriptor, BuildTree) {
   int16_t ex_max_def_levels[6] = {0, 1, 1, 2, 3, 3};
   int16_t ex_max_rep_levels[6] = {0, 0, 1, 1, 1, 2};
 
-  for (size_t i = 0; i < nleaves; ++i) {
+  for (int i = 0; i < nleaves; ++i) {
     const ColumnDescriptor* col = descr_.Column(i);
     EXPECT_EQ(ex_max_def_levels[i], col->max_definition_level()) << i;
     EXPECT_EQ(ex_max_rep_levels[i], col->max_repetition_level()) << i;

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/schema/test-util.h
----------------------------------------------------------------------
diff --git a/src/parquet/schema/test-util.h b/src/parquet/schema/test-util.h
index 5593abd..faf4a02 100644
--- a/src/parquet/schema/test-util.h
+++ b/src/parquet/schema/test-util.h
@@ -47,7 +47,7 @@ static inline SchemaElement NewPrimitive(const std::string& name,
 }
 
 static inline SchemaElement NewGroup(const std::string& name,
-    FieldRepetitionType::type repetition, size_t num_children) {
+    FieldRepetitionType::type repetition, int num_children) {
   SchemaElement result;
   result.__set_name(name);
   result.__set_repetition_type(repetition);

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/types.h
----------------------------------------------------------------------
diff --git a/src/parquet/types.h b/src/parquet/types.h
index e29f11c..dc741ce 100644
--- a/src/parquet/types.h
+++ b/src/parquet/types.h
@@ -186,7 +186,7 @@ static inline std::string FixedLenByteArrayToString(const FixedLenByteArray& a,
 }
 
 static inline int ByteCompare(const ByteArray& x1, const ByteArray& x2) {
-  int len = std::min(x1.len, x2.len);
+  uint32_t len = std::min(x1.len, x2.len);
   int cmp = memcmp(x1.ptr, x2.ptr, len);
   if (cmp != 0) return cmp;
   if (len < x1.len) return 1;
@@ -201,7 +201,7 @@ struct type_traits {
 template <>
 struct type_traits<Type::BOOLEAN> {
   typedef bool value_type;
-  static constexpr size_t value_byte_size = 1;
+  static constexpr int value_byte_size = 1;
 
   static constexpr const char* printf_code = "d";
 };
@@ -210,7 +210,7 @@ template <>
 struct type_traits<Type::INT32> {
   typedef int32_t value_type;
 
-  static constexpr size_t value_byte_size = 4;
+  static constexpr int value_byte_size = 4;
   static constexpr const char* printf_code = "d";
 };
 
@@ -218,7 +218,7 @@ template <>
 struct type_traits<Type::INT64> {
   typedef int64_t value_type;
 
-  static constexpr size_t value_byte_size = 8;
+  static constexpr int value_byte_size = 8;
   static constexpr const char* printf_code = "ld";
 };
 
@@ -226,7 +226,7 @@ template <>
 struct type_traits<Type::INT96> {
   typedef Int96 value_type;
 
-  static constexpr size_t value_byte_size = 12;
+  static constexpr int value_byte_size = 12;
   static constexpr const char* printf_code = "s";
 };
 
@@ -234,7 +234,7 @@ template <>
 struct type_traits<Type::FLOAT> {
   typedef float value_type;
 
-  static constexpr size_t value_byte_size = 4;
+  static constexpr int value_byte_size = 4;
   static constexpr const char* printf_code = "f";
 };
 
@@ -242,7 +242,7 @@ template <>
 struct type_traits<Type::DOUBLE> {
   typedef double value_type;
 
-  static constexpr size_t value_byte_size = 8;
+  static constexpr int value_byte_size = 8;
   static constexpr const char* printf_code = "lf";
 };
 
@@ -250,7 +250,7 @@ template <>
 struct type_traits<Type::BYTE_ARRAY> {
   typedef ByteArray value_type;
 
-  static constexpr size_t value_byte_size = sizeof(ByteArray);
+  static constexpr int value_byte_size = sizeof(ByteArray);
   static constexpr const char* printf_code = "s";
 };
 
@@ -258,7 +258,7 @@ template <>
 struct type_traits<Type::FIXED_LEN_BYTE_ARRAY> {
   typedef FixedLenByteArray value_type;
 
-  static constexpr size_t value_byte_size = sizeof(FixedLenByteArray);
+  static constexpr int value_byte_size = sizeof(FixedLenByteArray);
   static constexpr const char* printf_code = "s";
 };
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/util/bit-util.h
----------------------------------------------------------------------
diff --git a/src/parquet/util/bit-util.h b/src/parquet/util/bit-util.h
index 714911c..2b4014b 100644
--- a/src/parquet/util/bit-util.h
+++ b/src/parquet/util/bit-util.h
@@ -295,11 +295,11 @@ class BitUtil {
     return v | (static_cast<T>(0x1) << bitpos);
   }
 
-  static inline bool GetArrayBit(const uint8_t* bits, size_t i) {
+  static inline bool GetArrayBit(const uint8_t* bits, int i) {
     return bits[i / 8] & (1 << (i % 8));
   }
 
-  static inline void SetArrayBit(uint8_t* bits, size_t i, bool is_set) {
+  static inline void SetArrayBit(uint8_t* bits, int i, bool is_set) {
     bits[i / 8] |= (1 << (i % 8)) * is_set;
   }
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/util/mem-pool.cc
----------------------------------------------------------------------
diff --git a/src/parquet/util/mem-pool.cc b/src/parquet/util/mem-pool.cc
index 6e56c28..f8626bc 100644
--- a/src/parquet/util/mem-pool.cc
+++ b/src/parquet/util/mem-pool.cc
@@ -186,7 +186,7 @@ std::string MemPool::DebugString() {
   std::stringstream out;
   char str[16];
   out << "MemPool(#chunks=" << chunks_.size() << " [";
-  for (int i = 0; i < chunks_.size(); ++i) {
+  for (size_t i = 0; i < chunks_.size(); ++i) {
     sprintf(str, "0x%lx=", reinterpret_cast<size_t>(chunks_[i].data)); // NOLINT
     out << (i > 0 ? " " : "")
         << str
@@ -202,7 +202,7 @@ std::string MemPool::DebugString() {
 
 int64_t MemPool::GetTotalChunkSizes() const {
   int64_t result = 0;
-  for (int i = 0; i < chunks_.size(); ++i) {
+  for (size_t i = 0; i < chunks_.size(); ++i) {
     result += chunks_[i].size;
   }
   return result;
@@ -212,7 +212,7 @@ bool MemPool::CheckIntegrity(bool current_chunk_empty) {
   // check that current_chunk_idx_ points to the last chunk with allocated data
   DCHECK_LT(current_chunk_idx_, static_cast<int>(chunks_.size()));
   int64_t total_allocated = 0;
-  for (int i = 0; i < chunks_.size(); ++i) {
+  for (int i = 0; i < static_cast<int>(chunks_.size()); ++i) {
     DCHECK_GT(chunks_[i].size, 0);
     if (i < current_chunk_idx_) {
       DCHECK_GT(chunks_[i].allocated_bytes, 0);

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/util/rle-encoding.h
----------------------------------------------------------------------
diff --git a/src/parquet/util/rle-encoding.h b/src/parquet/util/rle-encoding.h
index 77749f5..b8dcc8e 100644
--- a/src/parquet/util/rle-encoding.h
+++ b/src/parquet/util/rle-encoding.h
@@ -234,7 +234,7 @@ class RleEncoder {
   /// many times in a row that value has been seen.  This is maintained even
   /// if we are in a literal run.  If the repeat_count_ get high enough, we switch
   /// to encoding repeated runs.
-  int64_t current_value_;
+  uint64_t current_value_;
   int repeat_count_;
 
   /// Number of literals in the current run.  This does not include the literals

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/util/rle-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/util/rle-test.cc b/src/parquet/util/rle-test.cc
index 5f18a6f..0a8309e 100644
--- a/src/parquet/util/rle-test.cc
+++ b/src/parquet/util/rle-test.cc
@@ -183,7 +183,7 @@ void ValidateRle(const vector<int>& values, int bit_width,
   EXPECT_LE(expected_len, len);
 
   RleEncoder encoder(buffer, len, bit_width);
-  for (int i = 0; i < values.size(); ++i) {
+  for (size_t i = 0; i < values.size(); ++i) {
     bool result = encoder.Put(values[i]);
     EXPECT_TRUE(result);
   }
@@ -198,7 +198,7 @@ void ValidateRle(const vector<int>& values, int bit_width,
 
   // Verify read
   RleDecoder decoder(buffer, len, bit_width);
-  for (int i = 0; i < values.size(); ++i) {
+  for (size_t i = 0; i < values.size(); ++i) {
     uint64_t val;
     bool result = decoder.Get(&val);
     EXPECT_TRUE(result);
@@ -212,7 +212,7 @@ bool CheckRoundTrip(const vector<int>& values, int bit_width) {
   const int len = 64 * 1024;
   uint8_t buffer[len];
   RleEncoder encoder(buffer, len, bit_width);
-  for (int i = 0; i < values.size(); ++i) {
+  for (size_t i = 0; i < values.size(); ++i) {
     bool result = encoder.Put(values[i]);
     if (!result) {
       return false;
@@ -222,7 +222,7 @@ bool CheckRoundTrip(const vector<int>& values, int bit_width) {
   int out;
 
   RleDecoder decoder(buffer, len, bit_width);
-  for (int i = 0; i < values.size(); ++i) {
+  for (size_t i = 0; i < values.size(); ++i) {
     uint64_t val;
     bool result = decoder.Get(&out);
     if (values[i] != out) {
@@ -345,9 +345,9 @@ TEST(BitRle, Flush) {
 
 // Test some random sequences.
 TEST(BitRle, Random) {
-  size_t niters = 50;
-  size_t ngroups = 1000;
-  size_t max_group_size = 16;
+  int niters = 50;
+  int ngroups = 1000;
+  int max_group_size = 16;
   vector<int> values(ngroups + max_group_size);
 
   // prng setup

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/5b3e9c10/src/parquet/util/test-common.h
----------------------------------------------------------------------
diff --git a/src/parquet/util/test-common.h b/src/parquet/util/test-common.h
index 9975ed9..637cd54 100644
--- a/src/parquet/util/test-common.h
+++ b/src/parquet/util/test-common.h
@@ -65,37 +65,37 @@ static inline bool vector_equal(const vector<T>& left, const vector<T>& right) {
 }
 
 template <typename T>
-static vector<T> slice(const vector<T>& values, size_t start, size_t end) {
+static vector<T> slice(const vector<T>& values, int start, int end) {
   if (end < start) {
     return vector<T>(0);
   }
 
   vector<T> out(end - start);
-  for (size_t i = start; i < end; ++i) {
+  for (int i = start; i < end; ++i) {
     out[i - start] = values[i];
   }
   return out;
 }
 
-static inline vector<bool> flip_coins_seed(size_t n, double p, uint32_t seed) {
+static inline vector<bool> flip_coins_seed(int n, double p, uint32_t seed) {
   std::mt19937 gen(seed);
   std::bernoulli_distribution d(p);
 
   vector<bool> draws;
-  for (size_t i = 0; i < n; ++i) {
+  for (int i = 0; i < n; ++i) {
     draws.push_back(d(gen));
   }
   return draws;
 }
 
-static inline vector<bool> flip_coins(size_t n, double p) {
+static inline vector<bool> flip_coins(int n, double p) {
   std::random_device rd;
   std::mt19937 gen(rd());
 
   std::bernoulli_distribution d(p);
 
   vector<bool> draws;
-  for (size_t i = 0; i < n; ++i) {
+  for (int i = 0; i < n; ++i) {
     draws.push_back(d(gen));
   }
   return draws;
@@ -176,12 +176,13 @@ void random_byte_array(int n, uint32_t seed, uint8_t *buf,
   std::uniform_int_distribution<int> d1(min_size, max_size);
   std::uniform_int_distribution<int> d2(0, 255);
   for (int i = 0; i < n; ++i) {
-    out[i].len = d1(gen);
+    int len = d1(gen);
+    out[i].len = len;
     out[i].ptr = buf;
-    for (int j = 0; j < out[i].len; ++j) {
+    for (int j = 0; j < len; ++j) {
       buf[j] = d2(gen) & 0xFF;
     }
-    buf += out[i].len;
+    buf += len;
   }
 }