You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@orc.apache.org by om...@apache.org on 2018/04/11 22:08:09 UTC

[1/2] orc git commit: ORC-332: Add syntax version to orc_proto.proto

Repository: orc
Updated Branches:
  refs/heads/branch-1.4 c455d98ba -> 7910fa968


ORC-332: Add syntax version to orc_proto.proto

Fixes #241

Signed-off-by: Owen O'Malley <om...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/orc/repo
Commit: http://git-wip-us.apache.org/repos/asf/orc/commit/7f326c36
Tree: http://git-wip-us.apache.org/repos/asf/orc/tree/7f326c36
Diff: http://git-wip-us.apache.org/repos/asf/orc/diff/7f326c36

Branch: refs/heads/branch-1.4
Commit: 7f326c361832e23e41744d60e70bce24e0079efd
Parents: c455d98
Author: rip-nsk <ri...@gmail.com>
Authored: Sun Apr 8 13:21:10 2018 -0700
Committer: Owen O'Malley <om...@apache.org>
Committed: Wed Apr 11 12:52:37 2018 -0700

----------------------------------------------------------------------
 proto/orc_proto.proto | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/orc/blob/7f326c36/proto/orc_proto.proto
----------------------------------------------------------------------
diff --git a/proto/orc_proto.proto b/proto/orc_proto.proto
index ee2ca45..bb3a378 100644
--- a/proto/orc_proto.proto
+++ b/proto/orc_proto.proto
@@ -16,6 +16,8 @@
  * limitations under the License.
  */
 
+syntax = "proto2";
+
 package orc.proto;
 
 option java_package = "org.apache.orc";


[2/2] orc git commit: ORC-281. Add missing override markers on destructors in InputStream.hh and OutputStream.hh

Posted by om...@apache.org.
ORC-281. Add missing override markers on destructors in InputStream.hh and OutputStream.hh

Fixes #203

Signed-off-by: Owen O'Malley <om...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/orc/repo
Commit: http://git-wip-us.apache.org/repos/asf/orc/commit/7910fa96
Tree: http://git-wip-us.apache.org/repos/asf/orc/tree/7910fa96
Diff: http://git-wip-us.apache.org/repos/asf/orc/diff/7910fa96

Branch: refs/heads/branch-1.4
Commit: 7910fa96891dd3b21e5c6a78fb6f75306e4a4e9d
Parents: 7f326c3
Author: Owen O'Malley <om...@apache.org>
Authored: Thu Dec 14 17:03:05 2017 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Wed Apr 11 15:07:53 2018 -0700

----------------------------------------------------------------------
 c++/src/ByteRLE.cc                      | 26 ++++----
 c++/src/ColumnPrinter.cc                | 28 ++++-----
 c++/src/ColumnReader.cc                 | 88 ++++++++++++++--------------
 c++/src/Compression.cc                  | 18 +++---
 c++/src/Compression.hh                  |  6 +-
 c++/src/MemoryPool.cc                   |  2 +-
 c++/src/OrcFile.cc                      |  2 +-
 c++/src/RLEv2.cc                        |  2 +-
 c++/src/Reader.cc                       |  2 +-
 c++/src/Statistics.cc                   |  4 +-
 c++/src/Statistics.hh                   | 24 ++++----
 c++/src/StripeStream.hh                 |  6 +-
 c++/src/Timezone.cc                     |  8 +--
 c++/src/TypeImpl.hh                     |  2 +-
 c++/src/wrap/coded-stream-wrapper.h     |  2 +
 c++/src/wrap/gmock.h                    |  2 +
 c++/src/wrap/gtest-wrapper.h            |  2 +
 c++/src/wrap/orc-proto-wrapper.cc       |  2 +
 c++/src/wrap/orc-proto-wrapper.hh       |  2 +
 c++/src/wrap/zero-copy-stream-wrapper.h |  2 +
 c++/test/TestColumnReader.cc            |  2 +-
 docker/ubuntu16-clang5/Dockerfile       | 48 +++++++++++++++
 tools/src/FileMemory.cc                 |  2 +-
 tools/test/gzip.cc                      |  8 +--
 24 files changed, 175 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/ByteRLE.cc
