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 2016/02/24 00:20:48 UTC

orc git commit: ORC-10. Correct bug when moving ORC files between timezones.

Repository: orc
Updated Branches:
  refs/heads/orc-10 [created] 6dce746c3


ORC-10. Correct bug when moving ORC files between timezones.


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

Branch: refs/heads/orc-10
Commit: 6dce746c30098a7b708dfa0aa5010260fd505ca6
Parents: ed772e7
Author: Owen O'Malley <om...@apache.org>
Authored: Thu Feb 11 15:21:03 2016 -0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Mon Feb 22 22:06:37 2016 -0800

----------------------------------------------------------------------
 README.md                     |   1 -
 c++/src/Adaptor.hh.in         |   8 +
 c++/src/CMakeLists.txt        |   4 +
 c++/src/ColumnPrinter.cc      |   2 +-
 c++/src/ColumnReader.cc       |  10 +-
 c++/src/ColumnReader.hh       |   7 +-
 c++/src/Reader.cc             |  38 +-
 c++/src/Timezone.cc           | 964 +++++++++++++++++++++++++++++++++++++
 c++/src/Timezone.hh           | 121 +++++
 c++/test/CMakeLists.txt       |   1 +
 c++/test/TestColumnPrinter.cc |  24 +-
 c++/test/TestColumnReader.cc  |  33 +-
 c++/test/TestTimezone.cc      | 361 ++++++++++++++
 docker/run-all.sh             |   7 +-
 tools/src/CMakeLists.txt      |   2 +-
 15 files changed, 1526 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/orc/blob/6dce746c/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 723e835..7711d4e 100644