----------------------------------------------------------------------
diff --git a/c++/src/ByteRLE.cc b/c++/src/ByteRLE.cc
index c34af73..51aa931 100644
--- a/c++/src/ByteRLE.cc
+++ b/c++/src/ByteRLE.cc
@@ -36,22 +36,22 @@ namespace orc {
   public:
     ByteRleDecoderImpl(std::unique_ptr<SeekableInputStream> input);
 
-    virtual ~ByteRleDecoderImpl();
+    virtual ~ByteRleDecoderImpl() override;
 
     /**
      * Seek to a particular spot.
      */
-    virtual void seek(PositionProvider&);
+    virtual void seek(PositionProvider&) override;
 
     /**
      * Seek over a given number of values.
      */
-    virtual void skip(uint64_t numValues);
+    virtual void skip(uint64_t numValues) override;
 
     /**
      * Read a number of values into the batch.
      */
-    virtual void next(char* data, uint64_t numValues, char* notNull);
+    virtual void next(char* data, uint64_t numValues, char* notNull) override;
 
   protected:
     inline void nextBuffer();
@@ -102,8 +102,8 @@ namespace orc {
     repeating = false;
     remainingValues = 0;
     value = 0;
-    bufferStart = 0;
-    bufferEnd = 0;
+    bufferStart = nullptr;
+    bufferEnd = nullptr;
   }
 
   ByteRleDecoderImpl::~ByteRleDecoderImpl() {
@@ -218,22 +218,22 @@ namespace orc {
   public:
     BooleanRleDecoderImpl(std::unique_ptr<SeekableInputStream> input);
 
-    virtual ~BooleanRleDecoderImpl();
+    virtual ~BooleanRleDecoderImpl() override;
 
     /**
      * Seek to a particular spot.
      */
-    virtual void seek(PositionProvider&);
+    virtual void seek(PositionProvider&) override;
 
     /**
      * Seek over a given number of values.
      */
-    virtual void skip(uint64_t numValues);
+    virtual void skip(uint64_t numValues) override;
 
     /**
      * Read a number of values into the batch.
      */
-    virtual void next(char* data, uint64_t numValues, char* notNull);
+    virtual void next(char* data, uint64_t numValues, char* notNull) override;
 
   protected:
     size_t remainingBits;
@@ -259,7 +259,7 @@ namespace orc {
     }
     if (consumed != 0) {
       remainingBits = 8 - consumed;
-      ByteRleDecoderImpl::next(&lastByte, 1, 0);
+      ByteRleDecoderImpl::next(&lastByte, 1, nullptr);
     }
   }
 
@@ -270,7 +270,7 @@ namespace orc {
       numValues -= remainingBits;
       uint64_t bytesSkipped = numValues / 8;
       ByteRleDecoderImpl::skip(bytesSkipped);
-      ByteRleDecoderImpl::next(&lastByte, 1, 0);
+      ByteRleDecoderImpl::next(&lastByte, 1, nullptr);
       remainingBits = 8 - (numValues % 8);
     }
   }
@@ -318,7 +318,7 @@ namespace orc {
     } else if (position < numValues) {
       // read the new bytes into the array
       uint64_t bytesRead = (nonNulls + 7) / 8;
-      ByteRleDecoderImpl::next(data + position, bytesRead, 0);
+      ByteRleDecoderImpl::next(data + position, bytesRead, nullptr);
       lastByte = data[position + bytesRead - 1];
       remainingBits = bytesRead * 8 - nonNulls;
       // expand the array backwards so that we don't clobber the data

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/ColumnPrinter.cc
----------------------------------------------------------------------
diff --git a/c++/src/ColumnPrinter.cc b/c++/src/ColumnPrinter.cc
index 2462143..379ef0b 100644
--- a/c++/src/ColumnPrinter.cc
+++ b/c++/src/ColumnPrinter.cc
@@ -36,7 +36,7 @@ namespace orc {
   class VoidColumnPrinter: public ColumnPrinter {
   public:
     VoidColumnPrinter(std::string&);
-    ~VoidColumnPrinter() {}
+    ~VoidColumnPrinter() override {}
     void printRow(uint64_t rowId) override;
     void reset(const ColumnVectorBatch& batch) override;
   };
@@ -46,7 +46,7 @@ namespace orc {
     const int64_t* data;
   public:
     BooleanColumnPrinter(std::string&);
-    ~BooleanColumnPrinter() {}
+    ~BooleanColumnPrinter() override {}
     void printRow(uint64_t rowId) override;
     void reset(const ColumnVectorBatch& batch) override;
   };
@@ -56,7 +56,7 @@ namespace orc {
     const int64_t* data;
   public:
     LongColumnPrinter(std::string&);
-    ~LongColumnPrinter() {}
+    ~LongColumnPrinter() override {}
     void printRow(uint64_t rowId) override;
     void reset(const ColumnVectorBatch& batch) override;
   };
@@ -68,7 +68,7 @@ namespace orc {
 
   public:
     DoubleColumnPrinter(std::string&, const Type& type);
-    virtual ~DoubleColumnPrinter() {}
+    virtual ~DoubleColumnPrinter() override {}
     void printRow(uint64_t rowId) override;
     void reset(const ColumnVectorBatch& batch) override;
   };
@@ -80,7 +80,7 @@ namespace orc {
 
   public:
     TimestampColumnPrinter(std::string&);
-    ~TimestampColumnPrinter() {}
+    ~TimestampColumnPrinter() override {}
     void printRow(uint64_t rowId) override;
     void reset(const ColumnVectorBatch& batch) override;
   };
@@ -91,7 +91,7 @@ namespace orc {
 
   public:
     DateColumnPrinter(std::string&);
-    ~DateColumnPrinter() {}
+    ~DateColumnPrinter() override {}
     void printRow(uint64_t rowId) override;
     void reset(const ColumnVectorBatch& batch) override;
   };
@@ -102,7 +102,7 @@ namespace orc {
     int32_t scale;
   public:
     Decimal64ColumnPrinter(std::string&);
-    ~Decimal64ColumnPrinter() {}
+    ~Decimal64ColumnPrinter() override {}
     void printRow(uint64_t rowId) override;
     void reset(const ColumnVectorBatch& batch) override;
   };
@@ -113,7 +113,7 @@ namespace orc {
     int32_t scale;
   public:
     Decimal128ColumnPrinter(std::string&);
-    ~Decimal128ColumnPrinter() {}
+    ~Decimal128ColumnPrinter() override {}
     void printRow(uint64_t rowId) override;
     void reset(const ColumnVectorBatch& batch) override;
   };
@@ -124,7 +124,7 @@ namespace orc {
     const int64_t* length;
   public:
     StringColumnPrinter(std::string&);
-    virtual ~StringColumnPrinter() {}
+    virtual ~StringColumnPrinter() override {}
     void printRow(uint64_t rowId) override;
     void reset(const ColumnVectorBatch& batch) override;
   };
@@ -135,7 +135,7 @@ namespace orc {
     const int64_t* length;
   public:
     BinaryColumnPrinter(std::string&);
-    virtual ~BinaryColumnPrinter() {}
+    virtual ~BinaryColumnPrinter() override {}
     void printRow(uint64_t rowId) override;
     void reset(const ColumnVectorBatch& batch) override;
   };
@@ -147,7 +147,7 @@ namespace orc {
 
   public:
     ListColumnPrinter(std::string&, const Type& type);
-    virtual ~ListColumnPrinter() {}
+    virtual ~ListColumnPrinter() override {}
     void printRow(uint64_t rowId) override;
     void reset(const ColumnVectorBatch& batch) override;
   };
@@ -160,7 +160,7 @@ namespace orc {
 
   public:
     MapColumnPrinter(std::string&, const Type& type);
-    virtual ~MapColumnPrinter() {}
+    virtual ~MapColumnPrinter() override {}
     void printRow(uint64_t rowId) override;
     void reset(const ColumnVectorBatch& batch) override;
   };
@@ -173,7 +173,7 @@ namespace orc {
 
   public:
     UnionColumnPrinter(std::string&, const Type& type);
-    virtual ~UnionColumnPrinter();
+    virtual ~UnionColumnPrinter() override;
     void printRow(uint64_t rowId) override;
     void reset(const ColumnVectorBatch& batch) override;
   };
@@ -184,7 +184,7 @@ namespace orc {
     std::vector<std::string> fieldNames;
   public:
     StructColumnPrinter(std::string&, const Type& type);
-    virtual ~StructColumnPrinter();
+    virtual ~StructColumnPrinter() override;
     void printRow(uint64_t rowId) override;
     void reset(const ColumnVectorBatch& batch) override;
   };

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/ColumnReader.cc
----------------------------------------------------------------------
diff --git a/c++/src/ColumnReader.cc b/c++/src/ColumnReader.cc
index 467f8bb..5bc1d5e 100644
--- a/c++/src/ColumnReader.cc
+++ b/c++/src/ColumnReader.cc
@@ -75,7 +75,7 @@ namespace orc {
         uint64_t chunkSize =
           std::min(remaining,
                    static_cast<uint64_t>(bufferSize));
-        decoder->next(buffer, chunkSize, 0);
+        decoder->next(buffer, chunkSize, nullptr);
         remaining -= chunkSize;
         for(uint64_t i=0; i < chunkSize; ++i) {
           if (!buffer[i]) {
@@ -134,7 +134,7 @@ namespace orc {
 
   public:
     BooleanColumnReader(const Type& type, StripeStreams& stipe);
-    ~BooleanColumnReader();
+    ~BooleanColumnReader() override;
 
     uint64_t skip(uint64_t numValues) override;
 
@@ -169,7 +169,7 @@ namespace orc {
     // we cheat here and use the long* and then expand it in a second pass.
     int64_t *ptr = dynamic_cast<LongVectorBatch&>(rowBatch).data.data();
     rle->next(reinterpret_cast<char*>(ptr),
-              numValues, rowBatch.hasNulls ? rowBatch.notNull.data() : 0);
+              numValues, rowBatch.hasNulls ? rowBatch.notNull.data() : nullptr);
     expandBytesToLongs(ptr, numValues);
   }
 
@@ -179,7 +179,7 @@ namespace orc {
 
   public:
     ByteColumnReader(const Type& type, StripeStreams& stipe);
-    ~ByteColumnReader();
+    ~ByteColumnReader() override;
 
     uint64_t skip(uint64_t numValues) override;
 
@@ -214,7 +214,7 @@ namespace orc {
     // we cheat here and use the long* and then expand it in a second pass.
     int64_t *ptr = dynamic_cast<LongVectorBatch&>(rowBatch).data.data();
     rle->next(reinterpret_cast<char*>(ptr),
-              numValues, rowBatch.hasNulls ? rowBatch.notNull.data() : 0);
+              numValues, rowBatch.hasNulls ? rowBatch.notNull.data() : nullptr);
     expandBytesToLongs(ptr, numValues);
   }
 
@@ -224,7 +224,7 @@ namespace orc {
 
   public:
     IntegerColumnReader(const Type& type, StripeStreams& stripe);
-    ~IntegerColumnReader();
+    ~IntegerColumnReader() override;
 
     uint64_t skip(uint64_t numValues) override;
 
@@ -258,7 +258,7 @@ namespace orc {
                                  char *notNull) {
     ColumnReader::next(rowBatch, numValues, notNull);
     rle->next(dynamic_cast<LongVectorBatch&>(rowBatch).data.data(),
-              numValues, rowBatch.hasNulls ? rowBatch.notNull.data() : 0);
+              numValues, rowBatch.hasNulls ? rowBatch.notNull.data() : nullptr);
   }
 
   class TimestampColumnReader: public ColumnReader {
@@ -270,7 +270,7 @@ namespace orc {
 
   public:
     TimestampColumnReader(const Type& type, StripeStreams& stripe);
-    ~TimestampColumnReader();
+    ~TimestampColumnReader() override;
 
     uint64_t skip(uint64_t numValues) override;
 
@@ -342,7 +342,7 @@ namespace orc {
   class DoubleColumnReader: public ColumnReader {
   public:
     DoubleColumnReader(const Type& type, StripeStreams& stripe);
-    ~DoubleColumnReader();
+    ~DoubleColumnReader() override;
 
     uint64_t skip(uint64_t numValues) override;
 
@@ -399,8 +399,8 @@ namespace orc {
                                             columnKind(type.getKind()),
                                             bytesPerValue((type.getKind() ==
                                                            FLOAT) ? 4 : 8),
-                                            bufferPointer(NULL),
-                                            bufferEnd(NULL) {
+                                            bufferPointer(nullptr),
+                                            bufferEnd(nullptr) {
     // PASS
   }
 
@@ -418,8 +418,8 @@ namespace orc {
       inputStream->Skip(static_cast<int>(bytesPerValue*numValues -
                                          static_cast<size_t>(bufferEnd -
                                                              bufferPointer)));
-      bufferEnd = NULL;
-      bufferPointer = NULL;
+      bufferEnd = nullptr;
+      bufferPointer = nullptr;
     }
 
     return numValues;
@@ -430,7 +430,7 @@ namespace orc {
                                 char *notNull) {
     ColumnReader::next(rowBatch, numValues, notNull);
     // update the notNull from the parent class
-    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : 0;
+    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : nullptr;
     double* outArray = dynamic_cast<DoubleVectorBatch&>(rowBatch).data.data();
 
     if (columnKind == FLOAT) {
@@ -482,7 +482,7 @@ namespace orc {
 
   public:
     StringDictionaryColumnReader(const Type& type, StripeStreams& stipe);
-    ~StringDictionaryColumnReader();
+    ~StringDictionaryColumnReader() override;
 
     uint64_t skip(uint64_t numValues) override;
 
@@ -511,7 +511,7 @@ namespace orc {
                        false, rleVersion, memoryPool);
     dictionaryOffset.resize(dictionaryCount+1);
     int64_t* lengthArray = dictionaryOffset.data();
-    lengthDecoder->next(lengthArray + 1, dictionaryCount, 0);
+    lengthDecoder->next(lengthArray + 1, dictionaryCount, nullptr);
     lengthArray[0] = 0;
     for(uint64_t i=1; i < dictionaryCount + 1; ++i) {
       lengthArray[i] += lengthArray[i-1];
@@ -538,7 +538,7 @@ namespace orc {
                                           char *notNull) {
     ColumnReader::next(rowBatch, numValues, notNull);
     // update the notNull from the parent class
-    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : 0;
+    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : nullptr;
     StringVectorBatch& byteBatch = dynamic_cast<StringVectorBatch&>(rowBatch);
     char *blob = dictionaryBlob.data();
     int64_t *dictionaryOffsets = dictionaryOffset.data();
@@ -584,7 +584,7 @@ namespace orc {
 
   public:
     StringDirectColumnReader(const Type& type, StripeStreams& stipe);
-    ~StringDirectColumnReader();
+    ~StringDirectColumnReader() override;
 
     uint64_t skip(uint64_t numValues) override;
 
@@ -605,7 +605,7 @@ namespace orc {
                                                   true),
                                  false, rleVersion, memoryPool);
     blobStream = stripe.getStream(columnId, proto::Stream_Kind_DATA, true);
-    lastBuffer = 0;
+    lastBuffer = nullptr;
     lastBufferLength = 0;
   }
 
@@ -623,8 +623,8 @@ namespace orc {
     while (done < numValues) {
       uint64_t step = std::min(BUFFER_SIZE,
                                     static_cast<size_t>(numValues - done));
-      lengthRle->next(buffer, step, 0);
-      totalBytes += computeSize(buffer, 0, step);
+      lengthRle->next(buffer, step, nullptr);
+      totalBytes += computeSize(buffer, nullptr, step);
       done += step;
     }
     if (totalBytes <= lastBufferLength) {
@@ -636,7 +636,7 @@ namespace orc {
       totalBytes -= lastBufferLength;
       blobStream->Skip(static_cast<int>(totalBytes));
       lastBufferLength = 0;
-      lastBuffer = 0;
+      lastBuffer = nullptr;
     }
     return numValues;
   }
@@ -664,7 +664,7 @@ namespace orc {
                                       char *notNull) {
     ColumnReader::next(rowBatch, numValues, notNull);
     // update the notNull from the parent class
-    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : 0;
+    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : nullptr;
     StringVectorBatch& byteBatch = dynamic_cast<StringVectorBatch&>(rowBatch);
     char **startPtr = byteBatch.data.data();
     int64_t *lengthPtr = byteBatch.length.data();
@@ -756,7 +756,7 @@ namespace orc {
 
   public:
     StructColumnReader(const Type& type, StripeStreams& stipe);
-    ~StructColumnReader();
+    ~StructColumnReader() override;
 
     uint64_t skip(uint64_t numValues) override;
 
@@ -806,7 +806,7 @@ namespace orc {
                                 char *notNull) {
     ColumnReader::next(rowBatch, numValues, notNull);
     uint64_t i=0;
-    notNull = rowBatch.hasNulls? rowBatch.notNull.data() : 0;
+    notNull = rowBatch.hasNulls? rowBatch.notNull.data() : nullptr;
     for(std::vector<ColumnReader*>::iterator ptr=children.begin();
         ptr != children.end(); ++ptr, ++i) {
       (*ptr)->next(*(dynamic_cast<StructVectorBatch&>(rowBatch).fields[i]),
@@ -821,7 +821,7 @@ namespace orc {
 
   public:
     ListColumnReader(const Type& type, StripeStreams& stipe);
-    ~ListColumnReader();
+    ~ListColumnReader() override;
 
     uint64_t skip(uint64_t numValues) override;
 
@@ -860,7 +860,7 @@ namespace orc {
       uint64_t lengthsRead = 0;
       while (lengthsRead < numValues) {
         uint64_t chunk = std::min(numValues - lengthsRead, BUFFER_SIZE);
-        rle->next(buffer, chunk, 0);
+        rle->next(buffer, chunk, nullptr);
         for(size_t i=0; i < chunk; ++i) {
           childrenElements += static_cast<size_t>(buffer[i]);
         }
@@ -879,7 +879,7 @@ namespace orc {
     ColumnReader::next(rowBatch, numValues, notNull);
     ListVectorBatch &listBatch = dynamic_cast<ListVectorBatch&>(rowBatch);
     int64_t* offsets = listBatch.offsets.data();
-    notNull = listBatch.hasNulls ? listBatch.notNull.data() : 0;
+    notNull = listBatch.hasNulls ? listBatch.notNull.data() : nullptr;
     rle->next(offsets, numValues, notNull);
     uint64_t totalChildren = 0;
     if (notNull) {
@@ -902,7 +902,7 @@ namespace orc {
     offsets[numValues] = static_cast<int64_t>(totalChildren);
     ColumnReader *childReader = child.get();
     if (childReader) {
-      childReader->next(*(listBatch.elements.get()), totalChildren, 0);
+      childReader->next(*(listBatch.elements.get()), totalChildren, nullptr);
     }
   }
 
@@ -914,7 +914,7 @@ namespace orc {
 
   public:
     MapColumnReader(const Type& type, StripeStreams& stipe);
-    ~MapColumnReader();
+    ~MapColumnReader() override;
 
     uint64_t skip(uint64_t numValues) override;
 
@@ -958,7 +958,7 @@ namespace orc {
       uint64_t lengthsRead = 0;
       while (lengthsRead < numValues) {
         uint64_t chunk = std::min(numValues - lengthsRead, BUFFER_SIZE);
-        rle->next(buffer, chunk, 0);
+        rle->next(buffer, chunk, nullptr);
         for(size_t i=0; i < chunk; ++i) {
           childrenElements += static_cast<size_t>(buffer[i]);
         }
@@ -982,7 +982,7 @@ namespace orc {
     ColumnReader::next(rowBatch, numValues, notNull);
     MapVectorBatch &mapBatch = dynamic_cast<MapVectorBatch&>(rowBatch);
     int64_t* offsets = mapBatch.offsets.data();
-    notNull = mapBatch.hasNulls ? mapBatch.notNull.data() : 0;
+    notNull = mapBatch.hasNulls ? mapBatch.notNull.data() : nullptr;
     rle->next(offsets, numValues, notNull);
     uint64_t totalChildren = 0;
     if (notNull) {
@@ -1005,11 +1005,11 @@ namespace orc {
     offsets[numValues] = static_cast<int64_t>(totalChildren);
     ColumnReader *rawKeyReader = keyReader.get();
     if (rawKeyReader) {
-      rawKeyReader->next(*(mapBatch.keys.get()), totalChildren, 0);
+      rawKeyReader->next(*(mapBatch.keys.get()), totalChildren, nullptr);
     }
     ColumnReader *rawElementReader = elementReader.get();
     if (rawElementReader) {
-      rawElementReader->next(*(mapBatch.elements.get()), totalChildren, 0);
+      rawElementReader->next(*(mapBatch.elements.get()), totalChildren, nullptr);
     }
   }
 
@@ -1022,7 +1022,7 @@ namespace orc {
 
   public:
     UnionColumnReader(const Type& type, StripeStreams& stipe);
-    ~UnionColumnReader();
+    ~UnionColumnReader() override;
 
     uint64_t skip(uint64_t numValues) override;
 
@@ -1067,14 +1067,14 @@ namespace orc {
     memset(counts, 0, sizeof(int64_t) * numChildren);
     while (lengthsRead < numValues) {
       uint64_t chunk = std::min(numValues - lengthsRead, BUFFER_SIZE);
-      rle->next(buffer, chunk, 0);
+      rle->next(buffer, chunk, nullptr);
       for(size_t i=0; i < chunk; ++i) {
         counts[static_cast<size_t>(buffer[i])] += 1;
       }
       lengthsRead += chunk;
     }
     for(size_t i=0; i < numChildren; ++i) {
-      if (counts[i] != 0 && childrenReader[i] != NULL) {
+      if (counts[i] != 0 && childrenReader[i] != nullptr) {
         childrenReader[i]->skip(static_cast<uint64_t>(counts[i]));
       }
     }
@@ -1090,7 +1090,7 @@ namespace orc {
     int64_t* counts = childrenCounts.data();
     memset(counts, 0, sizeof(int64_t) * numChildren);
     unsigned char* tags = unionBatch.tags.data();
-    notNull = unionBatch.hasNulls ? unionBatch.notNull.data() : 0;
+    notNull = unionBatch.hasNulls ? unionBatch.notNull.data() : nullptr;
     rle->next(reinterpret_cast<char *>(tags), numValues, notNull);
     // set the offsets for each row
     if (notNull) {
@@ -1180,7 +1180,7 @@ namespace orc {
 
   public:
     Decimal64ColumnReader(const Type& type, StripeStreams& stipe);
-    ~Decimal64ColumnReader();
+    ~Decimal64ColumnReader() override;
 
     uint64_t skip(uint64_t numValues) override;
 
@@ -1248,7 +1248,7 @@ namespace orc {
                                    uint64_t numValues,
                                    char *notNull) {
     ColumnReader::next(rowBatch, numValues, notNull);
-    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : 0;
+    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : nullptr;
     Decimal64VectorBatch &batch =
       dynamic_cast<Decimal64VectorBatch&>(rowBatch);
     int64_t* values = batch.values.data();
@@ -1295,7 +1295,7 @@ namespace orc {
   class Decimal128ColumnReader: public Decimal64ColumnReader {
   public:
     Decimal128ColumnReader(const Type& type, StripeStreams& stipe);
-    ~Decimal128ColumnReader();
+    ~Decimal128ColumnReader() override;
 
     void next(ColumnVectorBatch& rowBatch,
               uint64_t numValues,
@@ -1338,7 +1338,7 @@ namespace orc {
                                    uint64_t numValues,
                                    char *notNull) {
     ColumnReader::next(rowBatch, numValues, notNull);
-    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : 0;
+    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : nullptr;
     Decimal128VectorBatch &batch =
       dynamic_cast<Decimal128VectorBatch&>(rowBatch);
     Int128* values = batch.values.data();
@@ -1405,7 +1405,7 @@ namespace orc {
 
   public:
     DecimalHive11ColumnReader(const Type& type, StripeStreams& stipe);
-    ~DecimalHive11ColumnReader();
+    ~DecimalHive11ColumnReader() override;
 
     void next(ColumnVectorBatch& rowBatch,
               uint64_t numValues,
@@ -1429,7 +1429,7 @@ namespace orc {
                                        uint64_t numValues,
                                        char *notNull) {
     ColumnReader::next(rowBatch, numValues, notNull);
-    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : 0;
+    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : nullptr;
     Decimal128VectorBatch &batch =
       dynamic_cast<Decimal128VectorBatch&>(rowBatch);
     Int128* values = batch.values.data();

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/Compression.cc
----------------------------------------------------------------------
diff --git a/c++/src/Compression.cc b/c++/src/Compression.cc
index 81cc578..5394273 100644
--- a/c++/src/Compression.cc
+++ b/c++/src/Compression.cc
@@ -239,7 +239,7 @@ namespace orc {
     ZlibDecompressionStream(std::unique_ptr<SeekableInputStream> inStream,
                             size_t blockSize,
                             MemoryPool& pool);
-    virtual ~ZlibDecompressionStream();
+    virtual ~ZlibDecompressionStream() override;
     virtual bool Next(const void** data, int*size) override;
     virtual void BackUp(int count) override;
     virtual bool Skip(int count) override;
@@ -327,11 +327,11 @@ DIAGNOSTIC_IGNORE("-Wold-style-cast")
                        blockSize(_blockSize),
                        buffer(pool, _blockSize) {
     input.reset(inStream.release());
-    zstream.next_in = Z_NULL;
+    zstream.next_in = nullptr;
     zstream.avail_in = 0;
-    zstream.zalloc = Z_NULL;
-    zstream.zfree = Z_NULL;
-    zstream.opaque = Z_NULL;
+    zstream.zalloc = nullptr;
+    zstream.zfree = nullptr;
+    zstream.opaque = nullptr;
     zstream.next_out = reinterpret_cast<Bytef*>(buffer.data());
     zstream.avail_out = static_cast<uInt>(blockSize);
     int64_t result = inflateInit2(&zstream, -15);
@@ -504,7 +504,7 @@ DIAGNOSTIC_POP
                              size_t blockSize,
                              MemoryPool& pool);
 
-    virtual ~BlockDecompressionStream() {}
+    virtual ~BlockDecompressionStream() override {}
     virtual bool Next(const void** data, int*size) override;
     virtual void BackUp(int count) override;
     virtual bool Skip(int count) override;
@@ -599,11 +599,11 @@ DIAGNOSTIC_POP
                         inputBuffer(pool, bufferSize),
                         outputBuffer(pool, bufferSize),
                         state(DECOMPRESS_HEADER),
-                        outputBufferPtr(0),
+                        outputBufferPtr(nullptr),
                         outputBufferLength(0),
                         remainingLength(0),
-                        inputBufferPtr(0),
-                        inputBufferPtrEnd(0),
+                        inputBufferPtr(nullptr),
+                        inputBufferPtrEnd(nullptr),
                         bytesReturned(0) {
     input.reset(inStream.release());
   }

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/Compression.hh
----------------------------------------------------------------------
diff --git a/c++/src/Compression.hh b/c++/src/Compression.hh
index efd374a..a988495 100644
--- a/c++/src/Compression.hh
+++ b/c++/src/Compression.hh
@@ -52,7 +52,7 @@ namespace orc {
    */
   class SeekableInputStream: public google::protobuf::io::ZeroCopyInputStream {
   public:
-    virtual ~SeekableInputStream();
+    virtual ~SeekableInputStream() override;
     virtual void seek(PositionProvider& position) = 0;
     virtual std::string getName() const = 0;
   };
@@ -74,7 +74,7 @@ namespace orc {
     SeekableArrayInputStream(const char* list,
                              uint64_t length,
                              uint64_t block_size = 0);
-    virtual ~SeekableArrayInputStream();
+    virtual ~SeekableArrayInputStream() override;
     virtual bool Next(const void** data, int*size) override;
     virtual void BackUp(int count) override;
     virtual bool Skip(int count) override;
@@ -103,7 +103,7 @@ namespace orc {
                             uint64_t byteCount,
                             MemoryPool& pool,
                             uint64_t blockSize = 0);
-    virtual ~SeekableFileInputStream();
+    virtual ~SeekableFileInputStream() override;
 
     virtual bool Next(const void** data, int*size) override;
     virtual void BackUp(int count) override;

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/MemoryPool.cc
----------------------------------------------------------------------
diff --git a/c++/src/MemoryPool.cc b/c++/src/MemoryPool.cc
index 7fecf67..14b7ebf 100644
--- a/c++/src/MemoryPool.cc
+++ b/c++/src/MemoryPool.cc
@@ -33,7 +33,7 @@ namespace orc {
 
   class MemoryPoolImpl: public MemoryPool {
   public:
-    virtual ~MemoryPoolImpl();
+    virtual ~MemoryPoolImpl() override;
 
     char* malloc(uint64_t size) override;
     void free(char* p) override;

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/OrcFile.cc
----------------------------------------------------------------------
diff --git a/c++/src/OrcFile.cc b/c++/src/OrcFile.cc
index d5d00fd..138e1a3 100644
--- a/c++/src/OrcFile.cc
+++ b/c++/src/OrcFile.cc
@@ -51,7 +51,7 @@ namespace orc {
       totalLength = static_cast<uint64_t>(fileStat.st_size);
     }
 
-    ~FileInputStream();
+    ~FileInputStream() override;
 
     uint64_t getLength() const override {
       return totalLength;

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/RLEv2.cc
----------------------------------------------------------------------
diff --git a/c++/src/RLEv2.cc b/c++/src/RLEv2.cc
index 02f325f..10c6190 100644
--- a/c++/src/RLEv2.cc
+++ b/c++/src/RLEv2.cc
@@ -140,7 +140,7 @@ void RleDecoderV2::seek(PositionProvider& location) {
   // move the input stream
   inputStream->seek(location);
   // clear state
-  bufferEnd = bufferStart = 0;
+  bufferEnd = bufferStart = nullptr;
   runRead = runLength = 0;
   // skip ahead the given number of records
   skip(location.next());

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/Reader.cc
----------------------------------------------------------------------
diff --git a/c++/src/Reader.cc b/c++/src/Reader.cc
index 9c423bd..b6f6abf 100644
--- a/c++/src/Reader.cc
+++ b/c++/src/Reader.cc
@@ -804,7 +804,7 @@ namespace orc {
       std::min(static_cast<uint64_t>(data.capacity),
                rowsInCurrentStripe - currentRowInStripe);
     data.numElements = rowsToRead;
-    reader->next(data, rowsToRead, 0);
+    reader->next(data, rowsToRead, nullptr);
     // update row number
     previousRow = firstRowOfStripe[currentStripe] + currentRowInStripe;
     currentRowInStripe += rowsToRead;

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/Statistics.cc
----------------------------------------------------------------------
diff --git a/c++/src/Statistics.cc b/c++/src/Statistics.cc
index 54da928..bc822c0 100644
--- a/c++/src/Statistics.cc
+++ b/c++/src/Statistics.cc
@@ -290,8 +290,8 @@ namespace orc {
       upperBound = 0;
     }else{
       const proto::TimestampStatistics& stats = pb.timestampstatistics();
-      _hasMinimum = stats.has_minimumutc() || (stats.has_minimum() && (statContext.writerTimezone != NULL));
-      _hasMaximum = stats.has_maximumutc() || (stats.has_maximum() && (statContext.writerTimezone != NULL));
+      _hasMinimum = stats.has_minimumutc() || (stats.has_minimum() && (statContext.writerTimezone != nullptr));
+      _hasMaximum = stats.has_maximumutc() || (stats.has_maximum() && (statContext.writerTimezone != nullptr));
       _hasLowerBound = stats.has_minimumutc() || stats.has_minimum();
       _hasUpperBound = stats.has_maximumutc() || stats.has_maximum();
 

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/Statistics.hh
----------------------------------------------------------------------
diff --git a/c++/src/Statistics.hh b/c++/src/Statistics.hh
index de9c9f3..fb0ba56 100644
--- a/c++/src/Statistics.hh
+++ b/c++/src/Statistics.hh
@@ -35,8 +35,8 @@ namespace orc {
   struct StatContext {
     const bool correctStats;
     const Timezone* const writerTimezone;
-    StatContext() : correctStats(false), writerTimezone(NULL) {}
-    StatContext(bool cStat, const Timezone* const timezone = NULL) :
+    StatContext() : correctStats(false), writerTimezone(nullptr) {}
+    StatContext(bool cStat, const Timezone* const timezone = nullptr) :
         correctStats(cStat), writerTimezone(timezone) {}
   };
 
@@ -50,7 +50,7 @@ namespace orc {
 
   public:
     ColumnStatisticsImpl(const proto::ColumnStatistics& stats);
-    virtual ~ColumnStatisticsImpl();
+    virtual ~ColumnStatisticsImpl() override;
 
     uint64_t getNumberOfValues() const override {
       return valueCount;
@@ -72,7 +72,7 @@ namespace orc {
   public:
     BinaryColumnStatisticsImpl(const proto::ColumnStatistics& stats,
                                const StatContext& statContext);
-    virtual ~BinaryColumnStatisticsImpl();
+    virtual ~BinaryColumnStatisticsImpl() override;
 
     bool hasTotalLength() const override {
       return _hasTotalLength;
@@ -110,7 +110,7 @@ namespace orc {
 
   public:
     BooleanColumnStatisticsImpl(const proto::ColumnStatistics& stats, const StatContext& statContext);
-    virtual ~BooleanColumnStatisticsImpl();
+    virtual ~BooleanColumnStatisticsImpl() override;
 
     bool hasCount() const override {
       return _hasCount;
@@ -161,7 +161,7 @@ namespace orc {
 
   public:
     DateColumnStatisticsImpl(const proto::ColumnStatistics& stats, const StatContext& statContext);
-    virtual ~DateColumnStatisticsImpl();
+    virtual ~DateColumnStatisticsImpl() override;
 
     bool hasMinimum() const override {
       return _hasMinimum;
@@ -222,7 +222,7 @@ namespace orc {
 
   public:
     DecimalColumnStatisticsImpl(const proto::ColumnStatistics& stats, const StatContext& statContext);
-    virtual ~DecimalColumnStatisticsImpl();
+    virtual ~DecimalColumnStatisticsImpl() override;
 
     bool hasMinimum() const override {
       return _hasMinimum;
@@ -302,7 +302,7 @@ namespace orc {
 
   public:
     DoubleColumnStatisticsImpl(const proto::ColumnStatistics& stats);
-    virtual ~DoubleColumnStatisticsImpl();
+    virtual ~DoubleColumnStatisticsImpl() override;
 
     bool hasMinimum() const override {
       return _hasMinimum;
@@ -381,7 +381,7 @@ namespace orc {
 
   public:
     IntegerColumnStatisticsImpl(const proto::ColumnStatistics& stats);
-    virtual ~IntegerColumnStatisticsImpl();
+    virtual ~IntegerColumnStatisticsImpl() override;
 
     bool hasMinimum() const override {
       return _hasMinimum;
@@ -460,7 +460,7 @@ namespace orc {
 
   public:
     StringColumnStatisticsImpl(const proto::ColumnStatistics& stats, const StatContext& statContext);
-    virtual ~StringColumnStatisticsImpl();
+    virtual ~StringColumnStatisticsImpl() override;
 
     bool hasMinimum() const override {
       return _hasMinimum;
@@ -542,7 +542,7 @@ namespace orc {
   public:
     TimestampColumnStatisticsImpl(const proto::ColumnStatistics& stats,
                                   const StatContext& statContext);
-    virtual ~TimestampColumnStatisticsImpl();
+    virtual ~TimestampColumnStatisticsImpl() override;
 
     bool hasMinimum() const override {
       return _hasMinimum;
@@ -668,7 +668,7 @@ namespace orc {
       return *it;
     }
 
-    virtual ~StatisticsImpl();
+    virtual ~StatisticsImpl() override;
 
     uint32_t getNumberOfColumns() const override {
       return static_cast<uint32_t>(colStats.size());

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/StripeStream.hh
----------------------------------------------------------------------
diff --git a/c++/src/StripeStream.hh b/c++/src/StripeStream.hh
index 8f31397..7770564 100644
--- a/c++/src/StripeStream.hh
+++ b/c++/src/StripeStream.hh
@@ -49,7 +49,7 @@ namespace orc {
                       InputStream& input,
                       const Timezone& writerTimezone);
 
-    virtual ~StripeStreamsImpl();
+    virtual ~StripeStreamsImpl() override;
 
     virtual const std::vector<bool> getSelectedColumns() const override;
 
@@ -92,7 +92,7 @@ namespace orc {
       // PASS
     }
 
-    ~StreamInformationImpl();
+    ~StreamInformationImpl() override;
 
     StreamKind getKind() const override {
       return kind;
@@ -150,7 +150,7 @@ namespace orc {
       // PASS
     }
 
-    virtual ~StripeInformationImpl() {
+    virtual ~StripeInformationImpl() override {
       // PASS
     }
 

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/Timezone.cc
----------------------------------------------------------------------
diff --git a/c++/src/Timezone.cc b/c++/src/Timezone.cc
index bc2ffb6..6083293 100644
--- a/c++/src/Timezone.cc
+++ b/c++/src/Timezone.cc
@@ -238,7 +238,7 @@ namespace orc {
     }
 
   public:
-    virtual ~FutureRuleImpl();
+    virtual ~FutureRuleImpl() override;
     bool isDefined() const override;
     const TimezoneVariant& getVariant(int64_t clk) const override;
     void print(std::ostream& out) const override;
@@ -518,7 +518,7 @@ namespace orc {
 
   class Version1Parser: public VersionParser {
   public:
-    virtual ~Version1Parser();
+    virtual ~Version1Parser() override;
 
     virtual uint64_t getVersion() const override {
       return 1;
@@ -552,7 +552,7 @@ namespace orc {
 
   class Version2Parser: public VersionParser {
   public:
-    virtual ~Version2Parser();
+    virtual ~Version2Parser() override;
 
     virtual uint64_t getVersion() const override {
       return 2;
@@ -588,7 +588,7 @@ namespace orc {
   public:
     TimezoneImpl(const std::string& name,
                  const std::vector<unsigned char> bytes);
-    virtual ~TimezoneImpl();
+    virtual ~TimezoneImpl() override;
 
     /**
      * Get the variant for the given time (time_t).

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/TypeImpl.hh
----------------------------------------------------------------------
diff --git a/c++/src/TypeImpl.hh b/c++/src/TypeImpl.hh
index e2866e4..fd8126a 100644
--- a/c++/src/TypeImpl.hh
+++ b/c++/src/TypeImpl.hh
@@ -58,7 +58,7 @@ namespace orc {
     TypeImpl(TypeKind kind, uint64_t precision,
              uint64_t scale);
 
-    virtual ~TypeImpl();
+    virtual ~TypeImpl() override;
 
     uint64_t getColumnId() const override;
 

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/wrap/coded-stream-wrapper.h
----------------------------------------------------------------------
diff --git a/c++/src/wrap/coded-stream-wrapper.h b/c++/src/wrap/coded-stream-wrapper.h
index 07af410..3f0a806 100644
--- a/c++/src/wrap/coded-stream-wrapper.h
+++ b/c++/src/wrap/coded-stream-wrapper.h
@@ -22,6 +22,8 @@ DIAGNOSTIC_PUSH
 #ifdef __clang__
   DIAGNOSTIC_IGNORE("-Wshorten-64-to-32")
   DIAGNOSTIC_IGNORE("-Wreserved-id-macro")
+  DIAGNOSTIC_IGNORE("-Wunknown-warning-option")
+  DIAGNOSTIC_IGNORE("-Wzero-as-null-pointer-constant")
 #endif
 
 DIAGNOSTIC_IGNORE("-Wconversion")

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/wrap/gmock.h
----------------------------------------------------------------------
diff --git a/c++/src/wrap/gmock.h b/c++/src/wrap/gmock.h
index cca2a8a..bbf7956 100644
--- a/c++/src/wrap/gmock.h
+++ b/c++/src/wrap/gmock.h
@@ -31,8 +31,10 @@ DIAGNOSTIC_IGNORE("-Wundef")
 #ifdef __clang__
   DIAGNOSTIC_IGNORE("-Wnull-dereference")
   DIAGNOSTIC_IGNORE("-Wshift-sign-overflow")
+  DIAGNOSTIC_IGNORE("-Wunknown-warning-option")
   DIAGNOSTIC_IGNORE("-Wused-but-marked-unused")
   DIAGNOSTIC_IGNORE("-Wweak-vtables")
+  DIAGNOSTIC_IGNORE("-Wzero-as-null-pointer-constant")
 #endif
 
 #include "gmock/gmock.h"

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/wrap/gtest-wrapper.h
----------------------------------------------------------------------
diff --git a/c++/src/wrap/gtest-wrapper.h b/c++/src/wrap/gtest-wrapper.h
index fb95a1d..a14f6c3 100644
--- a/c++/src/wrap/gtest-wrapper.h
+++ b/c++/src/wrap/gtest-wrapper.h
@@ -26,7 +26,9 @@ DIAGNOSTIC_IGNORE("-Wsign-compare")
   DIAGNOSTIC_IGNORE("-Wconversion-null")
   DIAGNOSTIC_IGNORE("-Wexit-time-destructors")
   DIAGNOSTIC_IGNORE("-Wglobal-constructors")
+  DIAGNOSTIC_IGNORE("-Wunknown-warning-option")
   DIAGNOSTIC_IGNORE("-Wused-but-marked-unused")
+  DIAGNOSTIC_IGNORE("-Wzero-as-null-pointer-constant")
 #endif
 
 DIAGNOSTIC_PUSH

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/wrap/orc-proto-wrapper.cc
----------------------------------------------------------------------
diff --git a/c++/src/wrap/orc-proto-wrapper.cc b/c++/src/wrap/orc-proto-wrapper.cc
index 7234b7f..8b10cae 100644
--- a/c++/src/wrap/orc-proto-wrapper.cc
+++ b/c++/src/wrap/orc-proto-wrapper.cc
@@ -29,7 +29,9 @@ DIAGNOSTIC_IGNORE("-Wunused-parameter")
   DIAGNOSTIC_IGNORE("-Wnested-anon-types")
   DIAGNOSTIC_IGNORE("-Wreserved-id-macro")
   DIAGNOSTIC_IGNORE("-Wshorten-64-to-32")
+  DIAGNOSTIC_IGNORE("-Wunknown-warning-option")
   DIAGNOSTIC_IGNORE("-Wweak-vtables")
+  DIAGNOSTIC_IGNORE("-Wzero-as-null-pointer-constant")
 #endif
 
 #include "orc_proto.pb.cc"

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/wrap/orc-proto-wrapper.hh
----------------------------------------------------------------------
diff --git a/c++/src/wrap/orc-proto-wrapper.hh b/c++/src/wrap/orc-proto-wrapper.hh
index 2f2304d..5fc9558 100644
--- a/c++/src/wrap/orc-proto-wrapper.hh
+++ b/c++/src/wrap/orc-proto-wrapper.hh
@@ -27,7 +27,9 @@ DIAGNOSTIC_IGNORE("-Wunused-parameter")
   DIAGNOSTIC_IGNORE("-Wnested-anon-types")
   DIAGNOSTIC_IGNORE("-Wreserved-id-macro")
   DIAGNOSTIC_IGNORE("-Wshorten-64-to-32")
+  DIAGNOSTIC_IGNORE("-Wunknown-warning-option")
   DIAGNOSTIC_IGNORE("-Wweak-vtables")
+  DIAGNOSTIC_IGNORE("-Wzero-as-null-pointer-constant")
 #endif
 
 #include "orc_proto.pb.h"

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/src/wrap/zero-copy-stream-wrapper.h
----------------------------------------------------------------------
diff --git a/c++/src/wrap/zero-copy-stream-wrapper.h b/c++/src/wrap/zero-copy-stream-wrapper.h
index 6f03113..1a8c779 100644
--- a/c++/src/wrap/zero-copy-stream-wrapper.h
+++ b/c++/src/wrap/zero-copy-stream-wrapper.h
@@ -25,6 +25,8 @@ DIAGNOSTIC_IGNORE("-Wunused-parameter")
 
 #ifdef __clang__
   DIAGNOSTIC_IGNORE("-Wreserved-id-macro")
+  DIAGNOSTIC_IGNORE("-Wunknown-warning-option")
+  DIAGNOSTIC_IGNORE("-Wzero-as-null-pointer-constant")
 #endif
 
 #include <google/protobuf/io/zero_copy_stream.h>

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/c++/test/TestColumnReader.cc
----------------------------------------------------------------------
diff --git a/c++/test/TestColumnReader.cc b/c++/test/TestColumnReader.cc
index d09ecca..cfc34e8 100644
--- a/c++/test/TestColumnReader.cc
+++ b/c++/test/TestColumnReader.cc
@@ -37,7 +37,7 @@ namespace orc {
 
 class MockStripeStreams: public StripeStreams {
 public:
-  ~MockStripeStreams();
+  ~MockStripeStreams() override;
   std::unique_ptr<SeekableInputStream> getStream(uint64_t columnId,
                                                  proto::Stream_Kind kind,
                                                  bool stream) const override;

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/docker/ubuntu16-clang5/Dockerfile
----------------------------------------------------------------------
diff --git a/docker/ubuntu16-clang5/Dockerfile b/docker/ubuntu16-clang5/Dockerfile
new file mode 100644
index 0000000..f21c91f
--- /dev/null
+++ b/docker/ubuntu16-clang5/Dockerfile
@@ -0,0 +1,48 @@
+# 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.
+
+# ORC compile for Ubuntu 16 (clang 5)
+#
+
+FROM ubuntu:16.04
+MAINTAINER Owen O'Malley <ow...@hortonworks.com>
+
+RUN apt-get update
+RUN apt-get install -y wget software-properties-common
+RUN wget -O - https://apt.llvm.org/llvm-snapshot.gpg.key | apt-key add -
+RUN apt-add-repository "deb http://apt.llvm.org/xenial/ llvm-toolchain-xenial-5.0 main"
+RUN apt-get update
+RUN apt-get install -y \
+  cmake \
+  default-jdk \
+  clang-5.0 \
+  git \
+  libsasl2-dev \
+  libssl-dev \
+  make \
+  maven \
+  tzdata
+
+ENV CC=clang-5.0
+ENV CXX=clang++-5.0
+
+WORKDIR /root
+
+CMD git clone https://github.com/apache/orc.git -b master && \
+  mkdir orc/build && \
+  cd orc/build && \
+  cmake .. && \
+  make package test-out

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/tools/src/FileMemory.cc
----------------------------------------------------------------------
diff --git a/tools/src/FileMemory.cc b/tools/src/FileMemory.cc
index c133501..8d2928b 100644
--- a/tools/src/FileMemory.cc
+++ b/tools/src/FileMemory.cc
@@ -54,7 +54,7 @@ public:
   }
 
   TestMemoryPool(): totalMemory(0), maxMemory(0) {}
-  ~TestMemoryPool();
+  ~TestMemoryPool() override;
 };
 
 TestMemoryPool::~TestMemoryPool() {}

http://git-wip-us.apache.org/repos/asf/orc/blob/7910fa96/tools/test/gzip.cc
----------------------------------------------------------------------
diff --git a/tools/test/gzip.cc b/tools/test/gzip.cc
index 462eca3..c5ada8b 100644
--- a/tools/test/gzip.cc
+++ b/tools/test/gzip.cc
@@ -34,12 +34,12 @@ namespace orc {
     if (file == nullptr) {
       throw std::runtime_error("can't open " + filename);
     }
-    stream.zalloc = Z_NULL;
-    stream.zfree = Z_NULL;
-    stream.opaque = Z_NULL;
+    stream.zalloc = nullptr;
+    stream.zfree = nullptr;
+    stream.opaque = nullptr;
     stream.avail_in = 0;
     stream.avail_out = 1;
-    stream.next_in = Z_NULL;
+    stream.next_in = nullptr;
     int ret = inflateInit2(&stream, 16+MAX_WBITS);
     if (ret != Z_OK) {
       throw std::runtime_error("zlib failed initialization for " + filename);