--- a/README.md
+++ b/README.md
@@ -25,7 +25,6 @@ _Optimized Row Columnar_ (ORC) file format.
 
 ```shell
 -To compile:
-% export TZ=America/Los_Angeles
 % mkdir build
 % cd build
 % cmake ..

http://git-wip-us.apache.org/repos/asf/orc/blob/6dce746c/c++/src/Adaptor.hh.in
----------------------------------------------------------------------
diff --git a/c++/src/Adaptor.hh.in b/c++/src/Adaptor.hh.in
index 5b3a677..568054b 100644
--- a/c++/src/Adaptor.hh.in
+++ b/c++/src/Adaptor.hh.in
@@ -87,6 +87,14 @@
   #define UINT32_MAX 0xffffffff
 #endif
 
+#ifndef INT64_MAX
+  #define INT64_MAX 0x7fffffffffffffff
+#endif
+
+#ifndef INT64_MIN
+  #define INT64_MIN (-0x7fffffffffffffff - 1)
+#endif
+
 #define GTEST_LANG_CXX11 0
 
 #endif /* ADAPTER_HH */

http://git-wip-us.apache.org/repos/asf/orc/blob/6dce746c/c++/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/c++/src/CMakeLists.txt b/c++/src/CMakeLists.txt
index b7b5260..545e3ff 100644
--- a/c++/src/CMakeLists.txt
+++ b/c++/src/CMakeLists.txt
@@ -70,8 +70,11 @@ CHECK_CXX_SOURCE_RUNS("
 )
 
 CHECK_CXX_SOURCE_RUNS("
+    #include<stdlib.h>
     #include<time.h>
     int main(int, char *[]) {
+      setenv(\"TZ\", \"America/Los_Angeles\", 1);
+      tzset();
       struct tm time2037;
       struct tm time2038;
       strptime(\"2037-05-05 12:34:56\", \"%Y-%m-%d %H:%M:%S\", &time2037);
@@ -119,6 +122,7 @@ add_library (orc STATIC
   RLEv1.cc
   RLEv2.cc
   RLE.cc
+  Timezone.cc
   TypeImpl.cc
   Vector.cc
   )

http://git-wip-us.apache.org/repos/asf/orc/blob/6dce746c/c++/src/ColumnPrinter.cc
----------------------------------------------------------------------
diff --git a/c++/src/ColumnPrinter.cc b/c++/src/ColumnPrinter.cc
index 764da01..2462143 100644
--- a/c++/src/ColumnPrinter.cc
+++ b/c++/src/ColumnPrinter.cc
@@ -712,7 +712,7 @@ namespace orc {
       int64_t nanos = nanoseconds[rowId];
       time_t secs = static_cast<time_t>(seconds[rowId]);
       struct tm tmValue;
-      localtime_r(&secs, &tmValue);
+      gmtime_r(&secs, &tmValue);
       char timeBuffer[20];
       strftime(timeBuffer, sizeof(timeBuffer), "%Y-%m-%d %H:%M:%S", &tmValue);
       writeChar(buffer, '"');

http://git-wip-us.apache.org/repos/asf/orc/blob/6dce746c/c++/src/ColumnReader.cc
----------------------------------------------------------------------
diff --git a/c++/src/ColumnReader.cc b/c++/src/ColumnReader.cc
index ae4d9b6..8db16be 100644
--- a/c++/src/ColumnReader.cc
+++ b/c++/src/ColumnReader.cc
@@ -265,6 +265,7 @@ namespace orc {
   private:
     std::unique_ptr<orc::RleDecoder> secondsRle;
     std::unique_ptr<orc::RleDecoder> nanoRle;
+    const Timezone& writerTimezone;
     const int64_t epochOffset;
 
   public:
@@ -281,8 +282,9 @@ namespace orc {
 
   TimestampColumnReader::TimestampColumnReader(const Type& type,
                                                StripeStreams& stripe
-                                     ): ColumnReader(type, stripe),
-                                         epochOffset(stripe.getEpochOffset()) {
+                               ): ColumnReader(type, stripe),
+                                  writerTimezone(stripe.getWriterTimezone()),
+                                  epochOffset(writerTimezone.getEpoch()) {
     RleVersion vers = convertRleVersion(stripe.getEncoding(columnId).kind());
     secondsRle = createRleDecoder(stripe.getStream(columnId,
                                                    proto::Stream_Kind_DATA,
@@ -327,7 +329,9 @@ namespace orc {
             nanoBuffer[i] *= 10;
           }
         }
-        secsBuffer[i] += epochOffset;
+        int64_t writerTime = secsBuffer[i] + epochOffset;
+        secsBuffer[i] = writerTime +
+          writerTimezone.getVariant(writerTime).gmtOffset;
         if (secsBuffer[i] < 0 && nanoBuffer[i] != 0) {
           secsBuffer[i] -= 1;
         }

http://git-wip-us.apache.org/repos/asf/orc/blob/6dce746c/c++/src/ColumnReader.hh
----------------------------------------------------------------------
diff --git a/c++/src/ColumnReader.hh b/c++/src/ColumnReader.hh
index 142d41e..5cfd781 100644
--- a/c++/src/ColumnReader.hh
+++ b/c++/src/ColumnReader.hh
@@ -22,6 +22,7 @@
 #include "orc/Vector.hh"
 #include "ByteRLE.hh"
 #include "Compression.hh"
+#include "Timezone.hh"
 #include "wrap/orc-proto-wrapper.hh"
 
 namespace orc {
@@ -65,11 +66,9 @@ namespace orc {
     virtual MemoryPool& getMemoryPool() const = 0;
 
     /**
-     * Get the number of seconds between the ORC epoch and Unix epoch.
-     * ORC epoch is 1 Jan 2015 00:00:00 local.
-     * Unix epoch is 1 Jan 1970 00:00:00 UTC.
+     * Get the writer's timezone, so that we can convert their dates correctly.
      */
-    virtual int64_t getEpochOffset() const = 0;
+    virtual const Timezone& getWriterTimezone() const = 0;
   };
 
   /**

http://git-wip-us.apache.org/repos/asf/orc/blob/6dce746c/c++/src/Reader.cc
----------------------------------------------------------------------
diff --git a/c++/src/Reader.cc b/c++/src/Reader.cc
index 29bd439..2aa3ef5 100644
--- a/c++/src/Reader.cc
+++ b/c++/src/Reader.cc
@@ -1062,7 +1062,7 @@ namespace orc {
 
   class ReaderImpl : public Reader {
   private:
-    const int64_t epochOffset;
+    const std::string localTimezone;
 
     // inputs
     std::unique_ptr<InputStream> stream;
@@ -1215,27 +1215,13 @@ namespace orc {
     }
   }
 
-  int64_t getEpochOffset() {
-    // Build the literal for the ORC epoch
-    // 2015 Jan 1 00:00:00
-    struct tm epoch;
-    epoch.tm_sec = 0;
-    epoch.tm_min = 0;
-    epoch.tm_hour = 0;
-    epoch.tm_mday = 1;
-    epoch.tm_mon = 0;
-    epoch.tm_year = 2015 - 1900;
-    epoch.tm_isdst = 0;
-    return static_cast<int64_t>(mktime(&epoch));
-  }
-
   ReaderImpl::ReaderImpl(std::unique_ptr<InputStream> input,
                          const ReaderOptions& opts,
                          std::unique_ptr<proto::PostScript> _postscript,
                          std::unique_ptr<proto::Footer> _footer,
                          uint64_t _fileLength,
                          uint64_t _postscriptLength
-                         ): epochOffset(getEpochOffset()),
+                         ): localTimezone(getLocalTimezoneName()),
                             stream(std::move(input)),
                             options(opts),
                             fileLength(_fileLength),
@@ -1588,7 +1574,7 @@ namespace orc {
     const uint64_t stripeStart;
     InputStream& input;
     MemoryPool& memoryPool;
-    const int64_t epochOffset;
+    const Timezone& writerTimezone;
 
   public:
     StripeStreamsImpl(const ReaderImpl& reader,
@@ -1596,7 +1582,7 @@ namespace orc {
                       uint64_t stripeStart,
                       InputStream& input,
                       MemoryPool& memoryPool,
-                      int64_t epochOffset);
+                      const Timezone& writerTimezone);
 
     virtual ~StripeStreamsImpl();
 
@@ -1614,7 +1600,7 @@ namespace orc {
 
     MemoryPool& getMemoryPool() const override;
 
-    int64_t getEpochOffset() const override;
+    const Timezone& getWriterTimezone() const override;
   };
 
   uint64_t maxStreamsForType(const proto::Type& type) {
@@ -1727,13 +1713,13 @@ namespace orc {
                                        uint64_t _stripeStart,
                                        InputStream& _input,
                                        MemoryPool& _memoryPool,
-                                       int64_t _epochOffset
+                                       const Timezone& _writerTimezone
                                        ): reader(_reader),
                                           footer(_footer),
                                           stripeStart(_stripeStart),
                                           input(_input),
                                           memoryPool(_memoryPool),
-                                          epochOffset(_epochOffset) {
+                                          writerTimezone(_writerTimezone) {
     // PASS
   }
 
@@ -1754,8 +1740,8 @@ namespace orc {
     return footer.columns(static_cast<int>(columnId));
   }
 
-  int64_t StripeStreamsImpl::getEpochOffset() const {
-    return epochOffset;
+  const Timezone& StripeStreamsImpl::getWriterTimezone() const {
+    return writerTimezone;
   }
 
   std::unique_ptr<SeekableInputStream>
@@ -1795,11 +1781,15 @@ namespace orc {
     currentStripeInfo = footer->stripes(static_cast<int>(currentStripe));
     currentStripeFooter = getStripeFooter(currentStripeInfo);
     rowsInCurrentStripe = currentStripeInfo.numberofrows();
+    const Timezone& writerTimezone =
+      (currentStripeFooter.has_writertimezone() ?
+       getTimezoneByName(currentStripeFooter.writertimezone()) :
+       getTimezoneByName(localTimezone));
     StripeStreamsImpl stripeStreams(*this, currentStripeFooter,
                                     currentStripeInfo.offset(),
                                     *(stream.get()),
                                     memoryPool,
-                                    epochOffset);
+                                    writerTimezone);
     reader = buildReader(*(schema.get()), stripeStreams);
   }
 

http://git-wip-us.apache.org/repos/asf/orc/blob/6dce746c/c++/src/Timezone.cc
----------------------------------------------------------------------
diff --git a/c++/src/Timezone.cc b/c++/src/Timezone.cc
new file mode 100644
index 0000000..0d6ff93
--- /dev/null
+++ b/c++/src/Timezone.cc
@@ -0,0 +1,964 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "Timezone.hh"
+
+#include <errno.h>
+#include <iostream>
+#include <fcntl.h>
+#include <map>
+#include <sstream>
+#include <stdint.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <sys/stat.h>
+#include <time.h>
+#include <unistd.h>
+
+namespace orc {
+
+  // default location of the timezone files
+  static const char DEFAULT_TZDIR[] = "/usr/share/zoneinfo";
+
+  // location of a symlink to the local timezone
+  static const char LOCAL_TIMEZONE[] = "/etc/localtime";
+
+  enum TransitionKind {
+    TRANSITION_JULIAN,
+    TRANSITION_DAY,
+    TRANSITION_MONTH
+  };
+
+  static const int64_t MONTHS_PER_YEAR = 12;
+  /**
+   * The number of days in each month in non-leap and leap years.
+   */
+  static const int64_t DAYS_PER_MONTH[2][MONTHS_PER_YEAR] =
+     {{31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31},
+      {31, 29, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}};
+  static const int64_t SECONDS_PER_HOUR = 60 * 60;
+  static const int64_t SECONDS_PER_DAY = SECONDS_PER_HOUR * 24;
+  static const int64_t DAYS_PER_WEEK = 7;
+
+  // Leap years and day of the week repeat every 400 years, which makes it
+  // a good cycle length.
+  static const int64_t SECONDS_PER_400_YEARS =
+    SECONDS_PER_DAY * (365 * (300 + 3) + 366 * (100 - 3));
+
+  /**
+   * Is the given year a leap year?
+   */
+  bool isLeap(int64_t year) {
+    return (year % 4 == 0) && ((year % 100 != 0) || (year % 400 == 0));
+  }
+
+  /**
+   * Find the position that is the closest and less than or equal to the
+   * target.
+   * @return -1 if the target < array[0] or
+   *          i if array[i] <= target and (i == n or array[i] < array[i+1])
+   */
+  int64_t binarySearch(const std::vector<int64_t> &array, int64_t target) {
+    uint64_t min = 0;
+    uint64_t max = array.size() - 1;
+    uint64_t mid = (min + max) / 2;
+    while ((array[mid] != target) && (min < max)) {
+      if (array[mid] < target) {
+        min = mid + 1;
+      } else if (mid == 0) {
+        max = 0;
+      } else {
+        max = mid - 1;
+      }
+      mid = (min + max) / 2;
+    }
+    if (target < array[mid]) {
+      return static_cast<int64_t>(mid) - 1;
+    } else {
+      return static_cast<int64_t>(mid);
+    }
+  }
+
+  struct Transition {
+    TransitionKind kind;
+    int64_t day;
+    int64_t week;
+    int64_t month;
+    int64_t time;
+
+    std::string toString() const {
+      std::stringstream buffer;
+      switch (kind) {
+      case TRANSITION_JULIAN:
+        buffer << "julian " << day;
+        break;
+      case TRANSITION_DAY:
+        buffer << "day " << day;
+        break;
+      case TRANSITION_MONTH:
+        buffer << "month " << month << " week " << week << " day " << day;
+        break;
+      }
+      buffer << " at " << (time / (60 * 60)) << ":" << ((time / 60) % 60)
+             << ":" << (time % 60);
+      return buffer.str();
+    }
+
+    /**
+     * Get the transition time for the given year.
+     * @param year the year
+     * @return the number of seconds past local Jan 1 00:00:00 that the
+     *    transition happens.
+     */
+    int64_t getTime(int64_t year) const {
+      int64_t result = time;
+      switch (kind) {
+      case TRANSITION_JULIAN:
+        result += SECONDS_PER_DAY * day;
+        if (day > 60 && isLeap(year)) {
+          result += SECONDS_PER_DAY;
+        }
+        break;
+      case TRANSITION_DAY:
+        result += SECONDS_PER_DAY * day;
+        break;
+      case TRANSITION_MONTH: {
+        bool inLeap = isLeap(year);
+        int64_t adjustedMonth = (month + 9) % 12 + 1;
+        int64_t adjustedYear = (month <= 2) ? (year - 1) : year;
+        int64_t adjustedCentury = adjustedYear / 100;
+        int64_t adjustedRemainder = adjustedYear % 100;
+
+        // day of the week of the first day of month
+        int64_t dayOfWeek = ((26 * adjustedMonth - 2) / 10 +
+                             1 + adjustedRemainder + adjustedRemainder / 4 +
+                             adjustedCentury / 4 - 2 * adjustedCentury) % 7;
+        if (dayOfWeek < 0) {
+          dayOfWeek += DAYS_PER_WEEK;
+        }
+
+        int64_t d = day - dayOfWeek;
+        if (d < 0) {
+          d += DAYS_PER_WEEK;
+        }
+        for (int w = 1; w < week; ++w) {
+          if (d + DAYS_PER_WEEK >= DAYS_PER_MONTH[inLeap][month - 1]) {
+            break;
+          }
+          d += DAYS_PER_WEEK;
+        }
+        result += d * SECONDS_PER_DAY;
+
+        // Add in the time for the month
+        for(int m=0; m < month - 1; ++m) {
+          result += DAYS_PER_MONTH[inLeap][m] * SECONDS_PER_DAY;
+        }
+        break;
+      }
+      }
+      return result;
+    }
+  };
+
+  /**
+   * The current rule for finding timezone variants arbitrarily far in
+   * the future.  They are based on a string representation that
+   * specifies the standard name and offset. For timezones with
+   * daylight savings, the string specifies the daylight variant name
+   * and offset and the rules for switching between them.
+   *
+   * rule = <standard name><standard offset><daylight>?
+   * name = string with no numbers or '+', '-', or ','
+   * offset = [-+]?hh(:mm(:ss)?)?
+   * daylight = <name><offset>,<start day>(/<offset>)?,<end day>(/<offset>)?
+   * day = J<day without 2/29>|<day with 2/29>|M<month>.<week>.<day of week>
+   */
+  class FutureRuleImpl: public FutureRule {
+    std::string ruleString;
+    TimezoneVariant standard;
+    bool hasDst;
+    TimezoneVariant dst;
+    Transition start;
+    Transition end;
+
+    // expanded time_t offsets of transitions
+    std::vector<int64_t> offsets;
+
+    // Is the epoch (1 Jan 1970 00:00) in standard time?
+    // This code assumes that the transition dates fall in the same order
+    // each year. Hopefully no timezone regions decide to move across the
+    // equator, which is about what it would take.
+    bool startInStd;
+
+    void computeOffsets() {
+      if (!hasDst) {
+        startInStd = true;
+        offsets.resize(1);
+      } else {
+        // Insert a transition for the epoch and two per a year for the next
+        // 400 years. We assume that the all even positions are in standard
+        // time if and only if startInStd and the odd ones are the reverse.
+        offsets.resize(400 * 2 + 1);
+        startInStd = start.getTime(1970) < end.getTime(1970);
+        int64_t base = 0;
+        for(int64_t year = 1970; year < 1970 + 400; ++year) {
+          if (startInStd) {
+            offsets[static_cast<uint64_t>(year - 1970) * 2 + 1] =
+              base + start.getTime(year) - standard.gmtOffset;
+            offsets[static_cast<uint64_t>(year - 1970) * 2 + 2] =
+              base + end.getTime(year) - dst.gmtOffset;
+          } else {
+            offsets[static_cast<uint64_t>(year - 1970) * 2 + 1] =
+              base + end.getTime(year) - dst.gmtOffset;
+            offsets[static_cast<uint64_t>(year - 1970) * 2 + 2] =
+              base + start.getTime(year) - standard.gmtOffset;
+          }
+          base += (isLeap(year) ? 366 : 365) * SECONDS_PER_DAY;
+        }
+      }
+      offsets[0] = 0;
+    }
+
+  public:
+    virtual ~FutureRuleImpl();
+    bool isDefined() const override;
+    const TimezoneVariant& getVariant(int64_t clk) const override;
+    void print(std::ostream& out) const override;
+
+    friend class FutureRuleParser;
+  };
+
+  FutureRule::~FutureRule() {
+    // PASS
+  }
+
+  FutureRuleImpl::~FutureRuleImpl() {
+    // PASS
+  }
+
+  bool FutureRuleImpl::isDefined() const {
+    return ruleString.size() > 0;
+  }
+
+  const TimezoneVariant& FutureRuleImpl::getVariant(int64_t clk) const {
+    if (!hasDst) {
+      return standard;
+    } else {
+      int64_t adjusted = clk % SECONDS_PER_400_YEARS;
+      if (adjusted < 0) {
+        adjusted += SECONDS_PER_400_YEARS;
+      }
+      int64_t idx = binarySearch(offsets, adjusted);
+      if (startInStd == (idx % 2 == 0)) {
+        return standard;
+      } else {
+        return dst;
+      }
+    }
+  }
+
+  void FutureRuleImpl::print(std::ostream& out) const {
+    if (isDefined()) {
+      out << "  Future rule: " << ruleString << "\n";
+      out << "  standard " << standard.toString() << "\n";
+      if (hasDst) {
+        out << "  dst " << dst.toString() << "\n";
+        out << "  start " << start.toString() << "\n";
+        out << "  end " << end.toString() << "\n";
+      }
+    }
+  }
+
+  /**
+   * A parser for the future rule strings.
+   */
+  class FutureRuleParser {
+  public:
+    FutureRuleParser(const std::string& str,
+                     FutureRuleImpl* rule
+                     ): ruleString(str),
+                        length(str.size()),
+                        position(0),
+                        output(*rule) {
+      output.ruleString = str;
+      if (position != length) {
+        parseName(output.standard.name);
+        output.standard.gmtOffset = -parseOffset();
+        output.standard.isDst = false;
+        output.hasDst = position < length;
+        if (output.hasDst) {
+          parseName(output.dst.name);
+          output.dst.isDst = true;
+          if (ruleString[position] != ',') {
+            output.dst.gmtOffset = -parseOffset();
+          } else {
+            output.dst.gmtOffset = output.standard.gmtOffset + 60 * 60;
+          }
+          parseTransition(output.start);
+          parseTransition(output.end);
+        }
+        if (position != length) {
+          throwError("Extra text");
+        }
+        output.computeOffsets();
+      }
+    }
+
+  private:
+
+    const std::string& ruleString;
+    size_t length;
+    size_t position;
+    FutureRuleImpl &output;
+
+    void throwError(const char *msg) {
+      std::stringstream buffer;
+      buffer << msg << " at " << position << " in '" << ruleString << "'";
+      throw TimezoneError(buffer.str());
+    }
+
+    /**
+     * Parse the names of the form:
+     *    ([^-+0-9,]+|<[^>]+>)
+     * and set the output string.
+     */
+    void parseName(std::string& result) {
+      if (position == length) {
+        throwError("name required");
+      }
+      size_t start = position;
+      if (ruleString[position] == '<') {
+        while (position < length && ruleString[position] != '>') {
+          position += 1;
+        }
+        if (position == length) {
+          throwError("missing close '>'");
+        }
+        position +=1;
+      } else {
+        while (position < length) {
+          char ch = ruleString[position];
+          if (isdigit(ch) || ch == '-' || ch == '+' || ch == ',') {
+            break;
+          }
+          position += 1;
+        }
+      }
+      if (position == start) {
+        throwError("empty string not allowed");
+      }
+      result = ruleString.substr(start, position - start);
+    }
+
+    /**
+     * Parse an integer of the form [0-9]+ and return it.
+     */
+    int64_t parseNumber() {
+      if (position >= length) {
+        throwError("missing number");
+      }
+      int64_t result = 0;
+      while (position < length) {
+        char ch = ruleString[position];
+        if (isdigit(ch)) {
+          result = result * 10 + (ch - '0');
+          position += 1;
+        } else {
+          break;
+        }
+      }
+      return result;
+    }
+
+    /**
+     * Parse the offsets of the form:
+     *    [-+]?[0-9]+(:[0-9]+(:[0-9]+)?)?
+     * and convert it into a number of seconds.
+     */
+    int64_t parseOffset() {
+      int64_t scale = 3600;
+      bool isNegative = false;
+      if (position < length) {
+        char ch = ruleString[position];
+        isNegative = ch == '-';
+        if (ch == '-' || ch == '+') {
+          position += 1;
+        }
+      }
+      int64_t result = parseNumber() * scale;
+      while (position < length && scale > 1 && ruleString[position] == ':') {
+        scale /= 60;
+        position += 1;
+        result += parseNumber() * scale;
+      }
+      if (isNegative) {
+        result = -result;
+      }
+      return result;
+    }
+
+    /**
+     * Parse a transition of the following form:
+     *   ,(J<number>|<number>|M<number>.<number>.<number>)(/<offset>)?
+     */
+    void parseTransition(Transition& transition) {
+      if (length - position < 2 || ruleString[position] != ',') {
+        throwError("missing transition");
+      }
+      position += 1;
+      char ch = ruleString[position];
+      if (ch == 'J') {
+        transition.kind = TRANSITION_JULIAN;
+        position += 1;
+        transition.day = parseNumber();
+      } else if (ch == 'M') {
+        transition.kind = TRANSITION_MONTH;
+        position += 1;
+        transition.month = parseNumber();
+        if (position == length || ruleString[position] != '.') {
+          throwError("missing first .");
+        }
+        position += 1;
+        transition.week = parseNumber();
+        if (position == length || ruleString[position] != '.') {
+          throwError("missing second .");
+        }
+        position += 1;
+        transition.day = parseNumber();
+      } else {
+        transition.kind = TRANSITION_DAY;
+        transition.day = parseNumber();
+      }
+      if (position < length && ruleString[position] == '/') {
+        position += 1;
+        transition.time = parseOffset();
+      } else {
+        transition.time = 2 * 60 * 60;
+      }
+    }
+  };
+
+  /**
+   * Parse the POSIX TZ string.
+   */
+  std::unique_ptr<FutureRule> parseFutureRule(const std::string& ruleString) {
+    std::unique_ptr<FutureRule> result(new FutureRuleImpl());
+    FutureRuleParser parser(ruleString,
+                            dynamic_cast<FutureRuleImpl*>(result.get()));
+    return result;
+  }
+
+  std::string TimezoneVariant::toString() const {
+    std::stringstream buffer;
+    buffer << name << " " << gmtOffset;
+    if (isDst) {
+      buffer << " (dst)";
+    }
+    return buffer.str();
+  }
+
+  /**
+   * An abstraction of the differences between versions.
+   */
+  class VersionParser {
+  public:
+    virtual ~VersionParser();
+
+    /**
+     * Get the version number.
+     */
+    virtual uint64_t getVersion() const = 0;
+
+    /**
+     * Get the number of bytes
+     */
+    virtual uint64_t getTimeSize() const = 0;
+
+    /**
+     * Parse the time at the given location.
+     */
+    virtual int64_t parseTime(const unsigned char* ptr) const = 0;
+
+    /**
+     * Parse the future string
+     */
+    virtual std::string parseFutureString(const unsigned char *ptr,
+                                          uint64_t offset,
+                                          uint64_t length) const = 0;
+  };
+
+  VersionParser::~VersionParser() {
+    // PASS
+  }
+
+  static uint32_t decode32(const unsigned char* ptr) {
+    return static_cast<uint32_t>(ptr[0] << 24) |
+      static_cast<uint32_t>(ptr[1] << 16) |
+      static_cast<uint32_t>(ptr[2] << 8) |
+      static_cast<uint32_t>(ptr[3]);
+  }
+
+  class Version1Parser: public VersionParser {
+  public:
+    virtual ~Version1Parser();
+
+    virtual uint64_t getVersion() const override {
+      return 1;
+    }
+
+    /**
+     * Get the number of bytes
+     */
+    virtual uint64_t getTimeSize() const override {
+      return 4;
+    }
+
+    /**
+     * Parse the time at the given location.
+     */
+    virtual int64_t parseTime(const unsigned char* ptr) const override {
+      // sign extend from 32 bits
+      return static_cast<int32_t>(decode32(ptr));
+    }
+
+    virtual std::string parseFutureString(const unsigned char *,
+                                          uint64_t,
+                                          uint64_t) const override {
+      return "";
+    }
+  };
+
+  Version1Parser::~Version1Parser() {
+    // PASS
+  }
+
+  class Version2Parser: public VersionParser {
+  public:
+    virtual ~Version2Parser();
+
+    virtual uint64_t getVersion() const override {
+      return 2;
+    }
+
+    /**
+     * Get the number of bytes
+     */
+    virtual uint64_t getTimeSize() const override {
+      return 8;
+    }
+
+    /**
+     * Parse the time at the given location.
+     */
+    virtual int64_t parseTime(const unsigned char* ptr) const override {
+      return static_cast<int64_t>(decode32(ptr)) << 32 | decode32(ptr + 4);
+    }
+
+    virtual std::string parseFutureString(const unsigned char *ptr,
+                                          uint64_t offset,
+                                          uint64_t length) const override {
+      return std::string(reinterpret_cast<const char*>(ptr) + offset + 1,
+                         length - 2);
+    }
+  };
+
+  Version2Parser::~Version2Parser() {
+    // PASS
+  }
+
+  class TimezoneImpl: public Timezone {
+  public:
+    TimezoneImpl(const std::string& name,
+                 const std::vector<unsigned char> bytes);
+    virtual ~TimezoneImpl();
+
+    /**
+     * Get the variant for the given time (time_t).
+     */
+    const TimezoneVariant& getVariant(int64_t clk) const override;
+
+    void print(std::ostream&) const override;
+
+    uint64_t getVersion() const override {
+      return version;
+    }
+
+    int64_t getEpoch() const override {
+      return epoch;
+    }
+
+  private:
+    void parseTimeVariants(const unsigned char* ptr,
+                           uint64_t variantOffset,
+                           uint64_t variantCount,
+                           uint64_t nameOffset,
+                           uint64_t nameCount);
+    void parseZoneFile(const unsigned char* ptr,
+                       uint64_t sectionOffset,
+                       uint64_t fileLength,
+                       const VersionParser& version);
+    // filename
+    std::string filename;
+
+    // the version of the file
+    uint64_t version;
+
+    // the list of variants for this timezone
+    std::vector<TimezoneVariant> variants;
+
+    // the list of the times where the local rules change
+    std::vector<int64_t> transitions;
+
+    // the variant that starts at this transition.
+    std::vector<uint64_t> currentVariant;
+
+    // the variant before the first transition
+    uint64_t ancientVariant;
+
+    // the rule for future times
+    std::unique_ptr<FutureRule> futureRule;
+
+    // the last explicit transition after which we use the future rule
+    int64_t lastTransition;
+
+    // The ORC epoch time in this timezone.
+    int64_t epoch;
+  };
+
+  DIAGNOSTIC_PUSH
+  #ifdef __clang__
+    DIAGNOSTIC_IGNORE("-Wglobal-constructors")
+    DIAGNOSTIC_IGNORE("-Wexit-time-destructors")
+  #endif
+  static std::map<std::string, Timezone*> timezoneCache;
+  DIAGNOSTIC_POP
+
+  Timezone::~Timezone() {
+    // PASS
+  }
+
+  TimezoneImpl::TimezoneImpl(const std::string& _filename,
+                             const std::vector<unsigned char> buffer
+                             ): filename(_filename) {
+    parseZoneFile(&buffer[0], 0, buffer.size(), Version1Parser());
+    // Build the literal for the ORC epoch
+    // 2015 Jan 1 00:00:00
+    tm epochStruct;
+    epochStruct.tm_sec = 0;
+    epochStruct.tm_min = 0;
+    epochStruct.tm_hour = 0;
+    epochStruct.tm_mday = 1;
+    epochStruct.tm_mon = 0;
+    epochStruct.tm_year = 2015 - 1900;
+    epochStruct.tm_isdst = 0;
+    time_t utcEpoch = timegm(&epochStruct);
+    epoch = utcEpoch - getVariant(utcEpoch).gmtOffset;
+  }
+
+  const char* getTimezoneDirectory() {
+    const char *dir = getenv("TZDIR");
+    if (!dir) {
+      dir = DEFAULT_TZDIR;
+    }
+    return dir;
+  }
+
+  std::string getLocalTimezoneName() {
+    // use the TZ environment variable, if it is set.
+    const char *tz = getenv("TZ");
+    if (tz != nullptr) {
+      return std::string(tz);
+    }
+    // otherwise look at where /etc/localtime points and use that
+    struct stat linkStatus;
+    if (lstat(LOCAL_TIMEZONE, &linkStatus) == -1) {
+      throw TimezoneError(std::string("Can't stat local timezone link ") +
+                          LOCAL_TIMEZONE + ": " +
+                          strerror(errno));
+    }
+    std::vector<char> buffer(static_cast<size_t>(linkStatus.st_size + 1));
+    ssize_t len = readlink(LOCAL_TIMEZONE, &buffer[0], buffer.size());
+    if (len == -1 || static_cast<size_t>(len) >= buffer.size()) {
+      throw TimezoneError(std::string("Can't read local timezone link ") +
+                          LOCAL_TIMEZONE + ": " +
+                          strerror(errno));
+    }
+    buffer[static_cast<size_t>(len)] = 0;
+    const char *dirName = getTimezoneDirectory();
+    size_t dirLength = strlen(dirName);
+    if (strncmp(dirName, &buffer[0], dirLength) != 0) {
+      throw TimezoneError(std::string("Local timezone link is not in ") +
+                          dirName);
+    }
+    return std::string(&buffer[0] + dirLength);
+  }
+
+  /**
+   * Get a timezone by name (eg. America/Los_Angeles).
+   * Results are cached.
+   */
+  const Timezone& getTimezoneByName(const std::string& zone) {
+    std::map<std::string, Timezone*>::iterator itr = timezoneCache.find(zone);
+    if (itr != timezoneCache.end()) {
+      return *(itr->second);
+    }
+    std::string filename(getTimezoneDirectory());
+    filename += "/";
+    filename += zone;
+    int in = open(filename.c_str(), O_RDONLY);
+    if (in == -1) {
+      std::stringstream buffer;
+      buffer << "failed to open " << filename << " - " << strerror(errno);
+      throw TimezoneError(buffer.str());
+    }
+    struct stat fileInfo;
+    if (fstat(in, &fileInfo) == -1) {
+      std::stringstream buffer;
+      buffer << "failed to stat " << filename << " - " << strerror(errno);
+      throw TimezoneError(buffer.str());
+    }
+    if ((fileInfo.st_mode & S_IFMT) != S_IFREG) {
+      std::stringstream buffer;
+      buffer << "non-file in tzfile reader " << filename;
+      throw TimezoneError(buffer.str());
+    }
+    size_t size = static_cast<size_t>(fileInfo.st_size);
+    std::vector<unsigned char> buffer(size);
+    size_t posn = 0;
+    while (posn < size) {
+      ssize_t ret = read(in, &buffer[posn], size - posn);
+      if (ret == -1) {
+        throw TimezoneError(std::string("Failure to read timezone file ") +
+                            filename + " - " + strerror(errno));
+      }
+      posn += static_cast<size_t>(ret);
+    }
+    if (close(in) == -1) {
+      std::stringstream err;
+      err << "failed to close " << filename << " - " << strerror(errno);
+      throw TimezoneError(err.str());
+    }
+    Timezone* result = new TimezoneImpl(filename, buffer);
+    timezoneCache[zone] = result;
+    return *result;
+  }
+
+  /**
+   * Parse a set of bytes as a timezone file as if they came from filename.
+   */
+  std::unique_ptr<Timezone> getTimezone(const std::string& filename,
+                                        const std::vector<unsigned char>& b){
+    return std::unique_ptr<Timezone>(new TimezoneImpl(filename, b));
+  }
+
+  TimezoneImpl::~TimezoneImpl() {
+    // PASS
+  }
+
+  void TimezoneImpl::parseTimeVariants(const unsigned char* ptr,
+                                       uint64_t variantOffset,
+                                       uint64_t variantCount,
+                                       uint64_t nameOffset,
+                                       uint64_t nameCount) {
+    for(uint64_t variant=0; variant < variantCount; ++variant) {
+      variants[variant].gmtOffset =
+        static_cast<int32_t>(decode32(ptr + variantOffset + 6 * variant));
+      variants[variant].isDst = ptr[variantOffset + 6 * variant + 4];
+      uint nameStart = ptr[variantOffset + 6 * variant + 5];
+      if (nameStart >= nameCount) {
+        std::stringstream buffer;
+        buffer << "name out of range in variant " << variant
+               << " - " << nameStart << " >= " << nameCount;
+        throw TimezoneError(buffer.str());
+      }
+      variants[variant].name = std::string(reinterpret_cast<const char*>(ptr)
+                                           + nameOffset + nameStart);
+    }
+  }
+
+  /**
+   * Parse the zone file to get the bits we need.
+   * There are two versions of the timezone file:
+   *
+   * Version 1(version = 0x00):
+   *   Magic(version)
+   *   Header
+   *   TransitionTimes(4 byte)
+   *   TransitionRules
+   *   Rules
+   *   LeapSeconds(4 byte)
+   *   IsStd
+   *   IsGmt
+   *
+   * Version2:
+   *   Version1(0x32) = a version 1 copy of the data for old clients
+   *   Magic(0x32)
+   *   Header
+   *   TransitionTimes(8 byte)
+   *   TransitionRules
+   *   Rules
+   *   LeapSeconds(8 byte)
+   *   IsStd
+   *   IsGmt
+   *   FutureString
+   */
+  void TimezoneImpl::parseZoneFile(const unsigned char *ptr,
+                                   uint64_t sectionOffset,
+                                   uint64_t fileLength,
+                                   const VersionParser& versionParser) {
+    const uint64_t magicOffset  = sectionOffset + 0;
+    const uint64_t headerOffset = magicOffset + 20;
+
+    // check for validity before we start parsing
+    if (fileLength < headerOffset + 6 * 4 ||
+        strncmp(reinterpret_cast<const char*>(ptr) + magicOffset, "TZif", 4)
+          != 0) {
+      std::stringstream buffer;
+      buffer << "non-tzfile " << filename;
+      throw TimezoneError(buffer.str());
+    }
+
+    const uint64_t isGmtCount = decode32(ptr + headerOffset + 0);
+    const uint64_t isStdCount = decode32(ptr + headerOffset + 4);
+    const uint64_t leapCount  = decode32(ptr + headerOffset + 8);
+    const uint64_t timeCount  = decode32(ptr + headerOffset + 12);
+    const uint64_t variantCount  = decode32(ptr + headerOffset + 16);
+    const uint64_t nameCount  = decode32(ptr + headerOffset + 20);
+
+    const uint64_t timeOffset = headerOffset + 24;
+    const uint64_t timeVariantOffset =
+      timeOffset + versionParser.getTimeSize() * timeCount;
+    const uint64_t variantOffset = timeVariantOffset + timeCount;
+    const uint64_t nameOffset = variantOffset + variantCount * 6;
+    const uint64_t sectionLength = nameOffset + nameCount
+      + (versionParser.getTimeSize() + 4) * leapCount
+      + isGmtCount + isStdCount;
+
+    if (sectionLength > fileLength) {
+      std::stringstream buffer;
+      buffer << "tzfile too short " << filename
+             << " needs " << sectionLength << " and has " << fileLength;
+      throw TimezoneError(buffer.str());
+    }
+
+    // if it is version 2, skip over the old layout and read the new one.
+    if (sectionOffset == 0 && ptr[magicOffset + 4] != 0) {
+      parseZoneFile(ptr, sectionLength, fileLength, Version2Parser());
+      return;
+    }
+    version = versionParser.getVersion();
+    variants.resize(variantCount);
+    transitions.resize(timeCount);
+    currentVariant.resize(timeCount);
+    parseTimeVariants(ptr, variantOffset, variantCount, nameOffset,
+                      nameCount);
+    bool foundAncient = false;
+    for(uint64_t t=0; t < timeCount; ++t) {
+      transitions[t] =
+        versionParser.parseTime(ptr + timeOffset +
+                                t * versionParser.getTimeSize());
+      currentVariant[t] = ptr[timeVariantOffset + t];
+      if (currentVariant[t] >= variantCount) {
+        std::stringstream buffer;
+        buffer << "tzfile rule out of range " << filename
+               << " references rule " << currentVariant[t]
+               << " of " << variantCount;
+        throw TimezoneError(buffer.str());
+      }
+      // find the oldest standard time and use that as the ancient value
+      if (!foundAncient &&
+          !variants[currentVariant[t]].isDst) {
+        foundAncient = true;
+        ancientVariant = currentVariant[t];
+      }
+    }
+    if (!foundAncient) {
+      ancientVariant = 0;
+    }
+    futureRule = parseFutureRule(versionParser.parseFutureString
+                                   (ptr, sectionLength,
+                                    fileLength - sectionLength));
+
+    // find the lower bound for applying the future rule
+    if (futureRule->isDefined()) {
+      if (timeCount > 0) {
+        lastTransition = transitions[timeCount - 1];
+      } else {
+        lastTransition = INT64_MIN;
+      }
+    } else {
+      lastTransition = INT64_MAX;
+    }
+  }
+
+  const TimezoneVariant& TimezoneImpl::getVariant(int64_t clk) const {
+    // if it is after the last explicit entry in the table,
+    // use the future rule to get an answer
+    if (clk > lastTransition) {
+      return futureRule->getVariant(clk);
+    } else {
+      int64_t idx = binarySearch(transitions, clk);
+      if (idx < 0) {
+        idx = static_cast<int64_t>(ancientVariant);
+      }
+      return variants[currentVariant[static_cast<size_t>(idx)]];
+    }
+  }
+
+  void TimezoneImpl::print(std::ostream& out) const {
+    out << "Timezone file: " << filename << "\n";
+    out << "  Version: " << version << "\n";
+    futureRule->print(out);
+    for(uint64_t r=0; r < variants.size(); ++r) {
+      out <<  "  Variant " << r << ": "
+          << variants[r].toString() << "\n";
+    }
+    for(uint64_t t=0; t < transitions.size(); ++t) {
+      tm timeStruct;
+      tm* result = nullptr;
+      char buffer[25];
+      if (sizeof(time_t) >= 8) {
+        time_t val = transitions[t];
+        result = gmtime_r(&val, &timeStruct);
+        if (result) {
+          strftime(buffer, sizeof(buffer), "%F %H:%M:%S", &timeStruct);
+        }
+      }
+      std::cout << "  Transition: " << (result == nullptr ? "null" : buffer)
+                << " (" << transitions[t] << ") -> "
+                << variants[currentVariant[t]].name
+                << "\n";
+    }
+  }
+
+  TimezoneError::TimezoneError(const std::string& what
+                               ): std::runtime_error(what) {
+    // PASS
+  }
+
+  TimezoneError::TimezoneError(const TimezoneError& other
+                               ): std::runtime_error(other) {
+    // PASS
+  }
+
+  TimezoneError::~TimezoneError() ORC_NOEXCEPT {
+    // PASS
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/6dce746c/c++/src/Timezone.hh
----------------------------------------------------------------------
diff --git a/c++/src/Timezone.hh b/c++/src/Timezone.hh
new file mode 100644
index 0000000..e35f170
--- /dev/null
+++ b/c++/src/Timezone.hh
@@ -0,0 +1,121 @@
+/**
+ * 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.
+ */
+
+#ifndef TIMEZONE_HH
+#define TIMEZONE_HH
+
+// This file is for timezone routines.
+
+#include "Adaptor.hh"
+
+#include <memory>
+#include <stdexcept>
+#include <stdint.h>
+#include <string>
+#include <vector>
+
+namespace orc {
+
+  /**
+   * A variant  (eg. PST or PDT) of a timezone (eg. America/Los_Angeles).
+   */
+  struct TimezoneVariant {
+    int64_t gmtOffset;
+    bool isDst;
+    std::string name;
+
+    std::string toString() const;
+  };
+
+  /**
+   * A region that shares the same legal rules for wall clock time and
+   * day light savings transitions. They are typically named for the largest
+   * city in the region (eg. America/Los_Angeles or America/Mexico_City).
+   */
+  class Timezone {
+  public:
+    virtual ~Timezone();
+
+    /**
+     * Get the variant for the given time (time_t).
+     */
+    virtual const TimezoneVariant& getVariant(int64_t clk) const = 0;
+
+    /**
+     * Get the number of seconds between the ORC epoch in this timezone
+     * and Unix epoch.
+     * ORC epoch is 1 Jan 2015 00:00:00 local.
+     * Unix epoch is 1 Jan 1970 00:00:00 UTC.
+     */
+    virtual int64_t getEpoch() const = 0;
+
+    /**
+     * Print the timezone to the stream.
+     */
+    virtual void print(std::ostream&) const = 0;
+
+    /**
+     * Get the version of the zone file.
+     */
+    virtual uint64_t getVersion() const =0;
+  };
+
+  /**
+   * Get the local timezone name.
+   */
+  std::string getLocalTimezoneName();
+
+  /**
+   * Get a timezone by name (eg. America/Los_Angeles).
+   * Results are cached.
+   */
+  const Timezone& getTimezoneByName(const std::string& zone);
+
+  /**
+   * Parse a set of bytes as a timezone file as if they came from filename.
+   */
+  std::unique_ptr<Timezone> getTimezone(const std::string& filename,
+                                        const std::vector<unsigned char>& b);
+
+  class TimezoneError: public std::runtime_error {
+  public:
+    TimezoneError(const std::string& what);
+    TimezoneError(const TimezoneError&);
+    virtual ~TimezoneError() ORC_NOEXCEPT;
+  };
+
+  /**
+   * Represents the parsed POSIX timezone rule strings that are used to
+   * describe the future transitions, because they can go arbitrarily far into
+   * the future.
+   */
+  class FutureRule {
+  public:
+    virtual ~FutureRule();
+    virtual bool isDefined() const = 0;
+    virtual const TimezoneVariant& getVariant(int64_t clk) const = 0;
+    virtual void print(std::ostream& out) const = 0;
+  };
+
+  /**
+   * Parse the POSIX TZ string.
+   */
+  std::unique_ptr<FutureRule> parseFutureRule(const std::string& ruleString);
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/orc/blob/6dce746c/c++/test/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/c++/test/CMakeLists.txt b/c++/test/CMakeLists.txt
index 4a09e9e..69a0f6b 100644
--- a/c++/test/CMakeLists.txt
+++ b/c++/test/CMakeLists.txt
@@ -30,6 +30,7 @@ add_executable (test-orc
   TestDriver.cc
   TestInt128.cc
   TestRle.cc
+  TestTimezone.cc
   TestType.cc
 )
 

http://git-wip-us.apache.org/repos/asf/orc/blob/6dce746c/c++/test/TestColumnPrinter.cc
----------------------------------------------------------------------
diff --git a/c++/test/TestColumnPrinter.cc b/c++/test/TestColumnPrinter.cc
index 8cc3a22..f617f3b 100644
--- a/c++/test/TestColumnPrinter.cc
+++ b/c++/test/TestColumnPrinter.cc
@@ -133,18 +133,18 @@ namespace orc {
     TimestampVectorBatch batch(1024, *getDefaultPool());
     batch.numElements = 12;
     batch.hasNulls = false;
-    batch.data[0]  = 1420099200;
-    batch.data[1]  =  963298800;
-    batch.data[2]  = 1426197659;
-    batch.data[3]  = 1426197659;
-    batch.data[4]  = 1426197659;
-    batch.data[5]  = 1426197659;
-    batch.data[6]  = 1426197659;
-    batch.data[7]  = 1426197659;
-    batch.data[8]  = 1426197659;
-    batch.data[9]  = 1426197659;
-    batch.data[10] = 1426197659;
-    batch.data[11] = 1426197659;
+    batch.data[0]  = 1420070400;
+    batch.data[1]  =  963273600;
+    batch.data[2]  = 1426172459;
+    batch.data[3]  = 1426172459;
+    batch.data[4]  = 1426172459;
+    batch.data[5]  = 1426172459;
+    batch.data[6]  = 1426172459;
+    batch.data[7]  = 1426172459;
+    batch.data[8]  = 1426172459;
+    batch.data[9]  = 1426172459;
+    batch.data[10] = 1426172459;
+    batch.data[11] = 1426172459;
     batch.nanoseconds[0]  = 0;
     batch.nanoseconds[1]  = 0;
     batch.nanoseconds[2]  = 0;

http://git-wip-us.apache.org/repos/asf/orc/blob/6dce746c/c++/test/TestColumnReader.cc
----------------------------------------------------------------------
diff --git a/c++/test/TestColumnReader.cc b/c++/test/TestColumnReader.cc
index 075a069..18b3d67 100644
--- a/c++/test/TestColumnReader.cc
+++ b/c++/test/TestColumnReader.cc
@@ -25,6 +25,7 @@
 #include "wrap/gtest-wrapper.h"
 #include "wrap/gmock.h"
 
+#include <cmath>
 #include <iostream>
 #include <vector>
 
@@ -49,9 +50,8 @@ public:
     return *getDefaultPool();
   }
 
-  // the epoch offset for America/Los_Angeles
-  int64_t getEpochOffset() const {
-    return 1420099200;
+  const Timezone& getWriterTimezone() const override {
+    return getTimezoneByName("America/Los_Angeles");
   }
 };
 
@@ -67,6 +67,10 @@ MockStripeStreams::getStream(uint64_t columnId,
     (getStreamProxy(columnId, kind, shouldStream));
 }
 
+bool isNotNull(tm *timeptr) {
+  return timeptr != nullptr;
+}
+
 TEST(TestColumnReader, testBooleanWithNulls) {
   MockStripeStreams streams;
 
@@ -2380,7 +2384,7 @@ TEST(TestColumnReader, testFloatWithNulls) {
       EXPECT_EQ(0, doubleBatch->notNull[i]);
     } else if (i == 7) {
       EXPECT_EQ(1, doubleBatch->notNull[i]);
-      EXPECT_EQ(true, isnan(doubleBatch->data[i]));
+      EXPECT_EQ(true, std::isnan(doubleBatch->data[i]));
     } else {
       EXPECT_EQ(1, doubleBatch->notNull[i]);
       EXPECT_DOUBLE_EQ(test_vals[i], doubleBatch->data[i]);
@@ -2544,7 +2548,7 @@ TEST(TestColumnReader, testDoubleWithNulls) {
       EXPECT_EQ(0, doubleBatch->notNull[i]) << "Wrong value at " << i;
     } else if (i == 7) {
       EXPECT_EQ(1, doubleBatch->notNull[i]) << "Wrong value at " << i;
-      EXPECT_EQ(true, isnan(doubleBatch->data[i]));
+      EXPECT_EQ(true, std::isnan(doubleBatch->data[i]));
     } else {
       EXPECT_EQ(1, doubleBatch->notNull[i]) << "Wrong value at " << i;
       EXPECT_DOUBLE_EQ(test_vals[i], doubleBatch->data[i])
@@ -2705,7 +2709,11 @@ TEST(TestColumnReader, testTimestampSkipWithNulls) {
     } else {
       EXPECT_EQ(1, longBatch->notNull[i]);
       time_t time = static_cast<time_t>(longBatch->data[i]);
-      EXPECT_STREQ(expected[vals_ix], ctime(&time));
+      tm timeStruct;
+      ASSERT_PRED1(isNotNull, gmtime_r(&time, &timeStruct));
+      char buffer[30];
+      asctime_r(&timeStruct, buffer);
+      EXPECT_STREQ(expected[vals_ix], buffer);
       EXPECT_EQ(expected_nano[vals_ix], longBatch->nanoseconds[i]);
       vals_ix++;
     }
@@ -2724,7 +2732,11 @@ TEST(TestColumnReader, testTimestampSkipWithNulls) {
     } else {
       EXPECT_EQ(1, longBatch->notNull[i]);
       time_t time = static_cast<time_t>(longBatch->data[i]);
-      EXPECT_STREQ(expected[vals_ix], ctime(&time));
+      tm timeStruct;
+      ASSERT_PRED1(isNotNull, gmtime_r(&time, &timeStruct));
+      char buffer[30];
+      asctime_r(&timeStruct, buffer);
+      EXPECT_STREQ(expected[vals_ix], buffer);
       EXPECT_EQ(expected_nano[vals_ix], longBatch->nanoseconds[i]);
       vals_ix++;
     }
@@ -2765,7 +2777,6 @@ TEST(TestColumnReader, testTimestamp) {
       .WillRepeatedly(testing::Return(new SeekableArrayInputStream
                                       (buffer1, ARRAY_SIZE(buffer1))));
 
-
   const unsigned char buffer2[] = { 0xf6,
                                     0x00,
                                     0xa8, 0xd1, 0xf9, 0xd6, 0x03,
@@ -2822,7 +2833,11 @@ TEST(TestColumnReader, testTimestamp) {
 
   for (size_t i = 0; i < batch.numElements; ++i) {
     time_t time = static_cast<time_t>(longBatch->data[i]);
-    EXPECT_STREQ(expected[i], ctime(&time)) << "Wrong value at " << i;
+    tm timeStruct;
+    ASSERT_PRED1(isNotNull, gmtime_r(&time, &timeStruct));
+    char buffer[30];
+    asctime_r(&timeStruct, buffer);
+    EXPECT_STREQ(expected[i], buffer) << "Wrong value at " << i;
     EXPECT_EQ(expectedNano[i], longBatch->nanoseconds[i]);
   }
 }

http://git-wip-us.apache.org/repos/asf/orc/blob/6dce746c/c++/test/TestTimezone.cc
----------------------------------------------------------------------
diff --git a/c++/test/TestTimezone.cc b/c++/test/TestTimezone.cc
new file mode 100644
index 0000000..13a14ee
--- /dev/null
+++ b/c++/test/TestTimezone.cc
@@ -0,0 +1,361 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "Adaptor.hh"
+#include "Timezone.hh"
+#include "wrap/gtest-wrapper.h"
+
+#include <iostream>
+#include <unistd.h>
+#include <vector>
+
+namespace orc {
+
+  bool isLeap(int64_t year);
+
+  TEST(TestTimezone, isLeap) {
+    EXPECT_TRUE(isLeap(2000));
+    EXPECT_FALSE(isLeap(2001));
+    EXPECT_TRUE(isLeap(2004));
+    EXPECT_FALSE(isLeap(2100));
+    EXPECT_FALSE(isLeap(2200));
+    EXPECT_FALSE(isLeap(2300));
+    EXPECT_TRUE(isLeap(2400));
+  }
+
+  int64_t binarySearch(const std::vector<int64_t> &array, int64_t target);
+
+  TEST(TestTimezone, testBinarySearch) {
+    std::vector<int64_t> vect;
+    vect.push_back(0);
+    EXPECT_EQ(-1, binarySearch(vect, -5));
+    EXPECT_EQ(0, binarySearch(vect, 0));
+    EXPECT_EQ(0, binarySearch(vect, 5));
+    vect.push_back(2);
+    EXPECT_EQ(-1, binarySearch(vect, -1));
+    EXPECT_EQ(0, binarySearch(vect, 0));
+    EXPECT_EQ(0, binarySearch(vect, 1));
+    EXPECT_EQ(1, binarySearch(vect, 2));
+    EXPECT_EQ(1, binarySearch(vect, 3));
+  }
+
+  /**
+   * Parse a future rule string and return the parsed rule as a string.
+   */
+  std::string stringifyRule(const std::string& ruleString) {
+    std::unique_ptr<FutureRule> rule = parseFutureRule(ruleString);
+    std::stringstream buffer;
+    rule->print(buffer);
+    return buffer.str();
+  }
+
+
+  TEST(TestTimezone, parseFutureRule) {
+    EXPECT_EQ("  Future rule: FOO0\n  standard FOO 0\n",
+              stringifyRule("FOO0"));
+    EXPECT_EQ("  Future rule: <FOO+->010:02\n  standard <FOO+-> -36120\n",
+              stringifyRule("<FOO+->010:02"));
+    // unclosed '<'
+    EXPECT_THROW(stringifyRule("<FOO12"), TimezoneError);
+    // empty name
+    EXPECT_THROW(stringifyRule("+8"), TimezoneError);
+    // missing offset
+    EXPECT_THROW(stringifyRule("FOO"), TimezoneError);
+    EXPECT_EQ("  Future rule: FOO-123:45:67\n  standard FOO 445567\n",
+              stringifyRule("FOO-123:45:67"));
+    EXPECT_EQ("  Future rule: FOO+8\n  standard FOO -28800\n",
+              stringifyRule("FOO+8"));
+    EXPECT_EQ(("  Future rule: FOO8BAR,J10,20\n  standard FOO -28800\n"
+               "  dst BAR -25200 (dst)\n  start julian 10 at 2:0:0\n"
+               "  end day 20 at 2:0:0\n"),
+              stringifyRule("FOO8BAR,J10,20"));
+    EXPECT_EQ(("  Future rule: FOO+8BAR-0:30,M3.1.0,M10.5.6\n"
+               "  standard FOO -28800\n"
+               "  dst BAR 1800 (dst)\n"
+               "  start month 3 week 1 day 0 at 2:0:0\n"
+               "  end month 10 week 5 day 6 at 2:0:0\n"),
+              stringifyRule("FOO+8BAR-0:30,M3.1.0,M10.5.6"));
+    EXPECT_EQ(("  Future rule: FOO10BAR1,3,4\n"
+               "  standard FOO -36000\n"
+               "  dst BAR -3600 (dst)\n"
+               "  start day 3 at 2:0:0\n"
+               "  end day 4 at 2:0:0\n"),
+              stringifyRule("FOO10BAR1,3,4"));
+    // missing transition times
+    EXPECT_THROW(stringifyRule("FOO8BAR"), TimezoneError);
+    // check left over text
+    EXPECT_THROW(stringifyRule("FOO8BAR,10,20x"), TimezoneError);
+    EXPECT_EQ(("  Future rule: FOO8BAR,3/3,4/4:30\n"
+               "  standard FOO -28800\n"
+               "  dst BAR -25200 (dst)\n"
+               "  start day 3 at 3:0:0\n"
+               "  end day 4 at 4:30:0\n"),
+              stringifyRule("FOO8BAR,3/3,4/4:30"));
+    EXPECT_EQ(("  Future rule: FOO-8BAR,J3/3,M4.5.6/00:04:0007\n"
+               "  standard FOO 28800\n"
+               "  dst BAR 32400 (dst)\n"
+               "  start julian 3 at 3:0:0\n"
+               "  end month 4 week 5 day 6 at 0:4:7\n"),
+              stringifyRule("FOO-8BAR,J3/3,M4.5.6/00:04:0007"));
+    // too many fields in offset
+    EXPECT_THROW(stringifyRule("FOO8BAR,10,20/4:30:20:10"), TimezoneError);
+
+    EXPECT_FALSE(parseFutureRule("")->isDefined());
+    EXPECT_TRUE(parseFutureRule("FOO12")->isDefined());
+  }
+
+  const std::string& getZoneFromRule(FutureRule *rule,
+                                     const std::string& date) {
+    tm timeStruct;
+    if (strptime(date.c_str(), "%F %H:%M:%S", &timeStruct) == nullptr) {
+      throw TimezoneError("bad time " + date);
+    }
+    return rule->getVariant(timegm(&timeStruct)).name;
+  }
+
+  TEST(TestTimezone, useFutureRule) {
+    std::unique_ptr<FutureRule> rule =
+      parseFutureRule("FOO8BAR,M3.2.0,M11.1.0");
+    // 1970
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1970-01-01 00:00:00"));
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1970-03-08 09:59:59"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "1970-03-08 10:00:00"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "1970-04-01 00:00:00"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "1970-11-01 08:59:59"));
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1970-11-01 09:00:00"));
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1970-12-31 00:00:00"));
+
+    // 2369
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "2369-01-01 00:00:00"));
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "2369-03-09 09:59:59"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "2369-03-09 10:00:00"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "2369-11-02 08:59:59"));
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "2369-11-02 09:00:00"));
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "2369-12-31 00:00:00"));
+
+    // 2370
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "2370-01-01 00:00:00"));
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "2370-03-08 09:59:59"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "2370-03-08 10:00:00"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "2370-04-01 00:00:00"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "2370-11-01 08:59:59"));
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "2370-11-01 09:00:00"));
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "2370-12-31 00:00:00"));
+
+    rule = parseFutureRule("FOO8BAR,J10,J360");
+    //1970
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1970-01-11 09:59:59"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "1970-01-11 10:00:00"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "1970-12-27 08:59:59"));
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1970-12-27 09:00:00"));
+    //1972
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1972-01-11 09:59:59"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "1972-01-11 10:00:00"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "1972-12-27 08:59:59"));
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1972-12-27 09:00:00"));
+
+    rule = parseFutureRule("FOO8BAR,10,360");
+    #ifdef HAS_PRE_1970
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1969-01-11 09:59:59"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "1969-01-11 10:00:00"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "1969-12-27 08:59:59"));
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1969-12-27 09:00:00"));
+    #endif
+    //1970
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1970-01-11 09:59:59"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "1970-01-11 10:00:00"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "1970-12-27 08:59:59"));
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1970-12-27 09:00:00"));
+    //1972
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1972-01-11 09:59:59"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "1972-01-11 10:00:00"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "1972-12-26 08:59:59"));
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1972-12-26 09:00:00"));
+
+    // test a southern hemisphere timezone
+    rule = parseFutureRule("FOO8BAR,360,10");
+    //1970
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "1970-01-11 08:59:59"));
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1970-01-11 09:00:00"));
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1970-12-27 09:59:59"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "1970-12-27 10:00:00"));
+    //1972
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "1972-01-11 08:59:59"));
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1972-01-11 09:00:00"));
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1972-12-26 09:59:59"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "1972-12-26 10:00:00"));
+
+    rule = parseFutureRule("FOO8BAR,J10/3,J360/3:30");
+    //1970
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1970-01-11 10:59:59"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "1970-01-11 11:00:00"));
+    EXPECT_EQ("BAR", getZoneFromRule(rule.get(), "1970-12-27 10:29:59"));
+    EXPECT_EQ("FOO", getZoneFromRule(rule.get(), "1970-12-27 10:30:00"));
+  }
+
+  unsigned char decodeBase64Char(unsigned char ch) {
+    switch (ch >> 4) {
+    case 4: case 5: return static_cast<unsigned char>(ch - 'A');
+    case 6: case 7: return static_cast<unsigned char>(ch - 'a' + 26);
+    case 3: return static_cast<unsigned char>(ch - '0' + 52);
+    case 2: return static_cast<unsigned char>(ch == '+' ? 62 : 63);
+    default: return 255;
+    }
+  }
+
+  std::vector<unsigned char> decodeBase64(const std::string& input) {
+    std::vector<unsigned char> result;
+    result.reserve(3 * (input.size() / 4));
+    uint32_t accumulator = 0;
+    uint32_t bits = 0;
+    for(uint32_t c=0; c < input.size() && input[c] != '='; ++c) {
+      accumulator = (accumulator << 6) |
+        decodeBase64Char(static_cast<unsigned char>(input[c]));
+      bits += 6;
+      if (bits >= 8) {
+        bits -= 8;
+        result.push_back(static_cast<unsigned char>(accumulator >> bits));
+        accumulator &= ~(0xffffffff << bits);
+      }
+    }
+    return result;
+  }
+
+  static const char *LA_VER1 =
+    ("VFppZgAAAAAAAAAAAAAAAAAAAAAAAAAEAAAABAAAAAAAAAC5AAAABAAAABCepkig"
+"n7sVkKCGKqChmveQy4kaoNIj9HDSYSYQ1v50INiArZDa/tGg28CQENzes6DdqayQ3r6Vo"
+"N+JjpDgnneg4WlwkOJ+WaDjSVKQ5F47oOUpNJDmR1gg5xJREOgnOiDo8jMQ6gccIOrSFR"
+"Dr5v4g7LH3EO3G4CDukdkQ76/8oPBxuxDxj96g8n/BkPNvwKD0X6OQ9U+ioPY/hZD3L4S"
+"g+CiiEPkPZqD6CIQQ+viDIPvoZhD82GUg/chIEP64RyD/qCoQAJgpIAGIDBACeAsgA3Eo"
+"kARhJ6AFUQqQBkEJoAcw7JAHjUOgCRDOkAmtvyAK8LCQC+CvoAzZzRANwJGgDrmvEA+pr"
+"iAQmZEQEYmQIBJ5cxATaXIgFFlVEBVJVCAWOTcQFyk2IBgiU5AZCRggGgI1kBryNKAb4h"
+"eQHNIWoB3B+ZAesfigH6HbkCB2KyAhgb2QIlYNICNq2hAkNe8gJUq8ECYV0SAnKp4QJ/7"
+"toCkKgBAp3s+gKupiECu+saAs036QLZ6ToC6zYJAvfnWgMJNCkDFnkiAycySQM0d0IDRT"
+"BpA1J1YgNjLokDcHOCA4HAUQOOcaIDn75xA6xvwgO9vJEDywGKA9u6sQPo/6oD+bjRBAb"
+"9ygQYSpkEJPvqBDZIuQRC+goEVEbZBF89MgRy2KEEfTtSBJDWwQSbOXIErtThBLnLOgTN"
+"ZqkE18laBOtkyQT1x3oFCWLpBRPFmgUnYQkFMcO6BUVfKQVPwdoFY11JBW5TogWB7xEFj"
+"FHCBZ/tMQWqT+IFvetRBchOAgXb6XEF5kwiBfnnkQYE3eoGGHlZBiLcCgY2d3kGQNoqBl"
+"R1mQZe2EoGcnO5BnzWagaQcdkGmtSKBq5v+Qa5ZlIGzQHBBtdkcgbq/+EG9WKSBwj+AQc"
+"TYLIHJvwhBzFe0gdE+kEHT/CaB2OMCQdt7roHgYopB4vs2gefiEkHqer6B72GaQfH6RoH"
+"24SJB+XnOgf5gqkAABAAECAwEAAQABAAEAAQABAAEAAQABAAEAAQABAAEAAQABAAEAAQA"
+"BAAEAAQABAAEAAQABAAEAAQABAAEAAQABAAEAAQABAAEAAQABAAEAAQABAAEAAQABAAEA"
+"AQABAAEAAQABAAEAAQABAAEAAQABAAEAAQABAAEAAQABAAEAAQABAAEAAQABAAEAAQABA"
+"AEAAQABAAEAAQABAAEAAQABAAEAAQABAAEAAQABAAEAAQABAAEAAQAB//+dkAEA//+PgA"
+"AE//+dkAEI//+dkAEMUERUAFBTVABQV1QAUFBUAAAAAAEAAAAB");
+
+  static const char *LA_VER2 =
+    ("VFppZjIAAAAAAAAAAAAAAAAAAAAAAAAFAAAABQAAAAAAAAC6AAAABQAAABSAAAAAnqZIoJ"
+"+7FZCghiqgoZr3kMuJGqDSI/Rw0mEmENb+dCDYgK2Q2v7RoNvAkBDc3rOg3amskN6+laD"
+"fiY6Q4J53oOFpcJDiflmg40lSkOReO6DlKTSQ5kdYIOcSURDoJzog6PIzEOoHHCDq0hUQ"
+"6+b+IOyx9xDtxuAg7pHZEO+v/KDwcbsQ8Y/eoPJ/wZDzb8Cg9F+jkPVPoqD2P4WQ9y+Eo"
+"PgoohD5D2ag+giEEPr4gyD76GYQ/NhlIP3ISBD+uEcg/6gqEACYKSABiAwQAngLIANxKJ"
+"AEYSegBVEKkAZBCaAHMOyQB41DoAkQzpAJrb8gCvCwkAvgr6AM2c0QDcCRoA65rxAPqa4"
+"gEJmREBGJkCASeXMQE2lyIBRZVRAVSVQgFjk3EBcpNiAYIlOQGQkYIBoCNZAa8jSgG+IX"
+"kBzSFqAdwfmQHrH4oB+h25AgdisgIYG9kCJWDSAjatoQJDXvICVKvBAmFdEgJyqeECf+7"
+"aApCoAQKd7PoCrqYhArvrGgLNN+kC2ek6Aus2CQL351oDCTQpAxZ5IgMnMkkDNHdCA0Uw"
+"aQNSdWIDYy6JA3BzggOBwFEDjnGiA5++cQOsb8IDvbyRA8sBigPburED6P+qA/m40QQG/"
+"coEGEqZBCT76gQ2SLkEQvoKBFRG2QRfPTIEctihBH07UgSQ1sEEmzlyBK7U4QS5yzoEzW"
+"apBNfJWgTrZMkE9cd6BQli6QUTxZoFJ2EJBTHDugVFXykFT8HaBWNdSQVuU6IFge8RBYx"
+"RwgWf7TEFqk/iBb3rUQXITgIF2+lxBeZMIgX555EGBN3qBhh5WQYi3AoGNnd5BkDaKgZU"
+"dZkGXthKBnJzuQZ81moGkHHZBprUigaub/kGuWZSBs0BwQbXZHIG6v/hBvVikgcI/gEHE"
+"2CyByb8IQcxXtIHRPpBB0/wmgdjjAkHbe66B4GKKQeL7NoHn4hJB6nq+ge9hmkHx+kaB9"
+"uEiQfl5zoH+YKpACAQIBAgMEAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECA"
+"QIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAg"
+"ECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQI"
+"BAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQL//5EmAAD//52Q"
+"AQT//4+AAAj//52QAQz//52QARBMTVQAUERUAFBTVABQV1QAUFBUAAAAAAABAAAAAAFUW"
+"mlmMgAAAAAAAAAAAAAAAAAAAAAAAAUAAAAFAAAAAAAAALsAAAAFAAAAFPgAAAAAAAAA//"
+"///14EGsD/////nqZIoP////+fuxWQ/////6CGKqD/////oZr3kP/////LiRqg/////9I"
+"j9HD/////0mEmEP/////W/nQg/////9iArZD/////2v7RoP/////bwJAQ/////9zes6D/"
+"////3amskP/////evpWg/////9+JjpD/////4J53oP/////haXCQ/////+J+WaD/////4"
+"0lSkP/////kXjug/////+UpNJD/////5kdYIP/////nElEQ/////+gnOiD/////6PIzEP"
+"/////qBxwg/////+rSFRD/////6+b+IP/////ssfcQ/////+3G4CD/////7pHZEP/////"
+"vr/yg//////BxuxD/////8Y/eoP/////yf8GQ//////NvwKD/////9F+jkP/////1T6Kg"
+"//////Y/hZD/////9y+EoP/////4KKIQ//////kPZqD/////+giEEP/////6+IMg/////"
+"/voZhD//////NhlIP/////9yEgQ//////64RyD//////6gqEAAAAAAAmCkgAAAAAAGIDB"
+"AAAAAAAngLIAAAAAADcSiQAAAAAARhJ6AAAAAABVEKkAAAAAAGQQmgAAAAAAcw7JAAAAA"
+"AB41DoAAAAAAJEM6QAAAAAAmtvyAAAAAACvCwkAAAAAAL4K+gAAAAAAzZzRAAAAAADcCR"
+"oAAAAAAOua8QAAAAAA+priAAAAAAEJmREAAAAAARiZAgAAAAABJ5cxAAAAAAE2lyIAAAA"
+"AAUWVUQAAAAABVJVCAAAAAAFjk3EAAAAAAXKTYgAAAAABgiU5AAAAAAGQkYIAAAAAAaAj"
+"WQAAAAABryNKAAAAAAG+IXkAAAAAAc0hagAAAAAB3B+ZAAAAAAHrH4oAAAAAAfoduQAAA"
+"AACB2KyAAAAAAIYG9kAAAAAAiVg0gAAAAACNq2hAAAAAAJDXvIAAAAAAlSrwQAAAAACYV"
+"0SAAAAAAJyqeEAAAAAAn/u2gAAAAACkKgBAAAAAAKd7PoAAAAAAq6mIQAAAAACu+saAAA"
+"AAALNN+kAAAAAAtnpOgAAAAAC6zYJAAAAAAL351oAAAAAAwk0KQAAAAADFnkiAAAAAAMn"
+"MkkAAAAAAzR3QgAAAAADRTBpAAAAAANSdWIAAAAAA2MuiQAAAAADcHOCAAAAAAOBwFEAA"
+"AAAA45xogAAAAADn75xAAAAAAOsb8IAAAAAA728kQAAAAADywGKAAAAAAPburEAAAAAA+"
+"j/qgAAAAAD+bjRAAAAAAQG/coAAAAABBhKmQAAAAAEJPvqAAAAAAQ2SLkAAAAABEL6CgA"
+"AAAAEVEbZAAAAAARfPTIAAAAABHLYoQAAAAAEfTtSAAAAAASQ1sEAAAAABJs5cgAAAAAE"
+"rtThAAAAAAS5yzoAAAAABM1mqQAAAAAE18laAAAAAATrZMkAAAAABPXHegAAAAAFCWLpA"
+"AAAAAUTxZoAAAAABSdhCQAAAAAFMcO6AAAAAAVFXykAAAAABU/B2gAAAAAFY11JAAAAAA"
+"VuU6IAAAAABYHvEQAAAAAFjFHCAAAAAAWf7TEAAAAABapP4gAAAAAFvetRAAAAAAXITgI"
+"AAAAABdvpcQAAAAAF5kwiAAAAAAX555EAAAAABgTd6gAAAAAGGHlZAAAAAAYi3AoAAAAA"
+"BjZ3eQAAAAAGQNoqAAAAAAZUdZkAAAAABl7YSgAAAAAGcnO5AAAAAAZ81moAAAAABpBx2"
+"QAAAAAGmtSKAAAAAAaub/kAAAAABrlmUgAAAAAGzQHBAAAAAAbXZHIAAAAABur/4QAAAA"
+"AG9WKSAAAAAAcI/gEAAAAABxNgsgAAAAAHJvwhAAAAAAcxXtIAAAAAB0T6QQAAAAAHT/C"
+"aAAAAAAdjjAkAAAAAB23uugAAAAAHgYopAAAAAAeL7NoAAAAAB5+ISQAAAAAHqer6AAAA"
+"AAe9hmkAAAAAB8fpGgAAAAAH24SJAAAAAAfl5zoAAAAAB/mCqQAAIBAgECAwQCAQIBAgE"
+"CAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIB"
+"AgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECA"
+"QIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAgECAQIBAg"
+"ECAQIBAgECAQIBAgECAQIBAv//kSYAAP//nZABBP//j4AACP//nZABDP//nZABEExNVAB"
+"QRFQAUFNUAFBXVABQUFQAAAAAAAEAAAAAAQpQU1Q4UERULE0zLjIuMCxNMTEuMS4wCg==");
+
+  const std::string& getVariantFromZone(const Timezone& zone,
+                                        const std::string& date) {
+    tm timeStruct;
+    if (strptime(date.c_str(), "%F %H:%M:%S", &timeStruct) == nullptr) {
+      throw TimezoneError("bad time " + date);
+    }
+    return zone.getVariant(timegm(&timeStruct)).name;
+  }
+
+  TEST(TestTimezone, testParser) {
+    std::unique_ptr<Timezone> la = getTimezone("America/Los_Angeles",
+                                               decodeBase64(LA_VER1));
+    EXPECT_EQ(1, la->getVersion());
+    EXPECT_EQ("PST", getVariantFromZone(*la, "1974-01-06 09:59:59"));
+    EXPECT_EQ("PDT", getVariantFromZone(*la, "1974-01-06 10:00:00"));
+    // v1 won't have information past 2038
+    EXPECT_EQ("PST", getVariantFromZone(*la, "2100-03-14 09:59:59"));
+    EXPECT_EQ("PST", getVariantFromZone(*la, "2100-03-14 10:00:00"));
+
+    la = getTimezone("America/Los_Angeles", decodeBase64(LA_VER2));
+    EXPECT_EQ(2, la->getVersion());
+    EXPECT_EQ("PST", getVariantFromZone(*la, "1974-01-06 09:59:59"));
+    EXPECT_EQ("PDT", getVariantFromZone(*la, "1974-01-06 10:00:00"));
+    EXPECT_EQ("PST", getVariantFromZone(*la, "2100-03-14 09:59:59"));
+    EXPECT_EQ("PDT", getVariantFromZone(*la, "2100-03-14 10:00:00"));
+  }
+
+  TEST(TestTimezone, testZoneCache) {
+    const Timezone *la1 = &getTimezoneByName("America/Los_Angeles");
+    const Timezone *ny1 = &getTimezoneByName("America/New_York");
+    const Timezone *la2 = &getTimezoneByName("America/Los_Angeles");
+    const Timezone *ny2 = &getTimezoneByName("America/New_York");
+    EXPECT_EQ(la1, la2);
+    EXPECT_EQ(ny1, ny2);
+    EXPECT_EQ("PST", getVariantFromZone(*la1, "1974-01-06 09:59:59"));
+    EXPECT_EQ("PDT", getVariantFromZone(*la1, "1974-01-06 10:00:00"));
+    EXPECT_EQ("PDT", getVariantFromZone(*la1, "1974-10-27 08:59:59"));
+    EXPECT_EQ("PST", getVariantFromZone(*la1, "1974-10-27 09:00:00"));
+    EXPECT_EQ("EST", getVariantFromZone(*ny1, "1974-01-06 06:59:59"));
+    EXPECT_EQ("EDT", getVariantFromZone(*ny1, "1974-01-06 07:00:00"));
+    EXPECT_EQ("EDT", getVariantFromZone(*ny1, "1974-10-27 05:59:59"));
+    EXPECT_EQ("EST", getVariantFromZone(*ny1, "1974-10-27 06:00:00"));
+  }
+}  // namespace orc

http://git-wip-us.apache.org/repos/asf/orc/blob/6dce746c/docker/run-all.sh
----------------------------------------------------------------------
diff --git a/docker/run-all.sh b/docker/run-all.sh
index 0a34b9c..d124b3c 100755
--- a/docker/run-all.sh
+++ b/docker/run-all.sh
@@ -15,11 +15,14 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+REPOSITORY="https://github.com/apache/orc.git"
+BRANCH="master"
+
 start=`date`
 for os in centos5 centos6 centos7 debian6 debian7 ubuntu12 ubuntu14; do
   echo "Testing $os"
   ( cd $os && docker build -t "orc-$os" . )
-  docker run "orc-$os" || exit 1
+  docker run "orc-$os" /bin/bash -c "git clone $REPOSITORY -b $BRANCH && mkdir orc/build && cd orc/build && cmake .. && make package test-out" || exit 1
 done
 echo "Start: $start"
-echo "End:" `date`
\ No newline at end of file
+echo "End:" `date`

http://git-wip-us.apache.org/repos/asf/orc/blob/6dce746c/tools/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/tools/src/CMakeLists.txt b/tools/src/CMakeLists.txt
index 69cadd8..33f294e 100644
--- a/tools/src/CMakeLists.txt
+++ b/tools/src/CMakeLists.txt
@@ -18,7 +18,7 @@ include_directories (
   ${PROTOBUF_INCLUDE_DIRS}
   )
 
-set (CMAKE_CXX_FLAGS "-O0 ${CMAKE_CXX_FLAGS} -g ${CXX11_FLAGS} ${WARN_FLAGS}")
+set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -g ${CXX11_FLAGS} ${WARN_FLAGS}")
 
 add_executable (file-contents
   FileContents.cc