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 2015/07/06 23:52:30 UTC

[01/23] orc git commit: ORC-23. Simplify directory structure.

Repository: orc
Updated Branches:
  refs/heads/master 486433f56 -> 7f55b4537


http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools/test/ToolTest.cc
----------------------------------------------------------------------
diff --git a/tools/test/ToolTest.cc b/tools/test/ToolTest.cc
new file mode 100644
index 0000000..b4ddfbc
--- /dev/null
+++ b/tools/test/ToolTest.cc
@@ -0,0 +1,43 @@
+/**
+ * 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 "orc/orc-config.hh"
+#include "ToolTest.hh"
+
+#include "wrap/orc-proto-wrapper.hh"
+#include "wrap/gtest-wrapper.h"
+
+
+#include <iostream>
+
+const char *exampleDirectory = 0;
+
+GTEST_API_ int main(int argc, char **argv) {
+  GOOGLE_PROTOBUF_VERIFY_VERSION;
+  std::cout << "ORC version: " << ORC_VERSION << "\n";
+  if (argc == 2) {
+    exampleDirectory = argv[1];
+  } else {
+    exampleDirectory = "../examples";
+  }
+  std::cout << "example dir = " << exampleDirectory << "\n";
+  testing::InitGoogleTest(&argc, argv);
+  int result = RUN_ALL_TESTS();
+  return result;
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools/test/ToolTest.hh
----------------------------------------------------------------------
diff --git a/tools/test/ToolTest.hh b/tools/test/ToolTest.hh
new file mode 100644
index 0000000..48720b0
--- /dev/null
+++ b/tools/test/ToolTest.hh
@@ -0,0 +1,21 @@
+/**
+ * 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 <string>
+
+extern const char* exampleDirectory;

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools/test/gzip.cc
----------------------------------------------------------------------
diff --git a/tools/test/gzip.cc b/tools/test/gzip.cc
new file mode 100644
index 0000000..462eca3
--- /dev/null
+++ b/tools/test/gzip.cc
@@ -0,0 +1,115 @@
+/**
+ * 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 "gzip.hh"
+
+#include <iostream>
+#include <stdexcept>
+
+#ifdef __clang__
+#pragma clang diagnostic ignored "-Wold-style-cast"
+#endif
+
+namespace orc {
+
+  GzipTextReader::GzipTextReader(const std::string& _filename
+                                 ): filename(_filename) {
+    file = fopen(filename.c_str(), "rb");
+    if (file == nullptr) {
+      throw std::runtime_error("can't open " + filename);
+    }
+    stream.zalloc = Z_NULL;
+    stream.zfree = Z_NULL;
+    stream.opaque = Z_NULL;
+    stream.avail_in = 0;
+    stream.avail_out = 1;
+    stream.next_in = Z_NULL;
+    int ret = inflateInit2(&stream, 16+MAX_WBITS);
+    if (ret != Z_OK) {
+      throw std::runtime_error("zlib failed initialization for " + filename);
+    }
+    outPtr = nullptr;
+    outEnd = nullptr;
+    isDone = false;
+  }
+
+  bool GzipTextReader::nextBuffer() {
+    // if we are done, return
+    if (isDone) {
+      return false;
+    }
+    // if the last read is done, read more
+    if (stream.avail_in == 0 && stream.avail_out != 0) {
+      stream.next_in = input;
+      stream.avail_in = static_cast<uint>(fread(input, 1, sizeof(input),
+                                                file));
+      if (ferror(file)) {
+        throw std::runtime_error("failure reading " + filename);
+      }
+    }
+    stream.avail_out = sizeof(output);
+    stream.next_out = output;
+    int ret = inflate(&stream, Z_NO_FLUSH);
+    switch (ret) {
+    case Z_OK:
+      break;
+    case Z_STREAM_END:
+      isDone = true;
+      break;
+    case Z_STREAM_ERROR:
+      throw std::runtime_error("zlib stream problem");
+    case Z_NEED_DICT:
+    case Z_DATA_ERROR:
+      throw std::runtime_error("zlib data problem");
+    case Z_MEM_ERROR:
+      throw std::runtime_error("zlib memory problem");
+    case Z_BUF_ERROR:
+      throw std::runtime_error("zlib buffer problem");
+    default:
+      throw std::runtime_error("zlib unknown problem");
+    }
+    outPtr = output;
+    outEnd = output + (sizeof(output) - stream.avail_out);
+    return true;
+  }
+
+  bool GzipTextReader::nextLine(std::string& line) {
+    bool result = false;
+    line.clear();
+    while (true) {
+      if (outPtr == outEnd) {
+        if (!nextBuffer()) {
+          return result;
+        }
+      }
+      unsigned char ch = *(outPtr++);
+      if (ch == '\n') {
+        return true;
+      }
+      line += static_cast<char>(ch);
+    }
+  }
+
+  GzipTextReader::~GzipTextReader() {
+    inflateEnd(&stream);
+    if (fclose(file) != 0) {
+      std::cerr << "can't close file " << filename;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools/test/gzip.hh
----------------------------------------------------------------------
diff --git a/tools/test/gzip.hh b/tools/test/gzip.hh
new file mode 100644
index 0000000..4efebed
--- /dev/null
+++ b/tools/test/gzip.hh
@@ -0,0 +1,52 @@
+/**
+ * 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 ORC_GZIP
+#define ORC_GZIP
+
+#include <string>
+#include <stdio.h>
+#include "zlib.h"
+
+namespace orc {
+
+  class GzipTextReader {
+  private:
+    std::string filename;
+    FILE* file;
+    z_stream stream;
+    unsigned char input[64 * 1024];
+    unsigned char output[64 * 1024];
+    unsigned char *outPtr;
+    unsigned char *outEnd;
+    bool isDone;
+
+    bool nextBuffer();
+
+    // NOT IMPLEMENTED
+    GzipTextReader(const GzipTextReader&);
+    GzipTextReader& operator=(const GzipTextReader&);
+
+  public:
+    GzipTextReader(const std::string& filename);
+    ~GzipTextReader();
+    bool nextLine(std::string& line);
+  };
+
+}
+#endif


[19/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/ByteRLE.cc
----------------------------------------------------------------------
diff --git a/c++/src/orc/ByteRLE.cc b/c++/src/orc/ByteRLE.cc
deleted file mode 100644
index 2c0032c..0000000
--- a/c++/src/orc/ByteRLE.cc
+++ /dev/null
@@ -1,352 +0,0 @@
-/**
- * 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 <algorithm>
-#include <iostream>
-#include <string.h>
-#include <utility>
-
-#include "ByteRLE.hh"
-#include "Exceptions.hh"
-
-namespace orc {
-
-  const size_t MINIMUM_REPEAT = 3;
-
-  ByteRleDecoder::~ByteRleDecoder() {
-    // PASS
-  }
-
-  class ByteRleDecoderImpl: public ByteRleDecoder {
-  public:
-    ByteRleDecoderImpl(std::unique_ptr<SeekableInputStream> input);
-
-    virtual ~ByteRleDecoderImpl();
-
-    /**
-     * Seek to a particular spot.
-     */
-    virtual void seek(PositionProvider&);
-
-    /**
-     * Seek over a given number of values.
-     */
-    virtual void skip(uint64_t numValues);
-
-    /**
-     * Read a number of values into the batch.
-     */
-    virtual void next(char* data, uint64_t numValues, char* notNull);
-
-  protected:
-    inline void nextBuffer();
-    inline signed char readByte();
-    inline void readHeader();
-
-    std::unique_ptr<SeekableInputStream> inputStream;
-    size_t remainingValues;
-    char value;
-    const char* bufferStart;
-    const char* bufferEnd;
-    bool repeating;
-  };
-
-  void ByteRleDecoderImpl::nextBuffer() {
-    int bufferLength;
-    const void* bufferPointer;
-    bool result = inputStream->Next(&bufferPointer, &bufferLength);
-    if (!result) {
-      throw ParseError("bad read in nextBuffer");
-    }
-    bufferStart = static_cast<const char*>(bufferPointer);
-    bufferEnd = bufferStart + bufferLength;
-  }
-
-  signed char ByteRleDecoderImpl::readByte() {
-    if (bufferStart == bufferEnd) {
-      nextBuffer();
-    }
-    return *(bufferStart++);
-  }
-
-  void ByteRleDecoderImpl::readHeader() {
-    signed char ch = readByte();
-    if (ch < 0) {
-      remainingValues = static_cast<size_t>(-ch);
-      repeating = false;
-    } else {
-      remainingValues = static_cast<size_t>(ch) + MINIMUM_REPEAT;
-      repeating = true;
-      value = readByte();
-    }
-  }
-
-  ByteRleDecoderImpl::ByteRleDecoderImpl(std::unique_ptr<SeekableInputStream>
-                                         input) {
-    inputStream = std::move(input);
-    repeating = false;
-    remainingValues = 0;
-    value = 0;
-    bufferStart = 0;
-    bufferEnd = 0;
-  }
-
-  ByteRleDecoderImpl::~ByteRleDecoderImpl() {
-    // PASS
-  }
-
-  void ByteRleDecoderImpl::seek(PositionProvider& location) {
-    // move the input stream
-    inputStream->seek(location);
-    // force a re-read from the stream
-    bufferEnd = bufferStart;
-    // read a new header
-    readHeader();
-    // skip ahead the given number of records
-    skip(location.next());
-  }
-
-  void ByteRleDecoderImpl::skip(uint64_t numValues) {
-    while (numValues > 0) {
-      if (remainingValues == 0) {
-        readHeader();
-      }
-      size_t count = std::min(static_cast<size_t>(numValues), remainingValues);
-      remainingValues -= count;
-      numValues -= count;
-      // for literals we need to skip over count bytes, which may involve
-      // reading from the underlying stream
-      if (!repeating) {
-        size_t consumedBytes = count;
-        while (consumedBytes > 0) {
-          if (bufferStart == bufferEnd) {
-            nextBuffer();
-          }
-          size_t skipSize = std::min(static_cast<size_t>(consumedBytes),
-                                     static_cast<size_t>(bufferEnd -
-                                                         bufferStart));
-          bufferStart += skipSize;
-          consumedBytes -= skipSize;
-        }
-      }
-    }
-  }
-
-  void ByteRleDecoderImpl::next(char* data, uint64_t numValues,
-                                char* notNull) {
-    uint64_t position = 0;
-    // skip over null values
-    while (notNull && position < numValues && !notNull[position]) {
-      position += 1;
-    }
-    while (position < numValues) {
-      // if we are out of values, read more
-      if (remainingValues == 0) {
-        readHeader();
-      }
-      // how many do we read out of this block?
-      size_t count = std::min(static_cast<size_t>(numValues - position),
-                              remainingValues);
-      uint64_t consumed = 0;
-      if (repeating) {
-        if (notNull) {
-          for(uint64_t i=0; i < count; ++i) {
-            if (notNull[position + i]) {
-              data[position + i] = value;
-              consumed += 1;
-            }
-          }
-        } else {
-          memset(data + position, value, count);
-          consumed = count;
-        }
-      } else {
-        if (notNull) {
-          for(uint64_t i=0; i < count; ++i) {
-            if (notNull[position + i]) {
-              data[position + i] = readByte();
-              consumed += 1;
-            }
-          }
-        } else {
-          uint64_t i = 0;
-          while (i < count) {
-            if (bufferStart == bufferEnd) {
-              nextBuffer();
-            }
-            uint64_t copyBytes =
-              std::min(static_cast<uint64_t>(count - i),
-                       static_cast<uint64_t>(bufferEnd - bufferStart));
-            memcpy(data + position + i, bufferStart, copyBytes);
-            bufferStart += copyBytes;
-            i += copyBytes;
-          }
-          consumed = count;
-        }
-      }
-      remainingValues -= consumed;
-      position += count;
-      // skip over any null values
-      while (notNull && position < numValues && !notNull[position]) {
-        position += 1;
-      }
-    }
-  }
-
-  std::unique_ptr<ByteRleDecoder> createByteRleDecoder
-                                 (std::unique_ptr<SeekableInputStream> input) {
-    return std::unique_ptr<ByteRleDecoder>(new ByteRleDecoderImpl
-                                           (std::move(input)));
-  }
-
-  class BooleanRleDecoderImpl: public ByteRleDecoderImpl {
-  public:
-    BooleanRleDecoderImpl(std::unique_ptr<SeekableInputStream> input);
-
-    virtual ~BooleanRleDecoderImpl();
-
-    /**
-     * Seek to a particular spot.
-     */
-    virtual void seek(PositionProvider&);
-
-    /**
-     * Seek over a given number of values.
-     */
-    virtual void skip(uint64_t numValues);
-
-    /**
-     * Read a number of values into the batch.
-     */
-    virtual void next(char* data, uint64_t numValues, char* notNull);
-
-  protected:
-    size_t remainingBits;
-    char lastByte;
-  };
-
-  BooleanRleDecoderImpl::BooleanRleDecoderImpl
-                                (std::unique_ptr<SeekableInputStream> input
-                                 ): ByteRleDecoderImpl(std::move(input)) {
-    remainingBits = 0;
-    lastByte = 0;
-  }
-
-  BooleanRleDecoderImpl::~BooleanRleDecoderImpl() {
-    // PASS
-  }
-
-  void BooleanRleDecoderImpl::seek(PositionProvider& location) {
-    ByteRleDecoderImpl::seek(location);
-    uint64_t consumed = location.next();
-    if (consumed > 8) {
-      throw ParseError("bad position");
-    }
-    if (consumed != 0) {
-      remainingBits = 8 - consumed;
-      ByteRleDecoderImpl::next(&lastByte, 1, 0);
-    }
-  }
-
-  void BooleanRleDecoderImpl::skip(uint64_t numValues) {
-    if (numValues <= remainingBits) {
-      remainingBits -= numValues;
-    } else {
-      numValues -= remainingBits;
-      uint64_t bytesSkipped = numValues / 8;
-      ByteRleDecoderImpl::skip(bytesSkipped);
-      ByteRleDecoderImpl::next(&lastByte, 1, 0);
-      remainingBits = 8 - (numValues % 8);
-    }
-  }
-
-  void BooleanRleDecoderImpl::next(char* data, uint64_t numValues,
-                                   char* notNull) {
-    // next spot to fill in
-    uint64_t position = 0;
-
-    // use up any remaining bits
-    if (notNull) {
-      while(remainingBits > 0 && position < numValues) {
-        if (notNull[position]) {
-          remainingBits -= 1;
-          data[position] = (static_cast<unsigned char>(lastByte) >>
-                            remainingBits) & 0x1;
-        } else {
-          data[position] = 0;
-        }
-        position += 1;
-      }
-    } else {
-      while(remainingBits > 0 && position < numValues) {
-        remainingBits -= 1;
-        data[position++] = (static_cast<unsigned char>(lastByte) >>
-                            remainingBits) & 0x1;
-      }
-    }
-
-    // count the number of nonNulls remaining
-    uint64_t nonNulls = numValues - position;
-    if (notNull) {
-      for(uint64_t i=position; i < numValues; ++i) {
-        if (!notNull[i]) {
-          nonNulls -= 1;
-        }
-      }
-    }
-
-    // fill in the remaining values
-    if (nonNulls == 0) {
-      while (position < numValues) {
-        data[position++] = 0;
-      }
-    } else if (position < numValues) {
-      // read the new bytes into the array
-      uint64_t bytesRead = (nonNulls + 7) / 8;
-      ByteRleDecoderImpl::next(data + position, bytesRead, 0);
-      lastByte = data[position + bytesRead - 1];
-      remainingBits = bytesRead * 8 - nonNulls;
-      // expand the array backwards so that we don't clobber the data
-      uint64_t bitsLeft = bytesRead * 8 - remainingBits;
-      if (notNull) {
-        for(int64_t i=static_cast<int64_t>(numValues) - 1;
-            i >= static_cast<int64_t>(position); --i) {
-          if (notNull[i]) {
-            uint64_t shiftPosn = (-bitsLeft) % 8;
-            data[i] = (data[position + (bitsLeft - 1) / 8] >> shiftPosn) & 0x1;
-            bitsLeft -= 1;
-          } else {
-            data[i] = 0;
-          }
-        }
-      } else {
-        for(int64_t i=static_cast<int64_t>(numValues) - 1;
-            i >= static_cast<int64_t>(position); --i, --bitsLeft) {
-          uint64_t shiftPosn = (-bitsLeft) % 8;
-          data[i] = (data[position + (bitsLeft - 1) / 8] >> shiftPosn) & 0x1;
-        }
-      }
-    }
-  }
-
-  std::unique_ptr<ByteRleDecoder> createBooleanRleDecoder
-                                 (std::unique_ptr<SeekableInputStream> input) {
-    BooleanRleDecoderImpl* decoder = new BooleanRleDecoderImpl(std::move(input)) ;
-    return std::unique_ptr<ByteRleDecoder>(reinterpret_cast<ByteRleDecoder*>(decoder));
-  }
-}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/ByteRLE.hh
----------------------------------------------------------------------
diff --git a/c++/src/orc/ByteRLE.hh b/c++/src/orc/ByteRLE.hh
deleted file mode 100644
index 6762cb5..0000000
--- a/c++/src/orc/ByteRLE.hh
+++ /dev/null
@@ -1,71 +0,0 @@
-/**
- * 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 ORC_BYTE_RLE_HH
-#define ORC_BYTE_RLE_HH
-
-#include <memory>
-
-#include "Compression.hh"
-
-namespace orc {
-
-  class ByteRleDecoder {
-  public:
-    virtual ~ByteRleDecoder();
-
-    /**
-     * Seek to a particular spot.
-     */
-    virtual void seek(PositionProvider&) = 0;
-
-    /**
-     * Seek over a given number of values.
-     */
-    virtual void skip(uint64_t numValues) = 0;
-
-    /**
-     * Read a number of values into the batch.
-     * @param data the array to read into
-     * @param numValues the number of values to read
-     * @param notNull If the pointer is null, all values are read. If the
-     *    pointer is not null, positions that are false are skipped.
-     */
-    virtual void next(char* data, uint64_t numValues, char* notNull) = 0;
-  };
-
-  /**
-   * Create a byte RLE decoder.
-   * @param input the input stream to read from
-   */
-  std::unique_ptr<ByteRleDecoder> createByteRleDecoder
-                                 (std::unique_ptr<SeekableInputStream> input);
-
-  /**
-   * Create a boolean RLE decoder.
-   *
-   * Unlike the other RLE decoders, the boolean decoder sets the data to 0
-   * if the value is masked by notNull. This is required for the notNull stream
-   * processing to properly apply multiple masks from nested types.
-   * @param input the input stream to read from
-   */
-  std::unique_ptr<ByteRleDecoder> createBooleanRleDecoder
-                                 (std::unique_ptr<SeekableInputStream> input);
-}
-
-#endif

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/C09Adapter.cc
----------------------------------------------------------------------
diff --git a/c++/src/orc/C09Adapter.cc b/c++/src/orc/C09Adapter.cc
deleted file mode 100644
index 8afc752..0000000
--- a/c++/src/orc/C09Adapter.cc
+++ /dev/null
@@ -1,28 +0,0 @@
-/**
-* 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 "orc/C09Adapter.hh"
-#include <sstream>
-
-int64_t std::stoll(std::string str) {
-  int64_t val = 0;
-  stringstream ss ;
-  ss << str ;
-  ss >> val ;
-  return val;
-}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/ColumnPrinter.cc
----------------------------------------------------------------------
diff --git a/c++/src/orc/ColumnPrinter.cc b/c++/src/orc/ColumnPrinter.cc
deleted file mode 100644
index 3fd95e5..0000000
--- a/c++/src/orc/ColumnPrinter.cc
+++ /dev/null
@@ -1,724 +0,0 @@
-/**
- * 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 "orc/Adaptor.hh"
-#include "orc/ColumnPrinter.hh"
-#include "orc/orc-config.hh"
-
-#include <limits>
-#include <sstream>
-#include <stdexcept>
-#include <time.h>
-#include <typeinfo>
-
-#ifdef __clang__
-  #pragma clang diagnostic ignored "-Wformat-security"
-#endif
-
-namespace orc {
-
-  class BooleanColumnPrinter: public ColumnPrinter {
-  private:
-    const int64_t* data;
-  public:
-    BooleanColumnPrinter(std::string&, const Type&);
-    ~BooleanColumnPrinter() {}
-    void printRow(uint64_t rowId) override;
-    void reset(const ColumnVectorBatch& batch) override;
-  };
-
-  class LongColumnPrinter: public ColumnPrinter {
-  private:
-    const int64_t* data;
-  public:
-    LongColumnPrinter(std::string&, const Type&);
-    ~LongColumnPrinter() {}
-    void printRow(uint64_t rowId) override;
-    void reset(const ColumnVectorBatch& batch) override;
-  };
-
-  class DoubleColumnPrinter: public ColumnPrinter {
-  private:
-    const double* data;
-    const bool isFloat;
-
-  public:
-    DoubleColumnPrinter(std::string&, const Type&);
-    virtual ~DoubleColumnPrinter() {}
-    void printRow(uint64_t rowId) override;
-    void reset(const ColumnVectorBatch& batch) override;
-  };
-
-  class TimestampColumnPrinter: public ColumnPrinter {
-  private:
-    const int64_t* data;
-    time_t epoch;
-
-  public:
-    TimestampColumnPrinter(std::string&, const Type&);
-    ~TimestampColumnPrinter() {}
-    void printRow(uint64_t rowId) override;
-    void reset(const ColumnVectorBatch& batch) override;
-  };
-
-  class DateColumnPrinter: public ColumnPrinter {
-  private:
-    const int64_t* data;
-
-  public:
-    DateColumnPrinter(std::string&, const Type& type);
-    ~DateColumnPrinter() {}
-    void printRow(uint64_t rowId) override;
-    void reset(const ColumnVectorBatch& batch) override;
-  };
-
-  class Decimal64ColumnPrinter: public ColumnPrinter {
-  private:
-    const int64_t* data;
-    int32_t scale;
-  public:
-    Decimal64ColumnPrinter(std::string&, const Type& type);
-    ~Decimal64ColumnPrinter() {}
-    void printRow(uint64_t rowId) override;
-    void reset(const ColumnVectorBatch& batch) override;
-  };
-
-  class Decimal128ColumnPrinter: public ColumnPrinter {
-  private:
-    const Int128* data;
-    int32_t scale;
-  public:
-    Decimal128ColumnPrinter(std::string&, const Type& type);
-    ~Decimal128ColumnPrinter() {}
-    void printRow(uint64_t rowId) override;
-    void reset(const ColumnVectorBatch& batch) override;
-  };
-
-  class StringColumnPrinter: public ColumnPrinter {
-  private:
-    const char* const * start;
-    const int64_t* length;
-  public:
-    StringColumnPrinter(std::string&, const Type& type);
-    virtual ~StringColumnPrinter() {}
-    void printRow(uint64_t rowId) override;
-    void reset(const ColumnVectorBatch& batch) override;
-  };
-
-  class BinaryColumnPrinter: public ColumnPrinter {
-  private:
-    const char* const * start;
-    const int64_t* length;
-  public:
-    BinaryColumnPrinter(std::string&, const Type& type);
-    virtual ~BinaryColumnPrinter() {}
-    void printRow(uint64_t rowId) override;
-    void reset(const ColumnVectorBatch& batch) override;
-  };
-
-  class ListColumnPrinter: public ColumnPrinter {
-  private:
-    const int64_t* offsets;
-    std::unique_ptr<ColumnPrinter> elementPrinter;
-
-  public:
-    ListColumnPrinter(std::string&, const Type& type);
-    virtual ~ListColumnPrinter() {}
-    void printRow(uint64_t rowId) override;
-    void reset(const ColumnVectorBatch& batch) override;
-  };
-
-  class MapColumnPrinter: public ColumnPrinter {
-  private:
-    const int64_t* offsets;
-    std::unique_ptr<ColumnPrinter> keyPrinter;
-    std::unique_ptr<ColumnPrinter> elementPrinter;
-
-  public:
-    MapColumnPrinter(std::string&, const Type& type);
-    virtual ~MapColumnPrinter() {}
-    void printRow(uint64_t rowId) override;
-    void reset(const ColumnVectorBatch& batch) override;
-  };
-
-  class UnionColumnPrinter: public ColumnPrinter {
-  private:
-    const unsigned char *tags;
-    const uint64_t* offsets;
-    std::vector<ColumnPrinter*> fieldPrinter;
-
-  public:
-    UnionColumnPrinter(std::string&, const Type& type);
-    virtual ~UnionColumnPrinter();
-    void printRow(uint64_t rowId) override;
-    void reset(const ColumnVectorBatch& batch) override;
-  };
-
-  class StructColumnPrinter: public ColumnPrinter {
-  private:
-    std::vector<ColumnPrinter*> fieldPrinter;
-  public:
-    StructColumnPrinter(std::string&, const Type& type);
-    virtual ~StructColumnPrinter();
-    void printRow(uint64_t rowId) override;
-    void reset(const ColumnVectorBatch& batch) override;
-  };
-
-  void writeChar(std::string& file, char ch) {
-    file += ch;
-  }
-
-  void writeString(std::string& file, const char *ptr) {
-    size_t len = strlen(ptr);
-    file.append(ptr, len);
-  }
-
-  ColumnPrinter::ColumnPrinter(std::string& _buffer, const Type& _type
-                               ): buffer(_buffer),
-                                  type(_type) {
-    notNull = nullptr;
-    hasNulls = false;
-  }
-
-  ColumnPrinter::~ColumnPrinter() {
-    // PASS
-  }
-
-  void ColumnPrinter::reset(const ColumnVectorBatch& batch) {
-    hasNulls = batch.hasNulls;
-    if (hasNulls) {
-      notNull = batch.notNull.data();
-    } else {
-      notNull = nullptr ;
-    }
-  }
-
-  std::unique_ptr<ColumnPrinter> createColumnPrinter(std::string& buffer,
-                                                     const Type& type) {
-    ColumnPrinter *result;
-    switch(static_cast<int64_t>(type.getKind())) {
-    case BOOLEAN:
-      result = new BooleanColumnPrinter(buffer, type);
-      break;
-
-    case BYTE:
-    case SHORT:
-    case INT:
-    case LONG:
-      result = new LongColumnPrinter(buffer, type);
-      break;
-
-    case FLOAT:
-    case DOUBLE:
-      result = new DoubleColumnPrinter(buffer, type);
-      break;
-
-    case STRING:
-    case VARCHAR :
-    case CHAR:
-      result = new StringColumnPrinter(buffer, type);
-      break;
-
-    case BINARY:
-      result = new BinaryColumnPrinter(buffer, type);
-      break;
-
-    case TIMESTAMP:
-      result = new TimestampColumnPrinter(buffer, type);
-      break;
-
-    case LIST:
-      result = new ListColumnPrinter(buffer, type);
-      break;
-
-    case MAP:
-      result = new MapColumnPrinter(buffer, type);
-      break;
-
-    case STRUCT:
-      result = new StructColumnPrinter(buffer, type);
-      break;
-
-    case DECIMAL:
-      if (type.getPrecision() == 0 || type.getPrecision() > 18) {
-        result = new Decimal128ColumnPrinter(buffer, type);
-      } else {
-        result = new Decimal64ColumnPrinter(buffer, type);
-      }
-      break;
-
-    case DATE:
-      result = new DateColumnPrinter(buffer, type);
-      break;
-
-    case UNION:
-      result = new UnionColumnPrinter(buffer, type);
-      break;
-
-    default:
-      throw std::logic_error("unknown batch type");
-    }
-    return std::unique_ptr<ColumnPrinter>(result);
-  }
-
-  LongColumnPrinter::LongColumnPrinter(std::string& buffer,
-                                       const Type& type
-                                       ): ColumnPrinter(buffer, type) {
-    // pass
-  }
-
-  void LongColumnPrinter::reset(const  ColumnVectorBatch& batch) {
-    ColumnPrinter::reset(batch);
-    data = dynamic_cast<const LongVectorBatch&>(batch).data.data();
-  }
-
-  void LongColumnPrinter::printRow(uint64_t rowId) {
-    if (hasNulls && !notNull[rowId]) {
-      writeString(buffer, "null");
-    } else {
-      char numBuffer[64];
-      snprintf(numBuffer, sizeof(numBuffer), "%" INT64_FORMAT_STRING "d",
-               static_cast<int64_t >(data[rowId]));
-      writeString(buffer, numBuffer);
-    }
-  }
-
-  DoubleColumnPrinter::DoubleColumnPrinter(std::string& buffer,
-                                           const Type& type
-                                           ): ColumnPrinter(buffer, type),
-                                              isFloat(type.getKind() == FLOAT){
-    // PASS
-  }
-
-  void DoubleColumnPrinter::reset(const  ColumnVectorBatch& batch) {
-    ColumnPrinter::reset(batch);
-    data = dynamic_cast<const DoubleVectorBatch&>(batch).data.data();
-  }
-
-  void DoubleColumnPrinter::printRow(uint64_t rowId) {
-    if (hasNulls && !notNull[rowId]) {
-      writeString(buffer, "null");
-    } else {
-      char numBuffer[64];
-      snprintf(numBuffer, sizeof(numBuffer), isFloat ? "%.7g" : "%.14g",
-               data[rowId]);
-      writeString(buffer, numBuffer);
-    }
-  }
-
-  Decimal64ColumnPrinter::Decimal64ColumnPrinter(std::string& buffer,
-                                                 const  Type& type
-                                                 ): ColumnPrinter(buffer,
-                                                                  type) {
-    // PASS
-  }
-
-  void Decimal64ColumnPrinter::reset(const  ColumnVectorBatch& batch) {
-    ColumnPrinter::reset(batch);
-    data = dynamic_cast<const Decimal64VectorBatch&>(batch).values.data();
-    scale = dynamic_cast<const Decimal64VectorBatch&>(batch).scale;
-  }
-
-  std::string toDecimalString(int64_t value, int32_t scale) {
-    std::stringstream buffer;
-    if (scale == 0) {
-      buffer << value;
-      return buffer.str();
-    }
-    std::string sign = "";
-    if (value < 0) {
-      sign = "-";
-      value = -value;
-    }
-    buffer << value;
-    std::string str = buffer.str();
-    int32_t len = static_cast<int32_t>(str.length());
-    if (len > scale) {
-      return sign + str.substr(0, static_cast<size_t>(len - scale)) + "." +
-        str.substr(static_cast<size_t>(len - scale),
-                   static_cast<size_t>(scale));
-    } else if (len == scale) {
-      return sign + "0." + str;
-    } else {
-      std::string result = sign + "0.";
-      for(int32_t i=0; i < scale - len; ++i) {
-        result += "0";
-      }
-      return result + str;
-    }
-  }
-
-  void Decimal64ColumnPrinter::printRow(uint64_t rowId) {
-    if (hasNulls && !notNull[rowId]) {
-      writeString(buffer, "null");
-    } else {
-      writeString(buffer, toDecimalString(data[rowId], scale).c_str());
-    }
-  }
-
-  Decimal128ColumnPrinter::Decimal128ColumnPrinter(std::string& buffer,
-                                                   const Type& type
-                                                   ): ColumnPrinter(buffer,
-                                                                    type) {
-     // PASS
-   }
-
-   void Decimal128ColumnPrinter::reset(const  ColumnVectorBatch& batch) {
-     ColumnPrinter::reset(batch);
-     data = dynamic_cast<const Decimal128VectorBatch&>(batch).values.data();
-     scale =dynamic_cast<const Decimal128VectorBatch&>(batch).scale;
-   }
-
-   void Decimal128ColumnPrinter::printRow(uint64_t rowId) {
-     if (hasNulls && !notNull[rowId]) {
-       writeString(buffer, "null");
-     } else {
-       writeString(buffer, data[rowId].toDecimalString(scale).c_str());
-     }
-   }
-
-  StringColumnPrinter::StringColumnPrinter(std::string& buffer,
-                                           const Type& type
-                                           ): ColumnPrinter(buffer, type) {
-    // PASS
-  }
-
-  void StringColumnPrinter::reset(const ColumnVectorBatch& batch) {
-    ColumnPrinter::reset(batch);
-    start = dynamic_cast<const StringVectorBatch&>(batch).data.data();
-    length = dynamic_cast<const StringVectorBatch&>(batch).length.data();
-  }
-
-  void StringColumnPrinter::printRow(uint64_t rowId) {
-    if (hasNulls && !notNull[rowId]) {
-      writeString(buffer, "null");
-    } else {
-      writeChar(buffer, '"');
-      for(int64_t i=0; i < length[rowId]; ++i) {
-        char ch = static_cast<char>(start[rowId][i]);
-        switch (ch) {
-        case '\\':
-          writeString(buffer, "\\\\");
-          break;
-        case '\b':
-          writeString(buffer, "\\b");
-          break;
-        case '\f':
-          writeString(buffer, "\\f");
-          break;
-        case '\n':
-          writeString(buffer, "\\n");
-          break;
-        case '\r':
-          writeString(buffer, "\\r");
-          break;
-        case '\t':
-          writeString(buffer, "\\t");
-          break;
-        case '"':
-          writeString(buffer, "\\\"");
-          break;
-        default:
-          writeChar(buffer, ch);
-          break;
-        }
-      }
-      writeChar(buffer, '"');
-    }
-  }
-
-  ListColumnPrinter::ListColumnPrinter(std::string& buffer,
-                                       const Type& type
-                                       ): ColumnPrinter(buffer, type) {
-    elementPrinter = createColumnPrinter(buffer, type.getSubtype(0));
-  }
-
-  void ListColumnPrinter::reset(const  ColumnVectorBatch& batch) {
-    ColumnPrinter::reset(batch);
-    offsets = dynamic_cast<const ListVectorBatch&>(batch).offsets.data();
-    elementPrinter->reset(*dynamic_cast<const ListVectorBatch&>(batch).
-                          elements);
-  }
-
-  void ListColumnPrinter::printRow(uint64_t rowId) {
-    if (hasNulls && !notNull[rowId]) {
-      writeString(buffer, "null");
-    } else {
-      writeChar(buffer, '[');
-      for(int64_t i=offsets[rowId]; i < offsets[rowId+1]; ++i) {
-        if (i != offsets[rowId]) {
-          writeString(buffer, ", ");
-        }
-        elementPrinter->printRow(static_cast<uint64_t>(i));
-      }
-      writeChar(buffer, ']');
-    }
-  }
-
-  MapColumnPrinter::MapColumnPrinter(std::string& buffer,
-                                     const Type& type
-                                     ): ColumnPrinter(buffer, type) {
-    keyPrinter = createColumnPrinter(buffer, type.getSubtype(0));
-    elementPrinter = createColumnPrinter(buffer, type.getSubtype(1));
-  }
-
-  void MapColumnPrinter::reset(const  ColumnVectorBatch& batch) {
-    ColumnPrinter::reset(batch);
-    const MapVectorBatch& myBatch = dynamic_cast<const MapVectorBatch&>(batch);
-    offsets = myBatch.offsets.data();
-    keyPrinter->reset(*myBatch.keys);
-    elementPrinter->reset(*myBatch.elements);
-  }
-
-  void MapColumnPrinter::printRow(uint64_t rowId) {
-    if (hasNulls && !notNull[rowId]) {
-      writeString(buffer, "null");
-    } else {
-      writeChar(buffer, '[');
-      for(int64_t i=offsets[rowId]; i < offsets[rowId+1]; ++i) {
-        if (i != offsets[rowId]) {
-          writeString(buffer, ", ");
-        }
-        writeString(buffer, "{\"key\": ");
-        keyPrinter->printRow(static_cast<uint64_t>(i));
-        writeString(buffer, ", \"value\": ");
-        elementPrinter->printRow(static_cast<uint64_t>(i));
-        writeChar(buffer, '}');
-      }
-      writeChar(buffer, ']');
-    }
-  }
-
-  UnionColumnPrinter::UnionColumnPrinter(std::string& buffer,
-                                           const Type& type
-                                           ): ColumnPrinter(buffer, type) {
-    for(unsigned int i=0; i < type.getSubtypeCount(); ++i) {
-      fieldPrinter.push_back(createColumnPrinter(buffer, type.getSubtype(i))
-                             .release());
-    }
-  }
-
-  UnionColumnPrinter::~UnionColumnPrinter() {
-    for (size_t i = 0; i < fieldPrinter.size(); i++) {
-      delete fieldPrinter[i];
-    }
-  }
-
-  void UnionColumnPrinter::reset(const ColumnVectorBatch& batch) {
-    ColumnPrinter::reset(batch);
-    const UnionVectorBatch& unionBatch =
-      dynamic_cast<const UnionVectorBatch&>(batch);
-    tags = unionBatch.tags.data();
-    offsets = unionBatch.offsets.data();
-    for(size_t i=0; i < fieldPrinter.size(); ++i) {
-      fieldPrinter[i]->reset(*(unionBatch.children[i]));
-    }
-  }
-
-  void UnionColumnPrinter::printRow(uint64_t rowId) {
-    if (hasNulls && !notNull[rowId]) {
-      writeString(buffer, "null");
-    } else {
-      writeString(buffer, "{\"tag\": ");
-      char numBuffer[64];
-      snprintf(numBuffer, sizeof(numBuffer), "%" INT64_FORMAT_STRING "d",
-               static_cast<int64_t>(tags[rowId]));
-      writeString(buffer, numBuffer);
-      writeString(buffer, ", \"value\": ");
-      fieldPrinter[tags[rowId]]->printRow(offsets[rowId]);
-      writeChar(buffer, '}');
-    }
-  }
-
-  StructColumnPrinter::StructColumnPrinter(std::string& buffer,
-                                           const Type& type
-                                           ): ColumnPrinter(buffer, type) {
-    for(unsigned int i=0; i < type.getSubtypeCount(); ++i) {
-      fieldPrinter.push_back(createColumnPrinter(buffer, type.getSubtype(i))
-                             .release());
-    }
-  }
-
-  StructColumnPrinter::~StructColumnPrinter() {
-    for (size_t i = 0; i < fieldPrinter.size(); i++) {
-      delete fieldPrinter[i];
-    }
-  }
-
-  void StructColumnPrinter::reset(const ColumnVectorBatch& batch) {
-    ColumnPrinter::reset(batch);
-    const StructVectorBatch& structBatch =
-      dynamic_cast<const StructVectorBatch&>(batch);
-    for(size_t i=0; i < fieldPrinter.size(); ++i) {
-      fieldPrinter[i]->reset(*(structBatch.fields[i]));
-    }
-  }
-
-  void StructColumnPrinter::printRow(uint64_t rowId) {
-    if (hasNulls && !notNull[rowId]) {
-      writeString(buffer, "null");
-    } else {
-      writeChar(buffer, '{');
-      for(unsigned int i=0; i < fieldPrinter.size(); ++i) {
-        if (i != 0) {
-          writeString(buffer, ", ");
-        }
-        writeChar(buffer, '"');
-        writeString(buffer, type.getFieldName(i).c_str());
-        writeString(buffer, "\": ");
-        fieldPrinter[i]->printRow(rowId);
-      }
-      writeChar(buffer, '}');
-    }
-  }
-
-  DateColumnPrinter::DateColumnPrinter(std::string& buffer,
-                                       const Type& type
-                                       ): ColumnPrinter(buffer, type) {
-    // PASS
-  }
-
-  void DateColumnPrinter::printRow(uint64_t rowId) {
-    if (hasNulls && !notNull[rowId]) {
-      writeString(buffer, "null");
-    } else {
-      const time_t timeValue = data[rowId] * 24 * 60 * 60;
-      struct tm tmValue;
-      gmtime_r(&timeValue, &tmValue);
-      char timeBuffer[11];
-      strftime(timeBuffer, sizeof(timeBuffer), "%Y-%m-%d", &tmValue);
-      writeChar(buffer, '"');
-      writeString(buffer, timeBuffer);
-      writeChar(buffer, '"');
-    }
-  }
-
-  void DateColumnPrinter::reset(const ColumnVectorBatch& batch) {
-    ColumnPrinter::reset(batch);
-    data = dynamic_cast<const LongVectorBatch&>(batch).data.data();
-  }
-
-  BooleanColumnPrinter::BooleanColumnPrinter(std::string& buffer,
-                                             const Type& type
-                                             ): ColumnPrinter(buffer, type) {
-    // PASS
-  }
-
-  void BooleanColumnPrinter::printRow(uint64_t rowId) {
-    if (hasNulls && !notNull[rowId]) {
-      writeString(buffer, "null");
-    } else {
-      writeString(buffer, (data[rowId] ? "true" : "false"));
-    }
-  }
-
-  void BooleanColumnPrinter::reset(const ColumnVectorBatch& batch) {
-    ColumnPrinter::reset(batch);
-    data = dynamic_cast<const LongVectorBatch&>(batch).data.data();
-  }
-
-  BinaryColumnPrinter::BinaryColumnPrinter(std::string& buffer,
-                                           const Type& type
-                                           ): ColumnPrinter(buffer, type) {
-    // PASS
-  }
-
-  void BinaryColumnPrinter::printRow(uint64_t rowId) {
-    if (hasNulls && !notNull[rowId]) {
-      writeString(buffer, "null");
-    } else {
-      writeChar(buffer, '[');
-      for(int64_t i=0; i < length[rowId]; ++i) {
-        if (i != 0) {
-          writeString(buffer, ", ");
-        }
-        char numBuffer[64];
-        snprintf(numBuffer, sizeof(numBuffer), "%d",
-                 (static_cast<const int>(start[rowId][i]) & 0xff));
-        writeString(buffer, numBuffer);
-      }
-      writeChar(buffer, ']');
-    }
-  }
-
-  void BinaryColumnPrinter::reset(const ColumnVectorBatch& batch) {
-    ColumnPrinter::reset(batch);
-    start = dynamic_cast<const StringVectorBatch&>(batch).data.data();
-    length = dynamic_cast<const StringVectorBatch&>(batch).length.data();
-  }
-
-  TimestampColumnPrinter::TimestampColumnPrinter(std::string& buffer,
-                                                 const Type& type
-                                                 ): ColumnPrinter(buffer,
-                                                                  type) {
-    struct tm epochTm;
-    epochTm.tm_sec = 0;
-    epochTm.tm_min = 0;
-    epochTm.tm_hour = 0;
-    epochTm.tm_mday = 1;
-    epochTm.tm_mon = 0;
-    epochTm.tm_year = 70;
-    epochTm.tm_isdst = 0;
-    epoch = mktime(&epochTm);
-  }
-
-  void TimestampColumnPrinter::printRow(uint64_t rowId) {
-    const int64_t NANOS_PER_SECOND = 1000000000;
-    const int64_t NANO_DIGITS = 9;
-    if (hasNulls && !notNull[rowId]) {
-      writeString(buffer, "null");
-    } else {
-      int64_t nanos = data[rowId] % NANOS_PER_SECOND;
-      time_t seconds =
-        static_cast<time_t>(data[rowId] / NANOS_PER_SECOND) + epoch;
-      // make sure the nanos are positive
-      if (nanos < 0) {
-        seconds -= 1;
-        nanos = -nanos;
-      }
-      struct tm tmValue;
-      localtime_r(&seconds, &tmValue);
-      char timeBuffer[20];
-      strftime(timeBuffer, sizeof(timeBuffer), "%Y-%m-%d %H:%M:%S", &tmValue);
-      writeChar(buffer, '"');
-      writeString(buffer, timeBuffer);
-      writeChar(buffer, '.');
-      // remove trailing zeros off the back of the nanos value.
-      int64_t zeroDigits = 0;
-      if (nanos == 0) {
-        zeroDigits = 8;
-      } else {
-        while (nanos % 10 == 0) {
-          nanos /= 10;
-          zeroDigits += 1;
-        }
-      }
-      char numBuffer[64];
-      snprintf(numBuffer, sizeof(numBuffer),
-               "%0*" INT64_FORMAT_STRING "d\"",
-               static_cast<int>(NANO_DIGITS - zeroDigits),
-               static_cast<int64_t >(nanos));
-      writeString(buffer, numBuffer);
-    }
-  }
-
-  void TimestampColumnPrinter::reset(const ColumnVectorBatch& batch) {
-    ColumnPrinter::reset(batch);
-    data = dynamic_cast<const LongVectorBatch&>(batch).data.data();
-  }
-}


[12/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/test/TestCompression.cc
----------------------------------------------------------------------
diff --git a/c++/test/TestCompression.cc b/c++/test/TestCompression.cc
new file mode 100644
index 0000000..0f6eb3c
--- /dev/null
+++ b/c++/test/TestCompression.cc
@@ -0,0 +1,647 @@
+/**
+ * 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 "Compression.hh"
+#include "Exceptions.hh"
+#include "OrcTest.hh"
+#include "wrap/gtest-wrapper.h"
+
+#include <cstdio>
+#include <fstream>
+#include <iostream>
+#include <sstream>
+
+#include "zlib.h"
+#include "wrap/snappy-wrapper.h"
+
+namespace orc {
+
+  class TestCompression : public ::testing::Test {
+  public:
+    ~TestCompression();
+  protected:
+    // Per-test-case set-up.
+    static void SetUpTestCase() {
+      simpleFile = "simple-file.binary";
+      remove(simpleFile);
+      std::ofstream file;
+      file.exceptions(std::ofstream::failbit | std::ofstream::badbit);
+      file.open(simpleFile,
+                std::ios::out | std::ios::binary | std::ios::trunc);
+      for(unsigned int i = 0; i < 200; ++i) {
+        file.put(static_cast<char>(i));
+      }
+      file.close();
+    }
+
+    // Per-test-case tear-down.
+    static void TearDownTestCase() {
+      simpleFile = 0;
+    }
+
+    static const char *simpleFile;
+  };
+
+  const char *TestCompression::simpleFile;
+
+  TestCompression::~TestCompression() {
+    // PASS
+  }
+
+  TEST_F(TestCompression, testPrintBufferEmpty) {
+    std::ostringstream str;
+    printBuffer(str, 0, 0);
+    EXPECT_EQ("", str.str());
+  }
+
+  TEST_F(TestCompression, testPrintBufferSmall) {
+    std::vector<char> buffer(10);
+    std::ostringstream str;
+    for(size_t i=0; i < 10; ++i) {
+      buffer[i] = static_cast<char>(i);
+    }
+    printBuffer(str, buffer.data(), 10);
+    EXPECT_EQ("0000000 00 01 02 03 04 05 06 07 08 09\n", str.str());
+  }
+
+  TEST_F(TestCompression, testPrintBufferLong) {
+    std::vector<char> buffer(300);
+    std::ostringstream str;
+    for(size_t i=0; i < 300; ++i) {
+      buffer[i] = static_cast<char>(i);
+    }
+    printBuffer(str, buffer.data(), 300);
+    std::ostringstream expected;
+    expected << "0000000 00 01 02 03 04 05 06 07 08 09 0a 0b 0c 0d 0e 0f 10"
+             << " 11 12 13 14 15 16 17\n"
+             << "0000018 18 19 1a 1b 1c 1d 1e 1f 20 21 22 23 24 25 26 27 28"
+             << " 29 2a 2b 2c 2d 2e 2f\n"
+             << "0000030 30 31 32 33 34 35 36 37 38 39 3a 3b 3c 3d 3e 3f 40"
+             << " 41 42 43 44 45 46 47\n"
+             << "0000048 48 49 4a 4b 4c 4d 4e 4f 50 51 52 53 54 55 56 57 58"
+             << " 59 5a 5b 5c 5d 5e 5f\n"
+             << "0000060 60 61 62 63 64 65 66 67 68 69 6a 6b 6c 6d 6e 6f 70"
+             << " 71 72 73 74 75 76 77\n"
+             << "0000078 78 79 7a 7b 7c 7d 7e 7f 80 81 82 83 84 85 86 87 88"
+             << " 89 8a 8b 8c 8d 8e 8f\n"
+             << "0000090 90 91 92 93 94 95 96 97 98 99 9a 9b 9c 9d 9e 9f a0"
+             << " a1 a2 a3 a4 a5 a6 a7\n"
+             << "00000a8 a8 a9 aa ab ac ad ae af b0 b1 b2 b3 b4 b5 b6 b7 b8"
+             << " b9 ba bb bc bd be bf\n"
+             << "00000c0 c0 c1 c2 c3 c4 c5 c6 c7 c8 c9 ca cb cc cd ce cf d0"
+             << " d1 d2 d3 d4 d5 d6 d7\n"
+             << "00000d8 d8 d9 da db dc dd de df e0 e1 e2 e3 e4 e5 e6 e7 e8"
+             << " e9 ea eb ec ed ee ef\n"
+             << "00000f0 f0 f1 f2 f3 f4 f5 f6 f7 f8 f9 fa fb fc fd fe ff 00"
+             << " 01 02 03 04 05 06 07\n"
+             << "0000108 08 09 0a 0b 0c 0d 0e 0f 10 11 12 13 14 15 16 17 18"
+             << " 19 1a 1b 1c 1d 1e 1f\n"
+             << "0000120 20 21 22 23 24 25 26 27 28 29 2a 2b\n";
+    EXPECT_EQ(expected.str(), str.str());
+  }
+
+  TEST_F(TestCompression, testArrayBackup) {
+    std::vector<char> bytes(200);
+    for(size_t i=0; i < bytes.size(); ++i) {
+      bytes[i] = static_cast<char>(i);
+    }
+    SeekableArrayInputStream stream(bytes.data(), bytes.size(), 20);
+    const void *ptr;
+    int len;
+    ASSERT_THROW(stream.BackUp(10), std::logic_error);
+    EXPECT_EQ(true, stream.Next(&ptr, &len));
+    EXPECT_EQ(bytes.data(), static_cast<const char *>(ptr));
+    EXPECT_EQ(20, len);
+    stream.BackUp(0);
+    EXPECT_EQ(true, stream.Next(&ptr, &len));
+    EXPECT_EQ(bytes.data() + 20, static_cast<const char *>(ptr));
+    EXPECT_EQ(20, len);
+    stream.BackUp(10);
+    for(unsigned int i=0; i < 8; ++i) {
+      EXPECT_EQ(true, stream.Next(&ptr, &len));
+      unsigned int consumedBytes = 30 + 20 * i;
+      EXPECT_EQ(bytes.data() + consumedBytes, static_cast<const char *>(ptr));
+      EXPECT_EQ(consumedBytes + 20, stream.ByteCount());
+      EXPECT_EQ(20, len);
+    }
+    EXPECT_EQ(true, stream.Next(&ptr, &len));
+    EXPECT_EQ(bytes.data() + 190, static_cast<const char *>(ptr));
+    EXPECT_EQ(10, len);
+    EXPECT_EQ(true, !stream.Next(&ptr, &len));
+    EXPECT_EQ(0, len);
+    ASSERT_THROW(stream.BackUp(30), std::logic_error);
+    EXPECT_EQ(200, stream.ByteCount());
+  }
+
+  TEST_F(TestCompression, testArraySkip) {
+    std::vector<char> bytes(200);
+    for(size_t i=0; i < bytes.size(); ++i) {
+      bytes[i] = static_cast<char>(i);
+    }
+    SeekableArrayInputStream stream(bytes.data(), bytes.size(), 20);
+    const void *ptr;
+    int len;
+    ASSERT_EQ(true, stream.Next(&ptr, &len));
+    EXPECT_EQ(bytes.data(), static_cast<const char *>(ptr));
+    EXPECT_EQ(20, len);
+    ASSERT_EQ(true, !stream.Skip(-10));
+    ASSERT_EQ(true, stream.Skip(80));
+    ASSERT_EQ(true, stream.Next(&ptr, &len));
+    EXPECT_EQ(bytes.data() + 100, static_cast<const char *>(ptr));
+    EXPECT_EQ(20, len);
+    ASSERT_EQ(true, stream.Skip(80));
+    ASSERT_EQ(true, !stream.Next(&ptr, &len));
+    ASSERT_EQ(true, !stream.Skip(181));
+    EXPECT_EQ("SeekableArrayInputStream 200 of 200", stream.getName());
+  }
+
+  TEST_F(TestCompression, testArrayCombo) {
+    std::vector<char> bytes(200);
+    for(size_t i=0; i < bytes.size(); ++i) {
+      bytes[i] = static_cast<char>(i);
+    }
+    SeekableArrayInputStream stream(bytes.data(), bytes.size(), 20);
+    const void *ptr;
+    int len;
+    ASSERT_EQ(true, stream.Next(&ptr, &len));
+    EXPECT_EQ(bytes.data(), static_cast<const char *>(ptr));
+    EXPECT_EQ(20, len);
+    stream.BackUp(10);
+    EXPECT_EQ(10, stream.ByteCount());
+    stream.Skip(4);
+    EXPECT_EQ(14, stream.ByteCount());
+    ASSERT_EQ(true, stream.Next(&ptr, &len));
+    EXPECT_EQ(bytes.data() + 14, static_cast<const char *>(ptr));
+    EXPECT_EQ(true, !stream.Skip(320));
+    EXPECT_EQ(200, stream.ByteCount());
+    EXPECT_EQ(true, !stream.Next(&ptr, &len));
+  }
+
+  // this checks to make sure that a given set of bytes are ascending
+  void checkBytes(const char*data, int length,
+                  unsigned int startValue) {
+    for(unsigned int i=0; static_cast<int>(i) < length; ++i) {
+      EXPECT_EQ(startValue + i, static_cast<unsigned char>(data[i]))
+        << "Output wrong at " << startValue << " + " << i;
+    }
+  }
+
+  TEST_F(TestCompression, testFileBackup) {
+    SCOPED_TRACE("testFileBackup");
+    std::unique_ptr<InputStream> file = readLocalFile(simpleFile);
+    SeekableFileInputStream stream(file.get(), 0, 200, *getDefaultPool(), 20);
+    const void *ptr;
+    int len;
+    ASSERT_THROW(stream.BackUp(10), std::logic_error);
+    EXPECT_EQ(true, stream.Next(&ptr, &len));
+    EXPECT_EQ(20, len);
+    checkBytes(static_cast<const char*>(ptr), len, 0);
+    stream.BackUp(0);
+    EXPECT_EQ(true, stream.Next(&ptr, &len));
+    EXPECT_EQ(20, len);
+    checkBytes(static_cast<const char*>(ptr), len, 20);
+    stream.BackUp(10);
+    EXPECT_EQ(30, stream.ByteCount());
+    EXPECT_EQ(true, stream.Next(&ptr, &len));
+    EXPECT_EQ(10, len);
+    checkBytes(static_cast<const char*>(ptr), len, 30);
+    for(unsigned int i=0; i < 8; ++i) {
+      EXPECT_EQ(20 * i + 40, stream.ByteCount());
+      EXPECT_EQ(true, stream.Next(&ptr, &len));
+      EXPECT_EQ(20, len);
+      checkBytes(static_cast<const char*>(ptr), len, 20 * i + 40);
+    }
+    EXPECT_EQ(true, !stream.Next(&ptr, &len));
+    EXPECT_EQ(0, len);
+    ASSERT_THROW(stream.BackUp(30), std::logic_error);
+    EXPECT_EQ(200, stream.ByteCount());
+  }
+
+  TEST_F(TestCompression, testFileSkip) {
+    SCOPED_TRACE("testFileSkip");
+    std::unique_ptr<InputStream> file = readLocalFile(simpleFile);
+    SeekableFileInputStream stream(file.get(), 0, 200, *getDefaultPool(), 20);
+    const void *ptr;
+    int len;
+    ASSERT_EQ(true, stream.Next(&ptr, &len));
+    checkBytes(static_cast<const char*>(ptr), len, 0);
+    EXPECT_EQ(20, len);
+    ASSERT_EQ(true, !stream.Skip(-10));
+    ASSERT_EQ(true, stream.Skip(80));
+    ASSERT_EQ(true, stream.Next(&ptr, &len));
+    checkBytes(static_cast<const char*>(ptr), len, 100);
+    EXPECT_EQ(20, len);
+    ASSERT_EQ(true, !stream.Skip(80));
+    ASSERT_EQ(true, !stream.Next(&ptr, &len));
+    ASSERT_EQ(true, !stream.Skip(181));
+    EXPECT_EQ("simple-file.binary from 0 for 200", stream.getName());
+  }
+
+  TEST_F(TestCompression, testFileCombo) {
+    SCOPED_TRACE("testFileCombo");
+    std::unique_ptr<InputStream> file = readLocalFile(simpleFile);
+    SeekableFileInputStream stream(file.get(), 0, 200, *getDefaultPool(), 20);
+    const void *ptr;
+    int len;
+    ASSERT_EQ(true, stream.Next(&ptr, &len));
+    checkBytes(static_cast<const char*>(ptr), len, 0);
+    EXPECT_EQ(20, len);
+    stream.BackUp(10);
+    EXPECT_EQ(10, stream.ByteCount());
+    stream.Skip(4);
+    EXPECT_EQ(14, stream.ByteCount());
+    ASSERT_EQ(true, stream.Next(&ptr, &len));
+    checkBytes(static_cast<const char*>(ptr), len, 14);
+    EXPECT_EQ(true, !stream.Skip(320));
+    EXPECT_EQ(200, stream.ByteCount());
+    EXPECT_EQ(true, !stream.Next(&ptr, &len));
+  }
+
+  TEST_F(TestCompression, testFileSeek) {
+    SCOPED_TRACE("testFileSeek");
+    std::unique_ptr<InputStream> file = readLocalFile(simpleFile);
+    SeekableFileInputStream stream(file.get(), 0, 200, *getDefaultPool(), 20);
+    const void *ptr;
+    int len;
+    EXPECT_EQ(0, stream.ByteCount());
+    ASSERT_EQ(true, stream.Next(&ptr, &len));
+    checkBytes(static_cast<const char*>(ptr), len, 0);
+    EXPECT_EQ(20, len);
+    EXPECT_EQ(20, stream.ByteCount());
+    {
+      std::list<uint64_t> offsets(1, 100);
+      PositionProvider posn(offsets);
+      stream.seek(posn);
+    }
+    EXPECT_EQ(100, stream.ByteCount());
+    {
+      std::list<uint64_t> offsets(1, 5);
+      PositionProvider posn(offsets);
+      stream.seek(posn);
+    }
+    EXPECT_EQ(5, stream.ByteCount());
+    ASSERT_EQ(true, stream.Next(&ptr, &len));
+    checkBytes(static_cast<const char*>(ptr), len, 5);
+    EXPECT_EQ(20, len);
+    {
+      std::list<uint64_t> offsets(1, 201);
+      PositionProvider posn(offsets);
+      EXPECT_THROW(stream.seek(posn), std::logic_error);
+      EXPECT_EQ(200, stream.ByteCount());
+    }
+  }
+
+  TEST_F(TestCompression, testCreateNone) {
+    std::vector<char> bytes(10);
+    for(unsigned int i=0; i < bytes.size(); ++i) {
+      bytes[i] = static_cast<char>(i);
+    }
+    std::unique_ptr<SeekableInputStream> result =
+      createDecompressor(CompressionKind_NONE,
+                         std::unique_ptr<SeekableInputStream>
+                         (new SeekableArrayInputStream(bytes.data(),
+                                                       bytes.size())),
+                         32768,
+                         *getDefaultPool());
+    const void *ptr;
+    int length;
+    result->Next(&ptr, &length);
+    for(unsigned int i=0; i < bytes.size(); ++i) {
+      EXPECT_EQ(static_cast<char>(i), static_cast<const char*>(ptr)[i]);
+    }
+  }
+
+  TEST_F(TestCompression, testCreateLzo) {
+    const unsigned char buffer[] = {0};
+    EXPECT_THROW(createDecompressor(CompressionKind_LZO,
+                             std::unique_ptr<SeekableInputStream>
+                                    (new SeekableArrayInputStream(buffer, 0)),
+                                    32768, *getDefaultPool()),
+                 NotImplementedYet);
+  }
+
+  TEST(Zlib, testCreateZlib) {
+    const unsigned char buffer[] = {0x0b, 0x0, 0x0, 0x0, 0x1, 0x2, 0x3, 0x4};
+    std::unique_ptr<SeekableInputStream> result =
+      createDecompressor(CompressionKind_ZLIB,
+                         std::unique_ptr<SeekableInputStream>
+                         (new SeekableArrayInputStream
+                          (buffer, ARRAY_SIZE(buffer))),
+                         32768, *getDefaultPool());
+    EXPECT_EQ("zlib(SeekableArrayInputStream 0 of 8)", result->getName());
+    const void *ptr;
+    int length;
+    ASSERT_EQ(true, result->Next(&ptr, &length));
+    ASSERT_EQ(5, length);
+    for(unsigned int i=0; i < 5; ++i) {
+      EXPECT_EQ(static_cast<char>(i), static_cast<const char*>(ptr)[i]);
+    }
+    EXPECT_EQ("zlib(SeekableArrayInputStream 8 of 8)", result->getName());
+    EXPECT_EQ(5, result->ByteCount());
+    result->BackUp(3);
+    EXPECT_EQ(2, result->ByteCount());
+    ASSERT_EQ(true, result->Next(&ptr, &length));
+    ASSERT_EQ(3, length);
+    for(unsigned int i=0; i < 3; ++i) {
+      EXPECT_EQ(static_cast<char>(i+2), static_cast<const char*>(ptr)[i]);
+    }
+  }
+
+  TEST(Zlib, testLiteralBlocks) {
+    const unsigned char buffer[] = {0x19, 0x0, 0x0, 0x0, 0x1,
+				    0x2, 0x3, 0x4, 0x5, 0x6,
+				    0x7, 0x8, 0x9, 0xa, 0xb,
+				    0xb, 0x0, 0x0, 0xc, 0xd,
+				    0xe, 0xf, 0x10};
+    std::unique_ptr<SeekableInputStream> result =
+      createDecompressor(CompressionKind_ZLIB,
+                         std::unique_ptr<SeekableInputStream>
+                         (new SeekableArrayInputStream
+                          (buffer, ARRAY_SIZE(buffer), 5)),
+                         5, *getDefaultPool());
+    EXPECT_EQ("zlib(SeekableArrayInputStream 0 of 23)", result->getName());
+    const void *ptr;
+    int length;
+    ASSERT_EQ(true, result->Next(&ptr, &length));
+    ASSERT_EQ(2, length);
+    EXPECT_EQ(0, static_cast<const char*>(ptr)[0]);
+    EXPECT_EQ(1, static_cast<const char*>(ptr)[1]);
+    ASSERT_EQ(true, result->Next(&ptr, &length));
+    ASSERT_EQ(5, length);
+    EXPECT_EQ(2, static_cast<const char*>(ptr)[0]);
+    EXPECT_EQ(3, static_cast<const char*>(ptr)[1]);
+    EXPECT_EQ(4, static_cast<const char*>(ptr)[2]);
+    EXPECT_EQ(5, static_cast<const char*>(ptr)[3]);
+    EXPECT_EQ(6, static_cast<const char*>(ptr)[4]);
+    ASSERT_EQ(true, result->Next(&ptr, &length));
+    ASSERT_EQ(5, length);
+    EXPECT_EQ(7, static_cast<const char*>(ptr)[0]);
+    EXPECT_EQ(8, static_cast<const char*>(ptr)[1]);
+    EXPECT_EQ(9, static_cast<const char*>(ptr)[2]);
+    EXPECT_EQ(10, static_cast<const char*>(ptr)[3]);
+    EXPECT_EQ(11, static_cast<const char*>(ptr)[4]);
+    ASSERT_EQ(true, result->Next(&ptr, &length));
+    ASSERT_EQ(2, length);
+    EXPECT_EQ(12, static_cast<const char*>(ptr)[0]);
+    EXPECT_EQ(13, static_cast<const char*>(ptr)[1]);
+    ASSERT_EQ(true, result->Next(&ptr, &length));
+    ASSERT_EQ(3, length);
+    EXPECT_EQ(14, static_cast<const char*>(ptr)[0]);
+    EXPECT_EQ(15, static_cast<const char*>(ptr)[1]);
+    EXPECT_EQ(16, static_cast<const char*>(ptr)[2]);
+  }
+
+  TEST(Zlib, testInflate) {
+    const unsigned char buffer [] = {0xe, 0x0, 0x0, 0x63, 0x60, 0x64, 0x62,
+				     0xc0, 0x8d, 0x0};
+    std::unique_ptr<SeekableInputStream> result =
+      createDecompressor(CompressionKind_ZLIB,
+                         std::unique_ptr<SeekableInputStream>
+                         (new SeekableArrayInputStream
+                          (buffer, ARRAY_SIZE(buffer))), 1000,
+			 *getDefaultPool());
+    const void *ptr;
+    int length;
+    ASSERT_EQ(true, result->Next(&ptr, &length));
+    ASSERT_EQ(30, length);
+    for(int i=0; i < 10; ++i) {
+      for(int j=0; j < 3; ++j) {
+        EXPECT_EQ(j, static_cast<const char*>(ptr)[i * 3 + j]);
+      }
+    }
+  }
+
+  TEST(Zlib, testInflateSequence) {
+    const unsigned char buffer[] = {0xe, 0x0, 0x0, 0x63, 0x60,
+				    0x64, 0x62, 0xc0, 0x8d, 0x0,
+				    0xe, 0x0, 0x0, 0x63, 0x60,
+				    0x64, 0x62, 0xc0, 0x8d, 0x0};
+    std::unique_ptr<SeekableInputStream> result =
+      createDecompressor(CompressionKind_ZLIB,
+                         std::unique_ptr<SeekableInputStream>
+                         (new SeekableArrayInputStream
+                          (buffer, ARRAY_SIZE(buffer), 3)),
+                         1000,
+                         *getDefaultPool());
+    const void *ptr;
+    int length;
+    ASSERT_THROW(result->BackUp(20), std::logic_error);
+    ASSERT_EQ(true, result->Next(&ptr, &length));
+    ASSERT_EQ(30, length);
+    for(int i=0; i < 10; ++i) {
+      for(int j=0; j < 3; ++j) {
+        EXPECT_EQ(j, static_cast<const char*>(ptr)[i * 3 + j]);
+      }
+    }
+    result->BackUp(10);
+    ASSERT_THROW(result->BackUp(2), std::logic_error);
+    ASSERT_EQ(true, result->Next(&ptr, &length));
+    ASSERT_EQ(10, length);
+    for(int i=0; i < 10; ++i) {
+      EXPECT_EQ((i + 2) % 3, static_cast<const char*>(ptr)[i]);
+    }
+    ASSERT_EQ(true, result->Next(&ptr, &length));
+    ASSERT_EQ(30, length);
+    for(int i=0; i < 10; ++i) {
+      for(int j=0; j < 3; ++j) {
+        EXPECT_EQ(j, static_cast<const char*>(ptr)[i * 3 + j]);
+      }
+    }
+  }
+
+  TEST(Zlib, testSkip) {
+    const unsigned char buffer[] = {0x19, 0x0, 0x0, 0x0, 0x1,
+				    0x2, 0x3, 0x4, 0x5, 0x6,
+				    0x7, 0x8, 0x9, 0xa, 0xb,
+				    0xb, 0x0, 0x0, 0xc, 0xd,
+				    0xe, 0xf, 0x10};
+    std::unique_ptr<SeekableInputStream> result =
+      createDecompressor(CompressionKind_ZLIB,
+                         std::unique_ptr<SeekableInputStream>
+                         (new SeekableArrayInputStream
+                          (buffer, ARRAY_SIZE(buffer), 5)),
+                         5, *getDefaultPool());
+    const void *ptr;
+    int length;
+    ASSERT_EQ(true, result->Next(&ptr, &length));
+    ASSERT_EQ(2, length);
+    result->Skip(2);
+    ASSERT_EQ(true, result->Next(&ptr, &length));
+    ASSERT_EQ(3, length);
+    EXPECT_EQ(4, static_cast<const char*>(ptr)[0]);
+    EXPECT_EQ(5, static_cast<const char*>(ptr)[1]);
+    EXPECT_EQ(6, static_cast<const char*>(ptr)[2]);
+    result->BackUp(2);
+    ASSERT_EQ(true, result->Next(&ptr, &length));
+    ASSERT_EQ(2, length);
+    EXPECT_EQ(5, static_cast<const char*>(ptr)[0]);
+    EXPECT_EQ(6, static_cast<const char*>(ptr)[1]);
+    result->Skip(8);
+    ASSERT_EQ(true, result->Next(&ptr, &length));
+    ASSERT_EQ(2, length);
+    EXPECT_EQ(15, static_cast<const char*>(ptr)[0]);
+    EXPECT_EQ(16, static_cast<const char*>(ptr)[1]);
+  }
+
+#define HEADER_SIZE 3
+
+  class CompressBuffer {
+      std::vector<char> buf;
+
+  public:
+    CompressBuffer(size_t capacity) :
+      buf(capacity + HEADER_SIZE)
+    {}
+
+    char *getCompressed() {
+      return buf.data() + HEADER_SIZE;
+    }
+    char *getBuffer() {
+      return buf.data();
+    }
+
+    void writeHeader(size_t compressedSize) {
+      buf[0] = static_cast<char>(compressedSize << 1);
+      buf[1] = static_cast<char>(compressedSize >> 7);
+      buf[2] = static_cast<char>(compressedSize >> 15);
+    }
+
+    size_t getCompressedSize() const {
+      size_t header = static_cast<unsigned char>(buf[0]);
+      header |= static_cast<size_t>(static_cast<unsigned char>(buf[1])) << 8;
+      header |= static_cast<size_t>(static_cast<unsigned char>(buf[2])) << 16;
+      return header >> 1;
+    }
+
+    size_t getBufferSize() const {
+      return getCompressedSize() + HEADER_SIZE;
+    }
+  };
+
+  TEST(Snappy, testBasic) {
+    const int N = 1024;
+    std::vector<char> buf(N * sizeof(int));
+    for (int i=0; i < N; ++i) {
+      (reinterpret_cast<int *>(buf.data()))[i] = i % 8;
+    }
+
+    CompressBuffer compressBuffer(snappy::MaxCompressedLength(buf.size()));
+    size_t compressedSize;
+    snappy::RawCompress(buf.data(), buf.size(), compressBuffer.getCompressed(),
+                        &compressedSize);
+    // compressed size must be < original
+    ASSERT_LT(compressedSize, buf.size());
+    compressBuffer.writeHeader(compressedSize);
+
+    const long blockSize = 3;
+    std::unique_ptr<SeekableInputStream> result = createDecompressor
+        (CompressionKind_SNAPPY,
+         std::unique_ptr<SeekableInputStream>
+           (new SeekableArrayInputStream(compressBuffer.getBuffer(),
+                                         compressBuffer.getBufferSize(),
+                                         blockSize)),
+         buf.size(),
+         *getDefaultPool());
+    const void *data;
+    int length;
+    ASSERT_TRUE(result->Next(&data, &length));
+    ASSERT_EQ(N * sizeof(int), length);
+    for (int i=0; i < N; ++i) {
+      EXPECT_EQ(i % 8, (reinterpret_cast<const int *>(data))[i]);
+    }
+  }
+
+  TEST(Snappy, testMultiBuffer) {
+    const int N = 1024;
+    std::vector<char> buf(N * sizeof(int));
+    for (int i=0; i < N; ++i) {
+      (reinterpret_cast<int *>(buf.data()))[i] = i % 8;
+    }
+
+    CompressBuffer compressBuffer(snappy::MaxCompressedLength(buf.size()));
+    size_t compressedSize;
+    snappy::RawCompress(buf.data(), buf.size(), compressBuffer.getCompressed(),
+                        &compressedSize);
+    // compressed size must be < original
+    ASSERT_LT(compressedSize, buf.size());
+    compressBuffer.writeHeader(compressedSize);
+
+    std::vector<char> input(compressBuffer.getBufferSize() * 4);
+    ::memcpy(input.data(), compressBuffer.getBuffer(),
+             compressBuffer.getBufferSize());
+    ::memcpy(input.data() + compressBuffer.getBufferSize(),
+             compressBuffer.getBuffer(), compressBuffer.getBufferSize());
+    ::memcpy(input.data() + 2 * compressBuffer.getBufferSize(),
+             compressBuffer.getBuffer(), compressBuffer.getBufferSize());
+    ::memcpy(input.data() + 3 * compressBuffer.getBufferSize(),
+             compressBuffer.getBuffer(), compressBuffer.getBufferSize());
+
+    const long blockSize = 3;
+    std::unique_ptr<SeekableInputStream> result = createDecompressor
+        (CompressionKind_SNAPPY,
+         std::unique_ptr<SeekableInputStream>
+         (new SeekableArrayInputStream(input.data(), input.size(), blockSize)),
+         buf.size(),
+         *getDefaultPool());
+    for (int i=0; i < 4; ++i) {
+      const void *data;
+      int length;
+      ASSERT_TRUE(result->Next(&data, &length));
+      for (int j=0; j < N; ++j) {
+          EXPECT_EQ(j % 8, (reinterpret_cast<const int *>(data))[j]);
+      }
+    }
+  }
+
+  TEST(Snappy, testSkip) {
+    const int N = 1024;
+    std::vector<char> buf(N * sizeof(int));
+    for (int i=0; i < N; ++i) {
+      (reinterpret_cast<int *>(buf.data()))[i] = i % 8;
+    }
+
+    CompressBuffer compressBuffer(snappy::MaxCompressedLength(buf.size()));
+    size_t compressedSize;
+    snappy::RawCompress(buf.data(), buf.size(), compressBuffer.getCompressed(),
+                        &compressedSize);
+    // compressed size must be < original
+    ASSERT_LT(compressedSize, buf.size());
+    compressBuffer.writeHeader(compressedSize);
+
+    const long blockSize = 3;
+    std::unique_ptr<SeekableInputStream> result = createDecompressor
+        (CompressionKind_SNAPPY,
+         std::unique_ptr<SeekableInputStream>
+           (new SeekableArrayInputStream(compressBuffer.getBuffer(),
+                                         compressBuffer.getBufferSize(),
+                                         blockSize)),
+         buf.size(),
+         *getDefaultPool());
+    const void *data;
+    int length;
+    // skip 1/2; in 2 jumps
+    ASSERT_TRUE(result->Skip(static_cast<int>(((N / 2) - 2) * sizeof(int))));
+    ASSERT_TRUE(result->Skip(static_cast<int>(2 * sizeof(int))));
+    ASSERT_TRUE(result->Next(&data, &length));
+    ASSERT_EQ((N / 2) * sizeof(int), length);
+    for (int i=N/2; i < N; ++i) {
+      EXPECT_EQ(i % 8, (reinterpret_cast<const int *>(data))[i - N/2]);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/test/TestDriver.cc
----------------------------------------------------------------------
diff --git a/c++/test/TestDriver.cc b/c++/test/TestDriver.cc
new file mode 100644
index 0000000..a34ff07
--- /dev/null
+++ b/c++/test/TestDriver.cc
@@ -0,0 +1,33 @@
+/**
+ * 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 "orc/orc-config.hh"
+
+#include "wrap/orc-proto-wrapper.hh"
+#include "wrap/gtest-wrapper.h"
+
+#include <iostream>
+
+GTEST_API_ int main(int argc, char **argv) {
+  GOOGLE_PROTOBUF_VERIFY_VERSION;
+  std::cout << "ORC version: " << ORC_VERSION << "\n";
+  testing::InitGoogleTest(&argc, argv);
+  int result = RUN_ALL_TESTS();
+  return result;
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/test/TestInt128.cc
----------------------------------------------------------------------
diff --git a/c++/test/TestInt128.cc b/c++/test/TestInt128.cc
new file mode 100644
index 0000000..cc3b30f
--- /dev/null
+++ b/c++/test/TestInt128.cc
@@ -0,0 +1,587 @@
+/**
+ * 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 "orc/Int128.hh"
+
+#include "wrap/gtest-wrapper.h"
+#include "OrcTest.hh"
+
+#include <iostream>
+
+namespace orc {
+
+  TEST(Int128, simpleTest) {
+    Int128 x = 12;
+    Int128 y = 13;
+    x += y;
+    EXPECT_EQ(25, x.toLong());
+    EXPECT_EQ("0x00000000000000000000000000000019", x.toHexString());
+    y -= 1;
+    EXPECT_EQ("0x0000000000000000000000000000000c", y.toHexString());
+    EXPECT_EQ(12, y.toLong());
+    EXPECT_EQ(0, y.getHighBits());
+    EXPECT_EQ(12, y.getLowBits());
+    y -= 20;
+    EXPECT_EQ("0xfffffffffffffffffffffffffffffff8", y.toHexString());
+    EXPECT_EQ(-8, y.toLong());
+    EXPECT_EQ(-1, y.getHighBits());
+    EXPECT_EQ(static_cast<uint64_t>(-8), y.getLowBits());
+    Int128 z;
+    EXPECT_EQ(0, z.toLong());
+  }
+
+  TEST(Int128, testNegate) {
+    Int128 n = -1000000000000;
+    EXPECT_EQ("0xffffffffffffffffffffff172b5af000", n.toHexString());
+    n.negate();
+    EXPECT_EQ(1000000000000, n.toLong());
+    n.abs();
+    EXPECT_EQ(1000000000000, n.toLong());
+    n.negate();
+    EXPECT_EQ(-1000000000000, n.toLong());
+    n.abs();
+    EXPECT_EQ(1000000000000, n.toLong());
+
+    Int128 big(0x12345678, 0x9abcdef0);
+    EXPECT_EQ("0x0000000012345678000000009abcdef0", big.toHexString());
+    EXPECT_EQ(305419896, big.getHighBits());
+    EXPECT_EQ(2596069104, big.getLowBits());
+    big.negate();
+    EXPECT_EQ("0xffffffffedcba987ffffffff65432110", big.toHexString());
+    EXPECT_EQ(0xffffffffedcba987, big.getHighBits());
+    EXPECT_EQ(0xffffffff65432110, big.getLowBits());
+    big.negate();
+    EXPECT_EQ("0x0000000012345678000000009abcdef0", big.toHexString());
+    big.invert();
+    EXPECT_EQ("0xffffffffedcba987ffffffff6543210f", big.toHexString());
+    big.invert();
+    EXPECT_EQ("0x0000000012345678000000009abcdef0", big.toHexString());
+  }
+
+  TEST(Int128, testPlus) {
+    Int128 n(0x1000, 0xfffffffffffffff0);
+    EXPECT_EQ("0x0000000000001000fffffffffffffff0", n.toHexString());
+    n += 0x20;
+    EXPECT_EQ("0x00000000000010010000000000000010", n.toHexString());
+    n -= 0x20;
+    EXPECT_EQ("0x0000000000001000fffffffffffffff0", n.toHexString());
+    n += Int128(2,3);
+    EXPECT_EQ("0x0000000000001002fffffffffffffff3", n.toHexString());
+
+    Int128 x(static_cast<int64_t>(0xffffffffffffff00), 0x200);
+    EXPECT_EQ("0xffffffffffffff000000000000000200", x.toHexString());
+    x -= 0x300;
+    EXPECT_EQ("0xfffffffffffffeffffffffffffffff00", x.toHexString());
+    x -= 0x100;
+    EXPECT_EQ("0xfffffffffffffefffffffffffffffe00", x.toHexString());
+    x += 0x400;
+    EXPECT_EQ("0xffffffffffffff000000000000000200", x.toHexString());
+    x -= Int128(1,2);
+    EXPECT_EQ("0xfffffffffffffeff00000000000001fe", x.toHexString());
+  }
+
+  TEST(Int128, testLogic) {
+    Int128 n = Int128(0x00000000100000002, 0x0000000400000008);
+    n |= Int128(0x0000001000000020, 0x0000004000000080);
+    EXPECT_EQ("0x00000011000000220000004400000088", n.toHexString());
+    n =  Int128(0x0000111100002222, 0x0000333300004444);
+    n &= Int128(0x0000f00000000f00, 0x000000f00000000f);
+    EXPECT_EQ( "0x00001000000002000000003000000004", n.toHexString());
+  }
+
+  TEST(Int128, testShift) {
+    Int128 n(0x123456789abcdef0,0xfedcba9876543210);
+    EXPECT_EQ("0x123456789abcdef0fedcba9876543210", n.toHexString());
+    n <<= 0;
+    EXPECT_EQ("0x123456789abcdef0fedcba9876543210", n.toHexString());
+    n <<= 4;
+    EXPECT_EQ("0x23456789abcdef0fedcba98765432100", n.toHexString());
+    n <<= 8;
+    EXPECT_EQ("0x456789abcdef0fedcba9876543210000", n.toHexString());
+    n += 0x99;
+    EXPECT_EQ("0x456789abcdef0fedcba9876543210099", n.toHexString());
+    n <<= 64;
+    EXPECT_EQ("0xcba98765432100990000000000000000", n.toHexString());
+    n += 0x312;
+    EXPECT_EQ("0xcba98765432100990000000000000312", n.toHexString());
+    n <<= 120;
+    EXPECT_EQ("0x12000000000000000000000000000000", n.toHexString());
+    n += 0x411;
+    EXPECT_EQ("0x12000000000000000000000000000411", n.toHexString());
+    n <<= 128;
+    EXPECT_EQ(0, n.toLong());
+
+    n = Int128(0x123456789abcdef0,0xfedcba9876543210);
+    EXPECT_EQ("0x123456789abcdef0fedcba9876543210", n.toHexString());
+    n >>= 0;
+    EXPECT_EQ("0x123456789abcdef0fedcba9876543210", n.toHexString());
+    n >>= 4;
+    EXPECT_EQ("0x0123456789abcdef0fedcba987654321", n.toHexString());
+    n >>= 8;
+    EXPECT_EQ("0x000123456789abcdef0fedcba9876543", n.toHexString());
+    n += Int128(0x2400000000000000, 0x0);
+    EXPECT_EQ("0x240123456789abcdef0fedcba9876543", n.toHexString());
+    n >>= 64;
+    EXPECT_EQ("0x0000000000000000240123456789abcd", n.toHexString());
+    n += Int128(0x2400000000000000, 0x0);
+    EXPECT_EQ("0x2400000000000000240123456789abcd", n.toHexString());
+    n >>= 129;
+    EXPECT_EQ(0, n.toLong());
+    n = Int128(static_cast<int64_t>(0xfedcba0987654321),0x1234567890abcdef);
+    EXPECT_EQ("0xfedcba09876543211234567890abcdef", n.toHexString());
+    n >>= 64;
+    EXPECT_EQ("0xfffffffffffffffffedcba0987654321", n.toHexString());
+    n = Int128(static_cast<int64_t>(0xfedcba0987654321),0x1234567890abcdef);
+    n >>= 129;
+    EXPECT_EQ("0xffffffffffffffffffffffffffffffff", n.toHexString());
+    n = Int128(-1, 0xffffffffffffffff);
+    n >>= 4;
+    EXPECT_EQ("0x0fffffffffffffffffffffffffffffff", n.toHexString());
+    n = Int128(-0x100, 0xffffffffffffffff);
+    n >>= 68;
+    EXPECT_EQ("0xfffffffffffffffffffffffffffffff0", n.toHexString());
+  }
+
+  TEST(Int128, testCompare) {
+    Int128 x = 123;
+    EXPECT_EQ(Int128(123), x);
+    EXPECT_EQ(true, x == 123);
+    EXPECT_EQ(true, !(x == 124));
+    EXPECT_EQ(true, !(x == -124));
+    EXPECT_EQ(true, !(x == Int128(2, 123)));
+    EXPECT_EQ(true, !(x != 123));
+    EXPECT_EQ(true, x != -123);
+    EXPECT_EQ(true, x != 124);
+    EXPECT_EQ(true, x != Int128(-1, 123));
+    x = Int128(0x123, 0x456);
+    EXPECT_EQ(true, !(x < Int128(0x123, 0x455)));
+    EXPECT_EQ(true, !(x < Int128(0x123, 0x456)));
+    EXPECT_EQ(true, x < Int128(0x123, 0x457));
+    EXPECT_EQ(true, !(x < Int128(0x122, 0x456)));
+    EXPECT_EQ(true, x < Int128(0x124, 0x456));
+
+    EXPECT_EQ(true, !(x <= Int128(0x123, 0x455)));
+    EXPECT_EQ(true, x <= Int128(0x123, 0x456));
+    EXPECT_EQ(true, x <= Int128(0x123, 0x457));
+    EXPECT_EQ(true, !(x <= Int128(0x122, 0x456)));
+    EXPECT_EQ(true, x <= Int128(0x124, 0x456));
+
+    EXPECT_EQ(true, x > Int128(0x123, 0x455));
+    EXPECT_EQ(true, !(x > Int128(0x123, 0x456)));
+    EXPECT_EQ(true, !(x > Int128(0x123, 0x457)));
+    EXPECT_EQ(true, x > Int128(0x122, 0x456));
+    EXPECT_EQ(true, !(x > Int128(0x124, 0x456)));
+
+    EXPECT_EQ(true, x >= Int128(0x123, 0x455));
+    EXPECT_EQ(true, x >= Int128(0x123, 0x456));
+    EXPECT_EQ(true, !(x >= Int128(0x123, 0x457)));
+    EXPECT_EQ(true, x >= Int128(0x122, 0x456));
+    EXPECT_EQ(true, !(x >= Int128(0x124, 0x456)));
+
+    EXPECT_EQ(true, Int128(-3) < Int128(-2));
+    EXPECT_EQ(true, Int128(-3) < Int128(0));
+    EXPECT_EQ(true, Int128(-3) < Int128(3));
+    EXPECT_EQ(true, Int128(0) < Int128(5));
+    EXPECT_EQ(true, Int128::minimumValue() < 0);
+    EXPECT_EQ(true, Int128(0) < Int128::maximumValue());
+    EXPECT_EQ(true, Int128::minimumValue() < Int128::maximumValue());
+  }
+
+  TEST(Int128, testHash) {
+    EXPECT_EQ(0, Int128().hash());
+    EXPECT_EQ(0x123, Int128(0x123).hash());
+    EXPECT_EQ(0xc3c3c3c3,
+              Int128(0x0101010102020202, 0x4040404080808080).hash());
+    EXPECT_EQ(0x122, Int128(-0x123).hash());
+    EXPECT_EQ(0x12345678, Int128(0x1234567800000000, 0x0).hash());
+    EXPECT_EQ(0x12345678, Int128(0x12345678, 0x0).hash());
+    EXPECT_EQ(0x12345678, Int128(0x0, 0x1234567800000000).hash());
+    EXPECT_EQ(0x12345678, Int128(0x0, 0x12345678).hash());
+  }
+
+  TEST(Int128, testFitsInLong) {
+    EXPECT_EQ(true, Int128(0x0, 0x7fffffffffffffff).fitsInLong());
+    EXPECT_EQ(true, !Int128(0x0, 0x8000000000000000).fitsInLong());
+    EXPECT_EQ(true, !Int128(-1, 0x7fffffffffffffff).fitsInLong());
+    EXPECT_EQ(true, Int128(-1, 0x8000000000000000).fitsInLong());
+    EXPECT_EQ(true, !Int128(1, 0x8000000000000000).fitsInLong());
+    EXPECT_EQ(true, !Int128(1, 0x7fffffffffffffff).fitsInLong());
+    EXPECT_EQ(true, !Int128(-2, 0x8000000000000000).fitsInLong());
+    EXPECT_EQ(true, !Int128(-2, 0x7fffffffffffffff).fitsInLong());
+
+    EXPECT_EQ(0x7fffffffffffffff, Int128(0x0, 0x7fffffffffffffff).toLong());
+    EXPECT_THROW(Int128(1,1).toLong(), std::runtime_error);
+    EXPECT_EQ(0x8000000000000000, Int128(-1, 0x8000000000000000).toLong());
+  }
+
+  TEST(Int128, testMultiply) {
+    Int128 x = 2;
+    x *= 3;
+    EXPECT_EQ(6, x.toLong());
+    x *= -4;
+    EXPECT_EQ(-24, x.toLong());
+    x *= 5;
+    EXPECT_EQ(-120, x.toLong());
+    x *= -7;
+    EXPECT_EQ(840, x.toLong());
+    x = Int128(0x0123456776543210,0x1111222233334444);
+    x *= 2;
+    EXPECT_EQ(0x02468aceeca86420, x.getHighBits());
+    EXPECT_EQ(0x2222444466668888, x.getLowBits());
+
+    x = Int128(0x0534AB4C, 0x59D109ADF9892FCA);
+    x *= Int128(0, 0x9033b8c7a);
+    EXPECT_EQ("0x2eead9afd0c6e0e929c18da753113e44", x.toHexString());
+  }
+
+  TEST(Int128, testMultiplyInt) {
+    Int128 x = 2;
+    x *= 1;
+    EXPECT_EQ(2, x.toLong());
+    x *= 2;
+    EXPECT_EQ(4, x.toLong());
+
+    x = 5;
+    x *= 6432346;
+    EXPECT_EQ(6432346 * 5, x.toLong());
+
+    x = (1L << 62) + (3L << 34) + 3L;
+    x *= 96;
+    EXPECT_EQ("0x00000000000000180000048000000120", x.toHexString());
+
+    x = 1;
+    x <<= 126;
+    EXPECT_EQ("0x40000000000000000000000000000000", x.toHexString());
+    x *= 2;
+    EXPECT_EQ("0x80000000000000000000000000000000", x.toHexString());
+    x *= 2;
+    EXPECT_EQ("0x00000000000000000000000000000000", x.toHexString());
+  }
+
+  TEST(Int128, testFillInArray) {
+    Int128 x(0x123456789abcdef0, 0x23456789abcdef01);
+    uint32_t array[4];
+    bool wasNegative;
+    EXPECT_EQ(4, x.fillInArray(array, wasNegative));
+    EXPECT_EQ(true, !wasNegative);
+    EXPECT_EQ(0x12345678, array[0]);
+    EXPECT_EQ(0x9abcdef0, array[1]);
+    EXPECT_EQ(0x23456789, array[2]);
+    EXPECT_EQ(0xabcdef01, array[3]);
+
+    x = 0;
+    EXPECT_EQ(0, x.fillInArray(array, wasNegative));
+    EXPECT_EQ(true, !wasNegative);
+
+    x = 1;
+    EXPECT_EQ(1, x.fillInArray(array, wasNegative));
+    EXPECT_EQ(true, !wasNegative);
+    EXPECT_EQ(1, array[0]);
+
+    x = -12345;
+    EXPECT_EQ(1, x.fillInArray(array, wasNegative));
+    EXPECT_EQ(true, wasNegative);
+    EXPECT_EQ(12345, array[0]);
+
+    x = 0x80000000;
+    EXPECT_EQ(1, x.fillInArray(array, wasNegative));
+    EXPECT_EQ(true, !wasNegative);
+    EXPECT_EQ(0x80000000, array[0]);
+
+    x = Int128(0, 0x8000000000000000);
+    EXPECT_EQ(2, x.fillInArray(array, wasNegative));
+    EXPECT_EQ(true, !wasNegative);
+    EXPECT_EQ(0x80000000, array[0]);
+    EXPECT_EQ(0x0, array[1]);
+
+    x = Int128(0x80000000, 0x123456789abcdef0);
+    EXPECT_EQ(3, x.fillInArray(array, wasNegative));
+    EXPECT_EQ(true, !wasNegative);
+    EXPECT_EQ(0x80000000, array[0]);
+    EXPECT_EQ(0x12345678, array[1]);
+    EXPECT_EQ(0x9abcdef0, array[2]);
+  }
+
+  int fls(uint32_t x);
+
+  TEST(Int128, testFindLastSet) {
+    EXPECT_EQ(0, fls(0));
+    EXPECT_EQ(1, fls(1));
+    EXPECT_EQ(8, fls(0xff));
+    EXPECT_EQ(9, fls(0x100));
+    EXPECT_EQ(29, fls(0x12345678));
+    EXPECT_EQ(31, fls(0x40000000));
+    EXPECT_EQ(32, fls(0x80000000));
+  }
+
+  void shiftArrayLeft(uint32_t* array, int64_t length, int64_t bits);
+
+  TEST(Int128, testShiftArrayLeft) {
+    uint32_t array[5];
+    // make sure nothing blows up
+    array[0] = 0x12345678;
+    shiftArrayLeft(0, 0, 30);
+    EXPECT_EQ(0x12345678, array[0]);
+
+    array[0] = 0x12345678;
+    shiftArrayLeft(array, 1, 0);
+    EXPECT_EQ(0x12345678, array[0]);
+
+    array[0] = 0x12345678;
+    array[1] = 0x9abcdef0;
+    shiftArrayLeft(array, 1, 3);
+    EXPECT_EQ(0x91a2b3c0, array[0]);
+    EXPECT_EQ(0x9abcdef0, array[1]);
+
+    array[0] = 0x12345678;
+    array[1] = 0x9abcdeff;
+    array[2] = 0xfedcba98;
+    array[3] = 0x76543210;
+    shiftArrayLeft(array, 4, 4);
+    EXPECT_EQ(0x23456789, array[0]);
+    EXPECT_EQ(0xabcdefff, array[1]);
+    EXPECT_EQ(0xedcba987, array[2]);
+    EXPECT_EQ(0x65432100, array[3]);
+
+    array[0] = 0;
+    array[1] = 0x12345678;
+    array[2] = 0x9abcdeff;
+    array[3] = 0xfedcba98;
+    array[4] = 0x76543210;
+    shiftArrayLeft(array, 5, 8);
+    EXPECT_EQ(0x00000012, array[0]);
+    EXPECT_EQ(0x3456789a, array[1]);
+    EXPECT_EQ(0xbcdefffe, array[2]);
+    EXPECT_EQ(0xdcba9876, array[3]);
+    EXPECT_EQ(0x54321000, array[4]);
+  }
+
+  void shiftArrayRight(uint32_t* array, int64_t length, int64_t bits);
+
+  TEST(Int128, testShiftArrayRight) {
+    uint32_t array[4];
+    // make sure nothing blows up
+    array[0] = 0x12345678;
+    shiftArrayRight(0, 0, 30);
+    EXPECT_EQ(0x12345678, array[0]);
+
+    array[0] = 0x12345678;
+    array[1] = 0x9abcdef0;
+    shiftArrayRight(array, 1, 3);
+    EXPECT_EQ(0x2468acf, array[0]);
+    EXPECT_EQ(0x9abcdef0, array[1]);
+
+    array[0] = 0x12345678;
+    array[1] = 0x9abcdeff;
+    array[2] = 0xfedcba98;
+    array[3] = 0x76543210;
+    shiftArrayRight(array, 4, 4);
+    EXPECT_EQ(0x01234567, array[0]);
+    EXPECT_EQ(0x89abcdef, array[1]);
+    EXPECT_EQ(0xffedcba9, array[2]);
+    EXPECT_EQ(0x87654321, array[3]);
+  }
+
+  void fixDivisionSigns(Int128 &result, Int128 &remainder,
+                        bool dividendWasNegative, bool divisorWasNegative);
+
+  TEST(Int128, testFixDivisionSigns) {
+    Int128 x = 123;
+    Int128 y = 456;
+    fixDivisionSigns(x, y, false, false);
+    EXPECT_EQ(123, x.toLong());
+    EXPECT_EQ(456, y.toLong());
+
+    x = 123;
+    y = 456;
+    fixDivisionSigns(x, y, false, true);
+    EXPECT_EQ(-123, x.toLong());
+    EXPECT_EQ(456, y.toLong());
+
+    x = 123;
+    y = 456;
+    fixDivisionSigns(x, y, true, false);
+    EXPECT_EQ(-123, x.toLong());
+    EXPECT_EQ(-456, y.toLong());
+
+    x = 123;
+    y = 456;
+    fixDivisionSigns(x, y, true, true);
+    EXPECT_EQ(123, x.toLong());
+    EXPECT_EQ(-456, y.toLong());
+
+  }
+
+  void buildFromArray(Int128& value, uint32_t* array, int64_t length);
+
+  TEST(Int128, testBuildFromArray) {
+    Int128 result;
+    uint32_t array[4]={0x12345678, 0x9abcdef0, 0xfedcba98, 0x76543210};
+
+    buildFromArray(result, array, 0);
+    EXPECT_EQ(0, result.toLong());
+
+    buildFromArray(result, array, 1);
+    EXPECT_EQ(0x12345678, result.toLong());
+
+    buildFromArray(result, array, 2);
+    EXPECT_EQ(0x123456789abcdef0, result.toLong());
+
+    buildFromArray(result, array, 3);
+    EXPECT_EQ("0x00000000123456789abcdef0fedcba98", result.toHexString());
+
+    buildFromArray(result, array, 4);
+    EXPECT_EQ("0x123456789abcdef0fedcba9876543210", result.toHexString());
+
+    EXPECT_THROW(buildFromArray(result, array, 5), std::logic_error);
+  }
+
+  Int128 singleDivide(uint32_t* dividend, int64_t dividendLength,
+                      uint32_t divisor, Int128& remainder,
+                      bool dividendWasNegative, bool divisorWasNegative);
+
+  TEST(Int128, testSingleDivide) {
+    Int128 remainder;
+    uint32_t dividend[4];
+
+    dividend[0] = 23;
+    Int128 result = singleDivide(dividend, 1, 5, remainder, true, false);
+    EXPECT_EQ(-4, result.toLong());
+    EXPECT_EQ(-3, remainder.toLong());
+
+    dividend[0] = 0x100;
+    dividend[1] = 0x120;
+    dividend[2] = 0x140;
+    dividend[3] = 0x160;
+    result = singleDivide(dividend, 4, 0x20, remainder, false, false);
+    EXPECT_EQ("0x00000008000000090000000a0000000b", result.toHexString());
+    EXPECT_EQ(0, remainder.toLong());
+
+    dividend[0] = 0x101;
+    dividend[1] = 0x122;
+    dividend[2] = 0x143;
+    dividend[3] = 0x164;
+    result = singleDivide(dividend, 4, 0x20, remainder, false, false);
+    EXPECT_EQ("0x00000008080000091000000a1800000b", result.toHexString());
+    EXPECT_EQ(4, remainder.toLong());
+
+    dividend[0] = 0x12345678;
+    dividend[1] = 0x9abcdeff;
+    dividend[2] = 0xfedcba09;
+    dividend[3] = 0x87654321;
+    result = singleDivide(dividend, 4, 123, remainder, false, false);
+    EXPECT_EQ("0x0025e390971c97aaaaa84c7077bc23ed", result.toHexString());
+    EXPECT_EQ(0x42, remainder.toLong());
+  }
+
+  TEST(Int128, testDivide) {
+    Int128 dividend;
+    Int128 result;
+    Int128 remainder;
+
+    dividend = 0x12345678;
+    result = dividend.divide(0x123456789abcdef0, remainder);
+    EXPECT_EQ(0, result.toLong());
+    EXPECT_EQ(0x12345678, remainder.toLong());
+
+    EXPECT_THROW(dividend.divide(0, remainder), std::runtime_error);
+
+    dividend = Int128(0x123456789abcdeff, 0xfedcba0987654321);
+    result = dividend.divide(123, remainder);
+    EXPECT_EQ("0x0025e390971c97aaaaa84c7077bc23ed", result.toHexString());
+    EXPECT_EQ(0x42, remainder.toLong());
+
+    dividend = Int128(0x111111112fffffff, 0xeeeeeeeedddddddd);
+    result = dividend.divide(0x1111111123456789, remainder);
+    EXPECT_EQ("0x000000000000000100000000beeeeef7", result.toHexString());
+    EXPECT_EQ("0x0000000000000000037d3b3d60479aae", remainder.toHexString());
+
+    dividend = 1234234662345;
+    result = dividend.divide(642337, remainder);
+    EXPECT_EQ(1921475, result.toLong());
+    EXPECT_EQ(175270, remainder.toLong());
+
+    dividend = Int128(0x42395ADC0534AB4C, 0x59D109ADF9892FCA);
+    result = dividend.divide(0x1234F09DC19A, remainder);
+    EXPECT_EQ("0x000000000003a327c1348bccd2f06c27", result.toHexString());
+    EXPECT_EQ("0x000000000000000000000cacef73b954", remainder.toHexString());
+
+    dividend = Int128(0xfffffffffffffff, 0xf000000000000000);
+    result = dividend.divide(Int128(0, 0x1000000000000000), remainder);
+    EXPECT_EQ("0x0000000000000000ffffffffffffffff", result.toHexString());
+    EXPECT_EQ(0, remainder.toLong());
+
+    dividend = Int128(0x4000000000000000, 0);
+    result = dividend.divide(Int128(0, 0x400000007fffffff), remainder);
+    EXPECT_EQ("0x0000000000000000fffffffe00000007", result.toHexString());
+    EXPECT_EQ("0x00000000000000003ffffffa80000007", remainder.toHexString());
+  }
+
+  TEST(Int128, testToString) {
+    Int128 num = Int128(0x123456789abcdef0, 0xfedcba0987654321);
+    EXPECT_EQ("24197857203266734881846307133640229665", num.toString());
+
+    num = Int128(0, 0xab54a98ceb1f0ad2);
+    EXPECT_EQ("12345678901234567890", num.toString());
+
+    num = 12345678;
+    EXPECT_EQ("12345678", num.toString());
+
+    num = -1234;
+    EXPECT_EQ("-1234", num.toString());
+
+    num = Int128(0x13f20d9c2, 0xfff89d38e1c70cb1);
+    EXPECT_EQ("98765432109876543210987654321", num.toString());
+    num.negate();
+    EXPECT_EQ("-98765432109876543210987654321", num.toString());
+
+    num = Int128("10000000000000000000000000000000000000");
+    EXPECT_EQ("10000000000000000000000000000000000000", num.toString());
+
+    num = Int128("-1234");
+    EXPECT_EQ("-1234", num.toString());
+
+    num = Int128("-12345678901122334455667788990011122233");
+    EXPECT_EQ("-12345678901122334455667788990011122233", num.toString());
+  }
+
+  TEST(Int128, testToDecimalString) {
+    Int128 num = Int128("98765432109876543210987654321098765432");
+    EXPECT_EQ("98765432109876543210987654321098765432",
+              num.toDecimalString(0));
+    EXPECT_EQ("987654321098765432109876543210987.65432",
+              num.toDecimalString(5));
+    num.negate();
+    EXPECT_EQ("-98765432109876543210987654321098765432",
+              num.toDecimalString(0));
+    EXPECT_EQ("-987654321098765432109876543210987.65432",
+              num.toDecimalString(5));
+    num = 123;
+    EXPECT_EQ("12.3", num.toDecimalString(1));
+    EXPECT_EQ("0.123", num.toDecimalString(3));
+    EXPECT_EQ("0.0123", num.toDecimalString(4));
+    EXPECT_EQ("0.00123", num.toDecimalString(5));
+
+    num = -123;
+    EXPECT_EQ("-123", num.toDecimalString(0));
+    EXPECT_EQ("-12.3", num.toDecimalString(1));
+    EXPECT_EQ("-0.123", num.toDecimalString(3));
+    EXPECT_EQ("-0.0123", num.toDecimalString(4));
+    EXPECT_EQ("-0.00123", num.toDecimalString(5));
+  }
+}  // namespace orc


[20/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/Reader.cc
----------------------------------------------------------------------
diff --git a/c++/src/Reader.cc b/c++/src/Reader.cc
new file mode 100644
index 0000000..85f629f
--- /dev/null
+++ b/c++/src/Reader.cc
@@ -0,0 +1,1903 @@
+/**
+ * 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 "orc/Int128.hh"
+#include "orc/OrcFile.hh"
+#include "orc/Reader.hh"
+
+#include "Adaptor.hh"
+#include "ColumnReader.hh"
+#include "Exceptions.hh"
+#include "RLE.hh"
+#include "TypeImpl.hh"
+
+#include "wrap/coded-stream-wrapper.h"
+
+#include <algorithm>
+#include <iostream>
+#include <limits>
+#include <memory>
+#include <sstream>
+#include <string>
+#include <vector>
+
+namespace orc {
+
+  struct ReaderOptionsPrivate {
+    std::list<int64_t> includedColumns;
+    uint64_t dataStart;
+    uint64_t dataLength;
+    uint64_t tailLocation;
+    bool throwOnHive11DecimalOverflow;
+    int32_t forcedScaleOnHive11Decimal;
+    std::ostream* errorStream;
+    MemoryPool* memoryPool;
+    std::string serializedTail;
+
+    ReaderOptionsPrivate() {
+      includedColumns.assign(1,0);
+      dataStart = 0;
+      dataLength = std::numeric_limits<uint64_t>::max();
+      tailLocation = std::numeric_limits<uint64_t>::max();
+      throwOnHive11DecimalOverflow = true;
+      forcedScaleOnHive11Decimal = 6;
+      errorStream = &std::cerr;
+      memoryPool = getDefaultPool();
+    }
+  };
+
+  ReaderOptions::ReaderOptions():
+    privateBits(std::unique_ptr<ReaderOptionsPrivate>
+                (new ReaderOptionsPrivate())) {
+    // PASS
+  }
+
+  ReaderOptions::ReaderOptions(const ReaderOptions& rhs):
+    privateBits(std::unique_ptr<ReaderOptionsPrivate>
+                (new ReaderOptionsPrivate(*(rhs.privateBits.get())))) {
+    // PASS
+  }
+
+  ReaderOptions::ReaderOptions(ReaderOptions& rhs) {
+    // swap privateBits with rhs
+    ReaderOptionsPrivate* l = privateBits.release();
+    privateBits.reset(rhs.privateBits.release());
+    rhs.privateBits.reset(l);
+  }
+
+  ReaderOptions& ReaderOptions::operator=(const ReaderOptions& rhs) {
+    if (this != &rhs) {
+      privateBits.reset(new ReaderOptionsPrivate(*(rhs.privateBits.get())));
+    }
+    return *this;
+  }
+
+  ReaderOptions::~ReaderOptions() {
+    // PASS
+  }
+
+  ReaderOptions& ReaderOptions::include(const std::list<int64_t>& include) {
+    privateBits->includedColumns.assign(include.begin(), include.end());
+    return *this;
+  }
+
+  ReaderOptions& ReaderOptions::include(std::vector<int64_t> include) {
+    privateBits->includedColumns.assign(include.begin(), include.end());
+    return *this;
+  }
+
+  ReaderOptions& ReaderOptions::range(uint64_t offset,
+                                      uint64_t length) {
+    privateBits->dataStart = offset;
+    privateBits->dataLength = length;
+    return *this;
+  }
+
+  ReaderOptions& ReaderOptions::setTailLocation(uint64_t offset) {
+    privateBits->tailLocation = offset;
+    return *this;
+  }
+
+  ReaderOptions& ReaderOptions::setMemoryPool(MemoryPool& pool) {
+    privateBits->memoryPool = &pool;
+    return *this;
+  }
+
+  ReaderOptions& ReaderOptions::setSerializedFileTail(const std::string& value
+                                                      ) {
+    privateBits->serializedTail = value;
+    return *this;
+  }
+
+  MemoryPool* ReaderOptions::getMemoryPool() const{
+    return privateBits->memoryPool;
+  }
+
+  const std::list<int64_t>& ReaderOptions::getInclude() const {
+    return privateBits->includedColumns;
+  }
+
+  uint64_t ReaderOptions::getOffset() const {
+    return privateBits->dataStart;
+  }
+
+  uint64_t ReaderOptions::getLength() const {
+    return privateBits->dataLength;
+  }
+
+  uint64_t ReaderOptions::getTailLocation() const {
+    return privateBits->tailLocation;
+  }
+
+  ReaderOptions& ReaderOptions::throwOnHive11DecimalOverflow(bool shouldThrow){
+    privateBits->throwOnHive11DecimalOverflow = shouldThrow;
+    return *this;
+  }
+
+  bool ReaderOptions::getThrowOnHive11DecimalOverflow() const {
+    return privateBits->throwOnHive11DecimalOverflow;
+  }
+
+  ReaderOptions& ReaderOptions::forcedScaleOnHive11Decimal(int32_t forcedScale
+                                                           ) {
+    privateBits->forcedScaleOnHive11Decimal = forcedScale;
+    return *this;
+  }
+
+  int32_t ReaderOptions::getForcedScaleOnHive11Decimal() const {
+    return privateBits->forcedScaleOnHive11Decimal;
+  }
+
+  ReaderOptions& ReaderOptions::setErrorStream(std::ostream& stream) {
+    privateBits->errorStream = &stream;
+    return *this;
+  }
+
+  std::ostream* ReaderOptions::getErrorStream() const {
+    return privateBits->errorStream;
+  }
+
+  std::string ReaderOptions::getSerializedFileTail() const {
+    return privateBits->serializedTail;
+  }
+
+  StripeInformation::~StripeInformation() {
+
+  }
+
+  class ColumnStatisticsImpl: public ColumnStatistics {
+  private:
+    uint64_t valueCount;
+
+  public:
+    ColumnStatisticsImpl(const proto::ColumnStatistics& stats);
+    virtual ~ColumnStatisticsImpl();
+
+    uint64_t getNumberOfValues() const override {
+      return valueCount;
+    }
+
+    std::string toString() const override {
+      std::ostringstream buffer;
+      buffer << "Column has " << valueCount << " values" << std::endl;
+      return buffer.str();
+    }
+  };
+
+  class BinaryColumnStatisticsImpl: public BinaryColumnStatistics {
+  private:
+    bool _hasTotalLength;
+    uint64_t valueCount;
+    uint64_t totalLength;
+
+  public:
+    BinaryColumnStatisticsImpl(const proto::ColumnStatistics& stats,
+                               bool correctStats);
+    virtual ~BinaryColumnStatisticsImpl();
+
+    bool hasTotalLength() const override {
+      return _hasTotalLength;
+    }
+    uint64_t getNumberOfValues() const override {
+      return valueCount;
+    }
+
+    uint64_t getTotalLength() const override {
+      if(_hasTotalLength){
+        return totalLength;
+      }else{
+        throw ParseError("Total length is not defined.");
+      }
+    }
+
+    std::string toString() const override {
+      std::ostringstream buffer;
+      buffer << "Data type: Binary" << std::endl
+             << "Values: " << valueCount << std::endl;
+      if(_hasTotalLength){
+        buffer << "Total length: " << totalLength << std::endl;
+      }else{
+        buffer << "Total length: not defined" << std::endl;
+      }
+      return buffer.str();
+    }
+  };
+
+  class BooleanColumnStatisticsImpl: public BooleanColumnStatistics {
+  private:
+    bool _hasCount;
+    uint64_t valueCount;
+    uint64_t trueCount;
+
+  public:
+    BooleanColumnStatisticsImpl(const proto::ColumnStatistics& stats, bool correctStats);
+    virtual ~BooleanColumnStatisticsImpl();
+
+    bool hasCount() const override {
+      return _hasCount;
+    }
+
+    uint64_t getNumberOfValues() const override {
+      return valueCount;
+    }
+
+    uint64_t getFalseCount() const override {
+      if(_hasCount){
+        return valueCount - trueCount;
+      }else{
+        throw ParseError("False count is not defined.");
+      }
+    }
+
+    uint64_t getTrueCount() const override {
+      if(_hasCount){
+        return trueCount;
+      }else{
+        throw ParseError("True count is not defined.");
+      }
+    }
+
+    std::string toString() const override {
+      std::ostringstream buffer;
+      buffer << "Data type: Boolean" << std::endl
+             << "Values: " << valueCount << std::endl;
+      if(_hasCount){
+        buffer << "(true: " << trueCount << "; false: "
+               << valueCount - trueCount << ")" << std::endl;
+      } else {
+        buffer << "(true: not defined; false: not defined)" << std::endl;
+        buffer << "True and false count are not defined" << std::endl;
+      }
+      return buffer.str();
+    }
+  };
+
+  class DateColumnStatisticsImpl: public DateColumnStatistics {
+  private:
+    bool _hasMinimum;
+    bool _hasMaximum;
+    uint64_t valueCount;
+    int32_t minimum;
+    int32_t maximum;
+
+  public:
+    DateColumnStatisticsImpl(const proto::ColumnStatistics& stats, bool correctStats);
+    virtual ~DateColumnStatisticsImpl();
+
+    bool hasMinimum() const override {
+      return _hasMinimum;
+    }
+
+    bool hasMaximum() const override {
+      return _hasMaximum;
+    }
+
+    uint64_t getNumberOfValues() const override {
+      return valueCount;
+    }
+
+    int32_t getMinimum() const override {
+      if(_hasMinimum){
+        return minimum;
+      }else{
+        throw ParseError("Minimum is not defined.");
+      }
+    }
+
+    int32_t getMaximum() const override {
+      if(_hasMaximum){
+        return maximum;
+      }else{
+        throw ParseError("Maximum is not defined.");
+      }
+    }
+
+    std::string toString() const override {
+      std::ostringstream buffer;
+      buffer << "Data type: Date" << std::endl
+             << "Values: " << valueCount << std::endl;
+      if(_hasMinimum){
+        buffer << "Minimum: " << minimum << std::endl;
+      }else{
+        buffer << "Minimum: not defined" << std::endl;
+      }
+
+      if(_hasMaximum){
+        buffer << "Maximum: " << maximum << std::endl;
+      }else{
+        buffer << "Maximum: not defined" << std::endl;
+      }
+      return buffer.str();
+    }
+  };
+
+  class DecimalColumnStatisticsImpl: public DecimalColumnStatistics {
+  private:
+    bool _hasMinimum;
+    bool _hasMaximum;
+    bool _hasSum;
+    uint64_t valueCount;
+    std::string minimum;
+    std::string maximum;
+    std::string sum;
+
+  public:
+    DecimalColumnStatisticsImpl(const proto::ColumnStatistics& stats, bool correctStats);
+    virtual ~DecimalColumnStatisticsImpl();
+
+    bool hasMinimum() const override {
+      return _hasMinimum;
+    }
+
+    bool hasMaximum() const override {
+      return _hasMaximum;
+    }
+
+    bool hasSum() const override {
+      return _hasSum;
+    }
+
+    uint64_t getNumberOfValues() const override {
+      return valueCount;
+    }
+
+    Decimal getMinimum() const override {
+      if(_hasMinimum){
+        return Decimal(minimum);
+      }else{
+        throw ParseError("Minimum is not defined.");
+      }
+    }
+
+    Decimal getMaximum() const override {
+      if(_hasMaximum){
+        return Decimal(maximum);
+      }else{
+        throw ParseError("Maximum is not defined.");
+      }
+    }
+
+    Decimal getSum() const override {
+      if(_hasSum){
+        return Decimal(sum);
+      }else{
+        throw ParseError("Sum is not defined.");
+      }
+    }
+
+    std::string toString() const override {
+      std::ostringstream buffer;
+      buffer << "Data type: Decimal" << std::endl
+          << "Values: " << valueCount << std::endl;
+      if(_hasMinimum){
+        buffer << "Minimum: " << minimum << std::endl;
+      }else{
+        buffer << "Minimum: not defined" << std::endl;
+      }
+
+      if(_hasMaximum){
+        buffer << "Maximum: " << maximum << std::endl;
+      }else{
+        buffer << "Maximum: not defined" << std::endl;
+      }
+
+      if(_hasSum){
+        buffer << "Sum: " << sum << std::endl;
+      }else{
+        buffer << "Sum: not defined" << std::endl;
+      }
+
+      return buffer.str();
+    }
+  };
+
+  class DoubleColumnStatisticsImpl: public DoubleColumnStatistics {
+  private:
+    bool _hasMinimum;
+    bool _hasMaximum;
+    bool _hasSum;
+    uint64_t valueCount;
+    double minimum;
+    double maximum;
+    double sum;
+
+  public:
+    DoubleColumnStatisticsImpl(const proto::ColumnStatistics& stats);
+    virtual ~DoubleColumnStatisticsImpl();
+
+    bool hasMinimum() const override {
+      return _hasMinimum;
+    }
+
+    bool hasMaximum() const override {
+      return _hasMaximum;
+    }
+
+    bool hasSum() const override {
+      return _hasSum;
+    }
+
+    uint64_t getNumberOfValues() const override {
+      return valueCount;
+    }
+
+    double getMinimum() const override {
+      if(_hasMinimum){
+        return minimum;
+      }else{
+        throw ParseError("Minimum is not defined.");
+      }
+    }
+
+    double getMaximum() const override {
+      if(_hasMaximum){
+        return maximum;
+      }else{
+        throw ParseError("Maximum is not defined.");
+      }
+    }
+
+    double getSum() const override {
+      if(_hasSum){
+        return sum;
+      }else{
+        throw ParseError("Sum is not defined.");
+      }
+    }
+
+    std::string toString() const override {
+      std::ostringstream buffer;
+      buffer << "Data type: Double" << std::endl
+          << "Values: " << valueCount << std::endl;
+      if(_hasMinimum){
+        buffer << "Minimum: " << minimum << std::endl;
+      }else{
+        buffer << "Minimum: not defined" << std::endl;
+      }
+
+      if(_hasMaximum){
+        buffer << "Maximum: " << maximum << std::endl;
+      }else{
+        buffer << "Maximum: not defined" << std::endl;
+      }
+
+      if(_hasSum){
+        buffer << "Sum: " << sum << std::endl;
+      }else{
+        buffer << "Sum: not defined" << std::endl;
+      }
+      return buffer.str();
+    }
+  };
+
+  class IntegerColumnStatisticsImpl: public IntegerColumnStatistics {
+  private:
+    bool _hasMinimum;
+    bool _hasMaximum;
+    bool _hasSum;
+    uint64_t valueCount;
+    int64_t minimum;
+    int64_t maximum;
+    int64_t sum;
+
+  public:
+    IntegerColumnStatisticsImpl(const proto::ColumnStatistics& stats);
+    virtual ~IntegerColumnStatisticsImpl();
+
+    bool hasMinimum() const override {
+      return _hasMinimum;
+    }
+
+    bool hasMaximum() const override {
+      return _hasMaximum;
+    }
+
+    bool hasSum() const override {
+      return _hasSum;
+    }
+
+    uint64_t getNumberOfValues() const override {
+      return valueCount;
+    }
+
+    int64_t getMinimum() const override {
+      if(_hasMinimum){
+        return minimum;
+      }else{
+        throw ParseError("Minimum is not defined.");
+      }
+    }
+
+    int64_t getMaximum() const override {
+      if(_hasMaximum){
+        return maximum;
+      }else{
+        throw ParseError("Maximum is not defined.");
+      }
+    }
+
+    int64_t getSum() const override {
+      if(_hasSum){
+        return sum;
+      }else{
+        throw ParseError("Sum is not defined.");
+      }
+    }
+
+    std::string toString() const override {
+      std::ostringstream buffer;
+      buffer << "Data type: Integer" << std::endl
+          << "Values: " << valueCount << std::endl;
+      if(_hasMinimum){
+        buffer << "Minimum: " << minimum << std::endl;
+      }else{
+        buffer << "Minimum: not defined" << std::endl;
+      }
+
+      if(_hasMaximum){
+        buffer << "Maximum: " << maximum << std::endl;
+      }else{
+        buffer << "Maximum: not defined" << std::endl;
+      }
+
+      if(_hasSum){
+        buffer << "Sum: " << sum << std::endl;
+      }else{
+        buffer << "Sum: not defined" << std::endl;
+      }
+      return buffer.str();
+    }
+  };
+
+  class StringColumnStatisticsImpl: public StringColumnStatistics {
+  private:
+    bool _hasMinimum;
+    bool _hasMaximum;
+    bool _hasTotalLength;
+    uint64_t valueCount;
+    std::string minimum;
+    std::string maximum;
+    uint64_t totalLength;
+
+  public:
+    StringColumnStatisticsImpl(const proto::ColumnStatistics& stats, bool correctStats);
+    virtual ~StringColumnStatisticsImpl();
+
+    bool hasMinimum() const override {
+      return _hasMinimum;
+    }
+
+    bool hasMaximum() const override {
+      return _hasMaximum;
+    }
+
+    bool hasTotalLength() const override {
+      return _hasTotalLength;
+    }
+
+    uint64_t getNumberOfValues() const override {
+      return valueCount;
+    }
+
+    std::string getMinimum() const override {
+      if(_hasMinimum){
+        return minimum;
+      }else{
+        throw ParseError("Minimum is not defined.");
+      }
+    }
+
+    std::string getMaximum() const override {
+      if(_hasMaximum){
+        return maximum;
+      }else{
+        throw ParseError("Maximum is not defined.");
+      }
+    }
+
+    uint64_t getTotalLength() const override {
+      if(_hasTotalLength){
+        return totalLength;
+      }else{
+        throw ParseError("Total length is not defined.");
+      }
+    }
+
+    std::string toString() const override {
+      std::ostringstream buffer;
+      buffer << "Data type: String" << std::endl
+          << "Values: " << valueCount << std::endl;
+      if(_hasMinimum){
+        buffer << "Minimum: " << minimum << std::endl;
+      }else{
+        buffer << "Minimum is not defined" << std::endl;
+      }
+
+      if(_hasMaximum){
+        buffer << "Maximum: " << maximum << std::endl;
+      }else{
+        buffer << "Maximum is not defined" << std::endl;
+      }
+
+      if(_hasTotalLength){
+        buffer << "Total length: " << totalLength << std::endl;
+      }else{
+        buffer << "Total length is not defined" << std::endl;
+      }
+      return buffer.str();
+    }
+  };
+
+  class TimestampColumnStatisticsImpl: public TimestampColumnStatistics {
+  private:
+    bool _hasMinimum;
+    bool _hasMaximum;
+    uint64_t valueCount;
+    int64_t minimum;
+    int64_t maximum;
+
+  public:
+    TimestampColumnStatisticsImpl(const proto::ColumnStatistics& stats,
+                                  bool correctStats);
+    virtual ~TimestampColumnStatisticsImpl();
+
+    bool hasMinimum() const override {
+      return _hasMinimum;
+    }
+
+    bool hasMaximum() const override {
+      return _hasMaximum;
+    }
+
+    uint64_t getNumberOfValues() const override {
+      return valueCount;
+    }
+
+    int64_t getMinimum() const override {
+      if(_hasMinimum){
+        return minimum;
+      }else{
+        throw ParseError("Minimum is not defined.");
+      }
+    }
+
+    int64_t getMaximum() const override {
+      if(_hasMaximum){
+        return maximum;
+      }else{
+        throw ParseError("Maximum is not defined.");
+      }
+    }
+
+    std::string toString() const override {
+      std::ostringstream buffer;
+      buffer << "Data type: Timestamp" << std::endl
+          << "Values: " << valueCount << std::endl;
+      if(_hasMinimum){
+        buffer << "Minimum: " << minimum << std::endl;
+      }else{
+        buffer << "Minimum is not defined" << std::endl;
+      }
+
+      if(_hasMaximum){
+        buffer << "Maximum: " << maximum << std::endl;
+      }else{
+        buffer << "Maximum is not defined" << std::endl;
+      }
+      return buffer.str();
+    }
+  };
+
+  class StripeInformationImpl : public StripeInformation {
+    uint64_t offset;
+    uint64_t indexLength;
+    uint64_t dataLength;
+    uint64_t footerLength;
+    uint64_t numRows;
+
+  public:
+
+    StripeInformationImpl(uint64_t _offset,
+                          uint64_t _indexLength,
+                          uint64_t _dataLength,
+                          uint64_t _footerLength,
+                          uint64_t _numRows) :
+      offset(_offset),
+      indexLength(_indexLength),
+      dataLength(_dataLength),
+      footerLength(_footerLength),
+      numRows(_numRows)
+    {}
+
+    virtual ~StripeInformationImpl();
+
+    uint64_t getOffset() const override {
+      return offset;
+    }
+
+    uint64_t getLength() const override {
+      return indexLength + dataLength + footerLength;
+    }
+    uint64_t getIndexLength() const override {
+      return indexLength;
+    }
+
+    uint64_t getDataLength()const override {
+      return dataLength;
+    }
+
+    uint64_t getFooterLength() const override {
+      return footerLength;
+    }
+
+    uint64_t getNumberOfRows() const override {
+      return numRows;
+    }
+  };
+
+  ColumnStatistics* convertColumnStatistics(const proto::ColumnStatistics& s,
+                                            bool correctStats) {
+    if (s.has_intstatistics()) {
+      return new IntegerColumnStatisticsImpl(s);
+    } else if (s.has_doublestatistics()) {
+      return new DoubleColumnStatisticsImpl(s);
+    } else if (s.has_stringstatistics()) {
+      return new StringColumnStatisticsImpl(s, correctStats);
+    } else if (s.has_bucketstatistics()) {
+      return new BooleanColumnStatisticsImpl(s, correctStats);
+    } else if (s.has_decimalstatistics()) {
+      return new DecimalColumnStatisticsImpl(s, correctStats);
+    } else if (s.has_timestampstatistics()) {
+      return new TimestampColumnStatisticsImpl(s, correctStats);
+    } else if (s.has_datestatistics()) {
+      return new DateColumnStatisticsImpl(s, correctStats);
+    } else if (s.has_binarystatistics()) {
+      return new BinaryColumnStatisticsImpl(s, correctStats);
+    } else {
+      return new ColumnStatisticsImpl(s);
+    }
+  }
+
+  Statistics::~Statistics() {
+    // PASS
+  }
+
+  class StatisticsImpl: public Statistics {
+  private:
+    std::list<ColumnStatistics*> colStats;
+
+    // DELIBERATELY NOT IMPLEMENTED
+    StatisticsImpl(const StatisticsImpl&);
+    StatisticsImpl& operator=(const StatisticsImpl&);
+
+  public:
+    StatisticsImpl(const proto::StripeStatistics& stripeStats, bool correctStats) {
+      for(int i = 0; i < stripeStats.colstats_size(); i++) {
+        colStats.push_back(convertColumnStatistics
+                           (stripeStats.colstats(i), correctStats));
+      }
+    }
+
+    StatisticsImpl(const proto::Footer& footer, bool correctStats) {
+      for(int i = 0; i < footer.statistics_size(); i++) {
+        colStats.push_back(convertColumnStatistics
+                           (footer.statistics(i), correctStats));
+      }
+    }
+
+    virtual const ColumnStatistics* getColumnStatistics(uint32_t columnId
+                                                        ) const override {
+      std::list<ColumnStatistics*>::const_iterator it = colStats.begin();
+      std::advance(it, static_cast<int64_t>(columnId));
+      return *it;
+    }
+
+    virtual ~StatisticsImpl();
+
+    uint32_t getNumberOfColumns() const override {
+      return static_cast<uint32_t>(colStats.size());
+    }
+  };
+
+  StatisticsImpl::~StatisticsImpl() {
+    for(std::list<ColumnStatistics*>::iterator ptr = colStats.begin();
+        ptr != colStats.end();
+        ++ptr) {
+      delete *ptr;
+    }
+  }
+
+  Reader::~Reader() {
+    // PASS
+  }
+
+  StripeInformationImpl::~StripeInformationImpl() {
+    // PASS
+  }
+
+  static const uint64_t DIRECTORY_SIZE_GUESS = 16 * 1024;
+
+  class ReaderImpl : public Reader {
+  private:
+    // inputs
+    std::unique_ptr<InputStream> stream;
+    ReaderOptions options;
+    const uint64_t footerStart;
+    std::vector<bool> selectedColumns;
+
+    // custom memory pool
+    MemoryPool& memoryPool;
+
+    // postscript
+    std::unique_ptr<proto::PostScript> postscript;
+    const uint64_t blockSize;
+    const CompressionKind compression;
+
+    // footer
+    std::unique_ptr<proto::Footer> footer;
+    DataBuffer<uint64_t> firstRowOfStripe;
+    uint64_t numberOfStripes;
+    std::unique_ptr<Type> schema;
+
+    // metadata
+    mutable std::unique_ptr<proto::Metadata> metadata;
+    mutable bool isMetadataLoaded;
+
+    // reading state
+    uint64_t previousRow;
+    uint64_t firstStripe;
+    uint64_t currentStripe;
+    uint64_t lastStripe; // the stripe AFTER the last one
+    uint64_t currentRowInStripe;
+    uint64_t rowsInCurrentStripe;
+    proto::StripeInformation currentStripeInfo;
+    proto::StripeFooter currentStripeFooter;
+    std::unique_ptr<ColumnReader> reader;
+
+    // internal methods
+    proto::StripeFooter getStripeFooter(const proto::StripeInformation& info);
+    void startNextStripe();
+    void checkOrcVersion();
+    void selectTypeParent(size_t columnId);
+    void selectTypeChildren(size_t columnId);
+    void readMetadata() const;
+    std::unique_ptr<ColumnVectorBatch> createRowBatch(const Type& type,
+                                                      uint64_t capacity
+                                                      ) const;
+
+  public:
+    /**
+     * Constructor that lets the user specify additional options.
+     * @param stream the stream to read from
+     * @param options options for reading
+     * @param postscript the postscript for the file
+     * @param footer the footer for the file
+     * @param footerStart the byte offset of the start of the footer
+     */
+    ReaderImpl(std::unique_ptr<InputStream> stream,
+               const ReaderOptions& options,
+               std::unique_ptr<proto::PostScript> postscript,
+               std::unique_ptr<proto::Footer> footer,
+               uint64_t footerStart);
+
+    const ReaderOptions& getReaderOptions() const;
+
+    CompressionKind getCompression() const override;
+
+    std::string getFormatVersion() const override;
+
+    uint64_t getNumberOfRows() const override;
+
+    uint64_t getRowIndexStride() const override;
+
+    const std::string& getStreamName() const override;
+
+    std::list<std::string> getMetadataKeys() const override;
+
+    std::string getMetadataValue(const std::string& key) const override;
+
+    bool hasMetadataValue(const std::string& key) const override;
+
+    uint64_t getCompressionSize() const override;
+
+    uint64_t getNumberOfStripes() const override;
+
+    std::unique_ptr<StripeInformation> getStripe(uint64_t
+                                                 ) const override;
+
+    uint64_t getNumberOfStripeStatistics() const override;
+
+    std::unique_ptr<Statistics>
+    getStripeStatistics(uint64_t stripeIndex) const override;
+
+
+    uint64_t getContentLength() const override;
+
+    std::unique_ptr<Statistics> getStatistics() const override;
+
+    std::unique_ptr<ColumnStatistics> getColumnStatistics(uint32_t columnId
+                                                          ) const override;
+
+    const Type& getType() const override;
+
+    const std::vector<bool> getSelectedColumns() const override;
+
+    std::unique_ptr<ColumnVectorBatch> createRowBatch(uint64_t size
+                                                      ) const override;
+
+    bool next(ColumnVectorBatch& data) override;
+
+    uint64_t getRowNumber() const override;
+
+    void seekToRow(uint64_t rowNumber) override;
+
+    MemoryPool* getMemoryPool() const ;
+
+    bool hasCorrectStatistics() const override;
+
+    std::string getSerializedFileTail() const override;
+  };
+
+  InputStream::~InputStream() {
+    // PASS
+  };
+
+  uint64_t getCompressionBlockSize(const proto::PostScript& ps) {
+    if (ps.has_compressionblocksize()) {
+      return ps.compressionblocksize();
+    } else {
+      return 256 * 1024;
+    }
+  }
+
+  CompressionKind convertCompressionKind(const proto::PostScript& ps) {
+    if (ps.has_compression()) {
+      return static_cast<CompressionKind>(ps.compression());
+    } else {
+      throw ParseError("Unknown compression type");
+    }
+  }
+
+  ReaderImpl::ReaderImpl(std::unique_ptr<InputStream> input,
+                         const ReaderOptions& opts,
+                         std::unique_ptr<proto::PostScript> _postscript,
+                         std::unique_ptr<proto::Footer> _footer,
+                         uint64_t _footerStart
+                         ): stream(std::move(input)),
+                            options(opts),
+                            footerStart(_footerStart),
+                            memoryPool(*opts.getMemoryPool()),
+                            postscript(std::move(_postscript)),
+                            blockSize(getCompressionBlockSize(*postscript)),
+                            compression(convertCompressionKind(*postscript)),
+                            footer(std::move(_footer)),
+                            firstRowOfStripe(memoryPool, 0) {
+    isMetadataLoaded = false;
+    checkOrcVersion();
+    numberOfStripes = static_cast<uint64_t>(footer->stripes_size());
+    currentStripe = static_cast<uint64_t>(footer->stripes_size());
+    lastStripe = 0;
+    currentRowInStripe = 0;
+    uint64_t rowTotal = 0;
+
+    firstRowOfStripe.resize(static_cast<uint64_t>(footer->stripes_size()));
+    for(size_t i=0; i < static_cast<size_t>(footer->stripes_size()); ++i) {
+      firstRowOfStripe[i] = rowTotal;
+      proto::StripeInformation stripeInfo =
+        footer->stripes(static_cast<int>(i));
+      rowTotal += stripeInfo.numberofrows();
+      bool isStripeInRange = stripeInfo.offset() >= opts.getOffset() &&
+        stripeInfo.offset() < opts.getOffset() + opts.getLength();
+      if (isStripeInRange) {
+        if (i < currentStripe) {
+          currentStripe = i;
+        }
+        if (i >= lastStripe) {
+          lastStripe = i + 1;
+        }
+      }
+    }
+    firstStripe = currentStripe;
+
+    if (currentStripe == 0) {
+      previousRow = (std::numeric_limits<uint64_t>::max)();
+    } else if (currentStripe ==
+               static_cast<uint64_t>(footer->stripes_size())) {
+      previousRow = footer->numberofrows();
+    } else {
+      previousRow = firstRowOfStripe[firstStripe]-1;
+    }
+
+    schema = convertType(footer->types(0), *footer);
+    schema->assignIds(0);
+    previousRow = (std::numeric_limits<uint64_t>::max)();
+
+    selectedColumns.assign(static_cast<size_t>(footer->types_size()), false);
+
+    const std::list<int64_t>& included = options.getInclude();
+    for(std::list<int64_t>::const_iterator columnId = included.begin();
+        columnId != included.end(); ++columnId) {
+      if (*columnId <= static_cast<int64_t>(schema->getSubtypeCount())) {
+        selectTypeParent(static_cast<size_t>(*columnId));
+        selectTypeChildren(static_cast<size_t>(*columnId));
+      }
+    }
+  }
+
+  std::string ReaderImpl::getSerializedFileTail() const {
+    proto::FileTail tail;
+    proto::PostScript *mutable_ps = tail.mutable_postscript();
+    mutable_ps->CopyFrom(*postscript);
+    proto::Footer *mutableFooter = tail.mutable_footer();
+    mutableFooter->CopyFrom(*footer);
+    tail.set_footerstart(footerStart);
+    std::string result;
+    if (!tail.SerializeToString(&result)) {
+      throw ParseError("Failed to serialize file tail");
+    }
+    return result;
+  }
+
+  const ReaderOptions& ReaderImpl::getReaderOptions() const {
+    return options;
+  }
+
+  CompressionKind ReaderImpl::getCompression() const {
+    return compression;
+  }
+
+  uint64_t ReaderImpl::getCompressionSize() const {
+    return blockSize;
+  }
+
+  uint64_t ReaderImpl::getNumberOfStripes() const {
+    return numberOfStripes;
+  }
+
+  uint64_t ReaderImpl::getNumberOfStripeStatistics() const {
+    if (!isMetadataLoaded) {
+      readMetadata();
+    }
+    return metadata.get() == nullptr ? 0 :
+      static_cast<uint64_t>(metadata->stripestats_size());
+  }
+
+  std::unique_ptr<StripeInformation>
+  ReaderImpl::getStripe(uint64_t stripeIndex) const {
+    if (stripeIndex > getNumberOfStripes()) {
+      throw std::logic_error("stripe index out of range");
+    }
+    proto::StripeInformation stripeInfo =
+      footer->stripes(static_cast<int>(stripeIndex));
+
+    return std::unique_ptr<StripeInformation>
+      (new StripeInformationImpl
+       (stripeInfo.offset(),
+        stripeInfo.indexlength(),
+        stripeInfo.datalength(),
+        stripeInfo.footerlength(),
+        stripeInfo.numberofrows()));
+  }
+
+  std::string ReaderImpl::getFormatVersion() const {
+    std::stringstream result;
+    for(int i=0; i < postscript->version_size(); ++i) {
+      if (i != 0) {
+        result << ".";
+      }
+      result << postscript->version(i);
+    }
+    return result.str();
+  }
+
+  uint64_t ReaderImpl::getNumberOfRows() const {
+    return footer->numberofrows();
+  }
+
+  uint64_t ReaderImpl::getContentLength() const {
+    return footer->contentlength();
+  }
+
+  uint64_t ReaderImpl::getRowIndexStride() const {
+    return footer->rowindexstride();
+  }
+
+  const std::string& ReaderImpl::getStreamName() const {
+    return stream->getName();
+  }
+
+  std::list<std::string> ReaderImpl::getMetadataKeys() const {
+    std::list<std::string> result;
+    for(int i=0; i < footer->metadata_size(); ++i) {
+      result.push_back(footer->metadata(i).name());
+    }
+    return result;
+  }
+
+  std::string ReaderImpl::getMetadataValue(const std::string& key) const {
+    for(int i=0; i < footer->metadata_size(); ++i) {
+      if (footer->metadata(i).name() == key) {
+        return footer->metadata(i).value();
+      }
+    }
+    throw std::range_error("key not found");
+  }
+
+  bool ReaderImpl::hasMetadataValue(const std::string& key) const {
+    for(int i=0; i < footer->metadata_size(); ++i) {
+      if (footer->metadata(i).name() == key) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  void ReaderImpl::selectTypeParent(size_t columnId) {
+    for(size_t parent=0; parent < columnId; ++parent) {
+      const proto::Type& parentType = footer->types(static_cast<int>(parent));
+      for(int idx=0; idx < parentType.subtypes_size(); ++idx) {
+        uint64_t child = parentType.subtypes(idx);
+        if (child == columnId) {
+          if (!selectedColumns[parent]) {
+            selectedColumns[parent] = true;
+            selectTypeParent(parent);
+            return;
+          }
+        }
+      }
+    }
+  }
+
+  void ReaderImpl::selectTypeChildren(size_t columnId) {
+    if (!selectedColumns[columnId]) {
+      selectedColumns[columnId] = true;
+      const proto::Type& parentType =
+        footer->types(static_cast<int>(columnId));
+      for(int idx=0; idx < parentType.subtypes_size(); ++idx) {
+        uint64_t child = parentType.subtypes(idx);
+        selectTypeChildren(child);
+      }
+    }
+  }
+
+  const std::vector<bool> ReaderImpl::getSelectedColumns() const {
+    return selectedColumns;
+  }
+
+  const Type& ReaderImpl::getType() const {
+    return *(schema.get());
+  }
+
+  uint64_t ReaderImpl::getRowNumber() const {
+    return previousRow;
+  }
+
+  std::unique_ptr<Statistics> ReaderImpl::getStatistics() const {
+    return std::unique_ptr<Statistics>
+      (new StatisticsImpl(*footer,
+                          hasCorrectStatistics()));
+  }
+
+  std::unique_ptr<ColumnStatistics>
+  ReaderImpl::getColumnStatistics(uint32_t index) const {
+    if (index >= static_cast<uint64_t>(footer->statistics_size())) {
+      throw std::logic_error("column index out of range");
+    }
+    proto::ColumnStatistics col =
+      footer->statistics(static_cast<int32_t>(index));
+    return std::unique_ptr<ColumnStatistics> (convertColumnStatistics
+                                              (col, hasCorrectStatistics()));
+  }
+
+  void ReaderImpl::readMetadata() const {
+    uint64_t metadataSize = postscript->metadatalength();
+    uint64_t metadataStart = footerStart - metadataSize;
+    if (metadataSize != 0) {
+      std::unique_ptr<SeekableInputStream> pbStream =
+        createDecompressor(compression,
+                           std::unique_ptr<SeekableInputStream>
+                             (new SeekableFileInputStream(stream.get(),
+                                                          metadataStart,
+                                                          metadataSize,
+                                                          memoryPool)),
+                           blockSize,
+                           memoryPool);
+      metadata.reset(new proto::Metadata());
+      if (!metadata->ParseFromZeroCopyStream(pbStream.get())) {
+        throw ParseError("Failed to parse the metadata");
+      }
+    }
+    isMetadataLoaded = true;
+  }
+
+  std::unique_ptr<Statistics>
+  ReaderImpl::getStripeStatistics(uint64_t stripeIndex) const {
+    if (!isMetadataLoaded) {
+      readMetadata();
+    }
+    if (metadata.get() == nullptr) {
+      throw std::logic_error("No stripe statistics in file");
+    }
+    return std::unique_ptr<Statistics>
+      (new StatisticsImpl(metadata->stripestats
+                          (static_cast<int>(stripeIndex)),
+                          hasCorrectStatistics()));
+  }
+
+
+  void ReaderImpl::seekToRow(uint64_t rowNumber) {
+    // Empty file
+    if (lastStripe == 0) {
+      return;
+    }
+
+    // If we are reading only a portion of the file
+    // (bounded by firstStripe and lastStripe),
+    // seeking before or after the portion of interest should return no data.
+    // Implement this by setting previousRow to the number of rows in the file.
+
+    // seeking past lastStripe
+    if ( (lastStripe == static_cast<uint64_t>(footer->stripes_size())
+            && rowNumber >= footer->numberofrows())  ||
+         (lastStripe < static_cast<uint64_t>(footer->stripes_size())
+            && rowNumber >= firstRowOfStripe[lastStripe])   ) {
+      currentStripe = static_cast<uint64_t>(footer->stripes_size());
+      previousRow = footer->numberofrows();
+      return;
+    }
+
+    uint64_t seekToStripe = 0;
+    while (seekToStripe+1 < lastStripe &&
+                  firstRowOfStripe[seekToStripe+1] <= rowNumber) {
+      seekToStripe++;
+    }
+
+    // seeking before the first stripe
+    if (seekToStripe < firstStripe) {
+      currentStripe = static_cast<uint64_t>(footer->stripes_size());
+      previousRow = footer->numberofrows();
+      return;
+    }
+
+    currentStripe = seekToStripe;
+    currentRowInStripe = 0;
+    std::unique_ptr<orc::ColumnVectorBatch> batch =
+        createRowBatch(rowNumber-firstRowOfStripe[currentStripe]);
+    next(*batch);
+  }
+
+  bool ReaderImpl::hasCorrectStatistics() const {
+    return postscript->has_writerversion() && postscript->writerversion();
+  }
+
+  proto::StripeFooter ReaderImpl::getStripeFooter
+       (const proto::StripeInformation& info) {
+    uint64_t stripeFooterStart = info.offset() + info.indexlength() +
+      info.datalength();
+    uint64_t stripeFooterLength = info.footerlength();
+    std::unique_ptr<SeekableInputStream> pbStream =
+      createDecompressor(compression,
+                         std::unique_ptr<SeekableInputStream>
+                         (new SeekableFileInputStream(stream.get(),
+                                                      stripeFooterStart,
+                                                      stripeFooterLength,
+                                                      memoryPool,
+                                                      static_cast<int64_t>
+                                                      (blockSize)
+                                                      )),
+                         blockSize,
+                         memoryPool);
+    proto::StripeFooter result;
+    if (!result.ParseFromZeroCopyStream(pbStream.get())) {
+      throw ParseError(std::string("bad StripeFooter from ") +
+                       pbStream->getName());
+    }
+    return result;
+  }
+
+  class StripeStreamsImpl: public StripeStreams {
+  private:
+    const ReaderImpl& reader;
+    const proto::StripeFooter& footer;
+    const uint64_t stripeStart;
+    InputStream& input;
+    MemoryPool& memoryPool;
+
+  public:
+    StripeStreamsImpl(const ReaderImpl& reader,
+                      const proto::StripeFooter& footer,
+                      uint64_t stripeStart,
+                      InputStream& input,
+                      MemoryPool& memoryPool);
+
+    virtual ~StripeStreamsImpl();
+
+    virtual const ReaderOptions& getReaderOptions() const override;
+
+    virtual const std::vector<bool> getSelectedColumns() const override;
+
+    virtual proto::ColumnEncoding getEncoding(int64_t columnId) const override;
+
+    virtual std::unique_ptr<SeekableInputStream>
+    getStream(int64_t columnId,
+              proto::Stream_Kind kind,
+              bool shouldStream) const override;
+
+    MemoryPool& getMemoryPool() const override;
+  };
+
+  StripeStreamsImpl::StripeStreamsImpl(const ReaderImpl& _reader,
+                                       const proto::StripeFooter& _footer,
+                                       uint64_t _stripeStart,
+                                       InputStream& _input,
+                                       MemoryPool& _memoryPool
+                                       ): reader(_reader),
+                                          footer(_footer),
+                                          stripeStart(_stripeStart),
+                                          input(_input),
+                                          memoryPool(_memoryPool) {
+    // PASS
+  }
+
+  StripeStreamsImpl::~StripeStreamsImpl() {
+    // PASS
+  }
+
+  const ReaderOptions& StripeStreamsImpl::getReaderOptions() const {
+    return reader.getReaderOptions();
+  }
+
+  const std::vector<bool> StripeStreamsImpl::getSelectedColumns() const {
+    return reader.getSelectedColumns();
+  }
+
+  proto::ColumnEncoding StripeStreamsImpl::getEncoding(int64_t columnId) const {
+    return footer.columns(static_cast<int>(columnId));
+  }
+
+  std::unique_ptr<SeekableInputStream>
+  StripeStreamsImpl::getStream(int64_t columnId,
+                               proto::Stream_Kind kind,
+                               bool shouldStream) const {
+    uint64_t offset = stripeStart;
+    for(int i = 0; i < footer.streams_size(); ++i) {
+      const proto::Stream& stream = footer.streams(i);
+      if (stream.has_kind() &&
+          stream.kind() == kind &&
+          stream.column() == static_cast<uint64_t>(columnId)) {
+        int64_t myBlock = static_cast<int64_t>(shouldStream ?
+                                         1024 * 1024 :
+                                         stream.length());
+        return createDecompressor(reader.getCompression(),
+                                  std::unique_ptr<SeekableInputStream>
+                                  (new SeekableFileInputStream
+                                   (&input,
+                                    offset,
+                                    stream.length(),
+                                    memoryPool,
+                                    myBlock)),
+                                  reader.getCompressionSize(),
+                                  memoryPool);
+      }
+      offset += stream.length();
+    }
+    return std::unique_ptr<SeekableInputStream>();
+  }
+
+  MemoryPool& StripeStreamsImpl::getMemoryPool() const {
+    return memoryPool;
+  }
+
+  void ReaderImpl::startNextStripe() {
+    currentStripeInfo = footer->stripes(static_cast<int>(currentStripe));
+    currentStripeFooter = getStripeFooter(currentStripeInfo);
+    rowsInCurrentStripe = currentStripeInfo.numberofrows();
+    StripeStreamsImpl stripeStreams(*this, currentStripeFooter,
+                                    currentStripeInfo.offset(),
+                                    *(stream.get()),
+                                    memoryPool);
+    reader = buildReader(*(schema.get()), stripeStreams);
+  }
+
+  void ReaderImpl::checkOrcVersion() {
+    std::string version = getFormatVersion();
+    if (version != "0.11" && version != "0.12") {
+      *(options.getErrorStream())
+        << "Warning: ORC file " << stream->getName()
+        << " was written in an unknown format version "
+        << version << "\n";
+    }
+  }
+
+  bool ReaderImpl::next(ColumnVectorBatch& data) {
+    if (currentStripe >= lastStripe) {
+      data.numElements = 0;
+      if (lastStripe > 0) {
+        previousRow = firstRowOfStripe[lastStripe - 1] +
+          footer->stripes(static_cast<int>(lastStripe - 1)).numberofrows();
+      } else {
+        previousRow = 0;
+      }
+      return false;
+    }
+    if (currentRowInStripe == 0) {
+      startNextStripe();
+    }
+    uint64_t rowsToRead =
+      std::min(static_cast<uint64_t>(data.capacity),
+               rowsInCurrentStripe - currentRowInStripe);
+    data.numElements = rowsToRead;
+    reader->next(data, rowsToRead, 0);
+    // update row number
+    previousRow = firstRowOfStripe[currentStripe] + currentRowInStripe;
+    currentRowInStripe += rowsToRead;
+    if (currentRowInStripe >= rowsInCurrentStripe) {
+      currentStripe += 1;
+      currentRowInStripe = 0;
+    }
+    return rowsToRead != 0;
+  }
+
+  std::unique_ptr<ColumnVectorBatch> ReaderImpl::createRowBatch
+  (const Type& type, uint64_t capacity) const {
+    ColumnVectorBatch* result = nullptr;
+    const Type* subtype;
+    switch (static_cast<int64_t>(type.getKind())) {
+    case BOOLEAN:
+    case BYTE:
+    case SHORT:
+    case INT:
+    case LONG:
+    case TIMESTAMP:
+    case DATE:
+      result = new LongVectorBatch(capacity, memoryPool);
+      break;
+    case FLOAT:
+    case DOUBLE:
+      result = new DoubleVectorBatch(capacity, memoryPool);
+      break;
+    case STRING:
+    case BINARY:
+    case CHAR:
+    case VARCHAR:
+      result = new StringVectorBatch(capacity, memoryPool);
+      break;
+    case STRUCT:
+      result = new StructVectorBatch(capacity, memoryPool);
+      for(uint64_t i=0; i < type.getSubtypeCount(); ++i) {
+        subtype = &(type.getSubtype(i));
+        if (selectedColumns[static_cast<size_t>(subtype->getColumnId())]) {
+          dynamic_cast<StructVectorBatch*>(result)->fields.push_back
+            (createRowBatch(*subtype, capacity).release());
+        }
+      }
+      break;
+    case LIST:
+      result = new ListVectorBatch(capacity, memoryPool);
+      subtype = &(type.getSubtype(0));
+      if (selectedColumns[static_cast<size_t>(subtype->getColumnId())]) {
+        dynamic_cast<ListVectorBatch*>(result)->elements =
+          createRowBatch(*subtype, capacity);
+      }
+      break;
+    case MAP:
+      result = new MapVectorBatch(capacity, memoryPool);
+      subtype = &(type.getSubtype(0));
+      if (selectedColumns[static_cast<size_t>(subtype->getColumnId())]) {
+        dynamic_cast<MapVectorBatch*>(result)->keys =
+          createRowBatch(*subtype, capacity);
+      }
+      subtype = &(type.getSubtype(1));
+      if (selectedColumns[static_cast<size_t>(subtype->getColumnId())]) {
+        dynamic_cast<MapVectorBatch*>(result)->elements =
+          createRowBatch(*subtype, capacity);
+      }
+      break;
+    case DECIMAL:
+      if (type.getPrecision() == 0 || type.getPrecision() > 18) {
+        result = new Decimal128VectorBatch(capacity, memoryPool);
+      } else {
+        result = new Decimal64VectorBatch(capacity, memoryPool);
+      }
+      break;
+    case UNION:
+      result = new UnionVectorBatch(capacity, memoryPool);
+      for(uint64_t i=0; i < type.getSubtypeCount(); ++i) {
+        subtype = &(type.getSubtype(i));
+        if (selectedColumns[static_cast<size_t>(subtype->getColumnId())]) {
+          dynamic_cast<UnionVectorBatch*>(result)->children.push_back
+            (createRowBatch(*subtype, capacity).release());
+        }
+      }
+      break;
+    default:
+      throw NotImplementedYet("not supported yet");
+    }
+    return std::unique_ptr<ColumnVectorBatch>(result);
+  }
+
+  std::unique_ptr<ColumnVectorBatch> ReaderImpl::createRowBatch
+                                              (uint64_t capacity) const {
+    return createRowBatch(*(schema.get()), capacity);
+  }
+
+  void ensureOrcFooter(InputStream* stream,
+                       DataBuffer<char> *buffer,
+                       uint64_t postscriptLength) {
+
+    const std::string MAGIC("ORC");
+    const uint64_t magicLength = MAGIC.length();
+    const char * const bufferStart = buffer->data();
+    const uint64_t bufferLength = buffer->size();
+
+    if (postscriptLength < magicLength || bufferLength < magicLength) {
+      throw ParseError("Invalid ORC postscript length");
+    }
+    const char* magicStart = bufferStart + bufferLength - 1 - magicLength;
+
+    // Look for the magic string at the end of the postscript.
+    if (memcmp(magicStart, MAGIC.c_str(), magicLength) != 0) {
+      // If there is no magic string at the end, check the beginning.
+      // Only files written by Hive 0.11.0 don't have the tail ORC string.
+      char *frontBuffer = new char[magicLength];
+      stream->read(frontBuffer, magicLength, 0);
+      bool foundMatch = memcmp(frontBuffer, MAGIC.c_str(), magicLength) == 0;
+      delete[] frontBuffer;
+      if (!foundMatch) {
+        throw ParseError("Not an ORC file");
+      }
+    }
+  }
+
+  /**
+   * Read the file's postscript from the given buffer.
+   * @param stream the file stream
+   * @param buffer the buffer with the tail of the file.
+   * @param postscriptSize the length of postscript in bytes
+   */
+  std::unique_ptr<proto::PostScript> readPostscript(InputStream *stream,
+                                                    DataBuffer<char> *buffer,
+                                                    uint64_t postscriptSize) {
+    char *ptr = buffer->data();
+    uint64_t readSize = buffer->size();
+
+    ensureOrcFooter(stream, buffer, postscriptSize);
+
+    std::unique_ptr<proto::PostScript> postscript =
+      std::unique_ptr<proto::PostScript>(new proto::PostScript());
+    if (!postscript->ParseFromArray(ptr + readSize - 1 - postscriptSize,
+                                   static_cast<int>(postscriptSize))) {
+      throw ParseError("Failed to parse the postscript from " +
+                       stream->getName());
+    }
+    return std::move(postscript);
+  }
+
+  /**
+   * Parse the footer from the given buffer.
+   * @param stream the file's stream
+   * @param buffer the buffer to parse the footer from
+   * @param footerOffset the offset within the buffer that contains the footer
+   * @param ps the file's postscript
+   * @param memoryPool the memory pool to use
+   */
+  std::unique_ptr<proto::Footer> readFooter(InputStream* stream,
+                                            DataBuffer<char> *&buffer,
+                                            uint64_t footerOffset,
+                                            const proto::PostScript& ps,
+                                            MemoryPool& memoryPool) {
+    char *footerPtr = buffer->data() + footerOffset;
+
+    std::unique_ptr<SeekableInputStream> pbStream =
+      createDecompressor(convertCompressionKind(ps),
+                         std::unique_ptr<SeekableInputStream>
+                         (new SeekableArrayInputStream(footerPtr,
+                                                       ps.footerlength())),
+                         getCompressionBlockSize(ps),
+                         memoryPool);
+
+    std::unique_ptr<proto::Footer> footer =
+      std::unique_ptr<proto::Footer>(new proto::Footer());
+    if (!footer->ParseFromZeroCopyStream(pbStream.get())) {
+      throw ParseError("Failed to parse the footer from " +
+                       stream->getName());
+    }
+    return std::move(footer);
+  }
+
+  std::unique_ptr<Reader> createReader(std::unique_ptr<InputStream> stream,
+                                       const ReaderOptions& options) {
+    MemoryPool *memoryPool = options.getMemoryPool();
+    std::unique_ptr<proto::PostScript> ps;
+    std::unique_ptr<proto::Footer> footer;
+    uint64_t footerStart;
+    std::string serializedFooter = options.getSerializedFileTail();
+    if (serializedFooter.length() != 0) {
+      // Parse the file tail from the serialized one.
+      proto::FileTail tail;
+      if (!tail.ParseFromString(serializedFooter)) {
+        throw ParseError("Failed to parse the file tail from string");
+      }
+      ps.reset(new proto::PostScript(tail.postscript()));
+      footer.reset(new proto::Footer(tail.footer()));
+      footerStart = tail.footerstart();
+    } else {
+      // figure out the size of the file using the option or filesystem
+      uint64_t size = std::min(options.getTailLocation(),
+                               static_cast<uint64_t>(stream->getLength()));
+
+      //read last bytes into buffer to get PostScript
+      uint64_t readSize = std::min(size, DIRECTORY_SIZE_GUESS);
+      if (readSize < 4) {
+        throw ParseError("File size too small");
+      }
+      DataBuffer<char> *buffer = new DataBuffer<char>(*memoryPool, readSize);
+      stream->read(buffer->data(), readSize, size - readSize);
+
+      uint64_t postscriptSize = buffer->data()[readSize - 1] & 0xff;
+      ps = readPostscript(stream.get(), buffer, postscriptSize);
+      uint64_t footerSize = ps->footerlength();
+      uint64_t tailSize = 1 + postscriptSize + footerSize;
+      footerStart = size - tailSize;
+      uint64_t footerOffset;
+
+      if (tailSize > readSize) {
+        buffer->resize(footerSize);
+        stream->read(buffer->data(), footerSize, size - tailSize);
+        footerOffset = 0;
+      } else {
+        footerOffset = readSize - tailSize;
+      }
+
+      footer = readFooter(stream.get(), buffer, footerOffset, *ps,
+                          *memoryPool);
+      delete buffer;
+    }
+    return std::unique_ptr<Reader>(new ReaderImpl(std::move(stream),
+                                                  options,
+                                                  std::move(ps),
+                                                  std::move(footer),
+                                                  footerStart));
+  }
+
+  ColumnStatistics::~ColumnStatistics() {
+    // PASS
+  }
+
+  BinaryColumnStatistics::~BinaryColumnStatistics() {
+    // PASS
+  }
+
+  BooleanColumnStatistics::~BooleanColumnStatistics() {
+    // PASS
+  }
+
+  DateColumnStatistics::~DateColumnStatistics() {
+    // PASS
+  }
+
+  DecimalColumnStatistics::~DecimalColumnStatistics() {
+    // PASS
+  }
+
+  DoubleColumnStatistics::~DoubleColumnStatistics() {
+    // PASS
+  }
+
+  IntegerColumnStatistics::~IntegerColumnStatistics() {
+    // PASS
+  }
+
+  StringColumnStatistics::~StringColumnStatistics() {
+    // PASS
+  }
+
+  TimestampColumnStatistics::~TimestampColumnStatistics() {
+    // PASS
+  }
+
+  ColumnStatisticsImpl::~ColumnStatisticsImpl() {
+    // PASS
+  }
+
+  BinaryColumnStatisticsImpl::~BinaryColumnStatisticsImpl() {
+    // PASS
+  }
+
+  BooleanColumnStatisticsImpl::~BooleanColumnStatisticsImpl() {
+    // PASS
+  }
+
+  DateColumnStatisticsImpl::~DateColumnStatisticsImpl() {
+    // PASS
+  }
+
+  DecimalColumnStatisticsImpl::~DecimalColumnStatisticsImpl() {
+    // PASS
+  }
+
+  DoubleColumnStatisticsImpl::~DoubleColumnStatisticsImpl() {
+    // PASS
+  }
+
+  IntegerColumnStatisticsImpl::~IntegerColumnStatisticsImpl() {
+    // PASS
+  }
+
+  StringColumnStatisticsImpl::~StringColumnStatisticsImpl() {
+    // PASS
+  }
+
+  TimestampColumnStatisticsImpl::~TimestampColumnStatisticsImpl() {
+    // PASS
+  }
+
+  ColumnStatisticsImpl::ColumnStatisticsImpl
+  (const proto::ColumnStatistics& pb) {
+    valueCount = pb.numberofvalues();
+  }
+
+  BinaryColumnStatisticsImpl::BinaryColumnStatisticsImpl
+  (const proto::ColumnStatistics& pb, bool correctStats){
+    valueCount = pb.numberofvalues();
+    if (!pb.has_binarystatistics() || !correctStats) {
+      _hasTotalLength = false;
+    }else{
+      _hasTotalLength = pb.binarystatistics().has_sum();
+      totalLength = static_cast<uint64_t>(pb.binarystatistics().sum());
+    }
+  }
+
+  BooleanColumnStatisticsImpl::BooleanColumnStatisticsImpl
+  (const proto::ColumnStatistics& pb, bool correctStats){
+    valueCount = pb.numberofvalues();
+    if (!pb.has_bucketstatistics() || !correctStats) {
+      _hasCount = false;
+    }else{
+      _hasCount = true;
+      trueCount = pb.bucketstatistics().count(0);
+    }
+  }
+
+  DateColumnStatisticsImpl::DateColumnStatisticsImpl
+  (const proto::ColumnStatistics& pb, bool correctStats){
+    valueCount = pb.numberofvalues();
+    if (!pb.has_datestatistics() || !correctStats) {
+      _hasMinimum = false;
+      _hasMaximum = false;
+    }else{
+        _hasMinimum = pb.datestatistics().has_minimum();
+        _hasMaximum = pb.datestatistics().has_maximum();
+        minimum = pb.datestatistics().minimum();
+        maximum = pb.datestatistics().maximum();
+    }
+  }
+
+  DecimalColumnStatisticsImpl::DecimalColumnStatisticsImpl
+  (const proto::ColumnStatistics& pb, bool correctStats){
+    valueCount = pb.numberofvalues();
+    if (!pb.has_decimalstatistics() || !correctStats) {
+      _hasMinimum = false;
+      _hasMaximum = false;
+      _hasSum = false;
+    }else{
+      const proto::DecimalStatistics& stats = pb.decimalstatistics();
+      _hasMinimum = stats.has_minimum();
+      _hasMaximum = stats.has_maximum();
+      _hasSum = stats.has_sum();
+
+      minimum = stats.minimum();
+      maximum = stats.maximum();
+      sum = stats.sum();
+    }
+  }
+
+  DoubleColumnStatisticsImpl::DoubleColumnStatisticsImpl
+  (const proto::ColumnStatistics& pb){
+    valueCount = pb.numberofvalues();
+    if (!pb.has_doublestatistics()) {
+      _hasMinimum = false;
+      _hasMaximum = false;
+      _hasSum = false;
+    }else{
+      const proto::DoubleStatistics& stats = pb.doublestatistics();
+      _hasMinimum = stats.has_minimum();
+      _hasMaximum = stats.has_maximum();
+      _hasSum = stats.has_sum();
+
+      minimum = stats.minimum();
+      maximum = stats.maximum();
+      sum = stats.sum();
+    }
+  }
+
+  IntegerColumnStatisticsImpl::IntegerColumnStatisticsImpl
+  (const proto::ColumnStatistics& pb){
+    valueCount = pb.numberofvalues();
+    if (!pb.has_intstatistics()) {
+      _hasMinimum = false;
+      _hasMaximum = false;
+      _hasSum = false;
+    }else{
+      const proto::IntegerStatistics& stats = pb.intstatistics();
+      _hasMinimum = stats.has_minimum();
+      _hasMaximum = stats.has_maximum();
+      _hasSum = stats.has_sum();
+
+      minimum = stats.minimum();
+      maximum = stats.maximum();
+      sum = stats.sum();
+    }
+  }
+
+  StringColumnStatisticsImpl::StringColumnStatisticsImpl
+  (const proto::ColumnStatistics& pb, bool correctStats){
+    valueCount = pb.numberofvalues();
+    if (!pb.has_stringstatistics() || !correctStats) {
+      _hasMinimum = false;
+      _hasMaximum = false;
+      _hasTotalLength = false;
+    }else{
+      const proto::StringStatistics& stats = pb.stringstatistics();
+      _hasMinimum = stats.has_minimum();
+      _hasMaximum = stats.has_maximum();
+      _hasTotalLength = stats.has_sum();
+
+      minimum = stats.minimum();
+      maximum = stats.maximum();
+      totalLength = static_cast<uint64_t>(stats.sum());
+    }
+  }
+
+  TimestampColumnStatisticsImpl::TimestampColumnStatisticsImpl
+  (const proto::ColumnStatistics& pb, bool correctStats){
+    valueCount = pb.numberofvalues();
+    if (!pb.has_timestampstatistics() || !correctStats) {
+      _hasMinimum = false;
+      _hasMaximum = false;
+    }else{
+      const proto::TimestampStatistics& stats = pb.timestampstatistics();
+      _hasMinimum = stats.has_minimum();
+      _hasMaximum = stats.has_maximum();
+
+      minimum = stats.minimum();
+      maximum = stats.maximum();
+    }
+  }
+
+}// namespace

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/TypeImpl.cc
----------------------------------------------------------------------
diff --git a/c++/src/TypeImpl.cc b/c++/src/TypeImpl.cc
new file mode 100644
index 0000000..4d37d27
--- /dev/null
+++ b/c++/src/TypeImpl.cc
@@ -0,0 +1,369 @@
+/**
+ * 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 "Exceptions.hh"
+#include "TypeImpl.hh"
+
+#include <iostream>
+#include <sstream>
+
+namespace orc {
+
+  Type::~Type() {
+    // PASS
+  }
+
+  TypeImpl::TypeImpl(TypeKind _kind) {
+    columnId = 0;
+    kind = _kind;
+    maxLength = 0;
+    precision = 0;
+    scale = 0;
+    subtypeCount = 0;
+  }
+
+  TypeImpl::TypeImpl(TypeKind _kind, uint64_t _maxLength) {
+    columnId = 0;
+    kind = _kind;
+    maxLength = _maxLength;
+    precision = 0;
+    scale = 0;
+    subtypeCount = 0;
+  }
+
+  TypeImpl::TypeImpl(TypeKind _kind, uint64_t _precision,
+                     uint64_t _scale) {
+    columnId = 0;
+    kind = _kind;
+    maxLength = 0;
+    precision = _precision;
+    scale = _scale;
+    subtypeCount = 0;
+  }
+
+  TypeImpl::TypeImpl(TypeKind _kind,
+                     const std::vector<Type*>& types,
+                     const std::vector<std::string>& _fieldNames) {
+    columnId = 0;
+    kind = _kind;
+    maxLength = 0;
+    precision = 0;
+    scale = 0;
+    subtypeCount = static_cast<uint64_t>(types.size());
+    subTypes.assign(types.begin(), types.end());
+    fieldNames.assign(_fieldNames.begin(), _fieldNames.end());
+  }
+
+  TypeImpl::TypeImpl(TypeKind _kind, const std::vector<Type*>& types) {
+    columnId = 0;
+    kind = _kind;
+    maxLength = 0;
+    precision = 0;
+    scale = 0;
+    subtypeCount = static_cast<uint64_t>(types.size());
+    subTypes.assign(types.begin(), types.end());
+  }
+
+  int64_t TypeImpl::assignIds(int64_t root) {
+    columnId = root;
+    int64_t current = root + 1;
+    for(uint64_t i=0; i < subtypeCount; ++i) {
+      current = subTypes[i]->assignIds(current);
+    }
+    return current;
+  }
+
+  TypeImpl::~TypeImpl() {
+    for (std::vector<Type*>::iterator it = subTypes.begin();
+        it != subTypes.end(); it++) {
+      delete (*it) ;
+    }
+  }
+
+  int64_t TypeImpl::getColumnId() const {
+    return columnId;
+  }
+
+  TypeKind TypeImpl::getKind() const {
+    return kind;
+  }
+
+  uint64_t TypeImpl::getSubtypeCount() const {
+    return subtypeCount;
+  }
+
+  const Type& TypeImpl::getSubtype(uint64_t i) const {
+    return *(subTypes[i]);
+  }
+
+  const std::string& TypeImpl::getFieldName(uint64_t i) const {
+    return fieldNames[i];
+  }
+
+  uint64_t TypeImpl::getMaximumLength() const {
+    return maxLength;
+  }
+
+  uint64_t TypeImpl::getPrecision() const {
+    return precision;
+  }
+
+  uint64_t TypeImpl::getScale() const {
+    return scale;
+  }
+
+  Type& TypeImpl::addStructField(std::unique_ptr<Type> fieldType,
+                                 const std::string& fieldName) {
+    Type* result = fieldType.release();
+    subTypes.push_back(result);
+    fieldNames.push_back(fieldName);
+    subtypeCount += 1;
+    return *result;
+  }
+
+  std::string TypeImpl::toString() const {
+    switch (static_cast<int64_t>(kind)) {
+    case BOOLEAN:
+      return "boolean";
+    case BYTE:
+      return "tinyint";
+    case SHORT:
+      return "smallint";
+    case INT:
+      return "int";
+    case LONG:
+      return "bigint";
+    case FLOAT:
+      return "float";
+    case DOUBLE:
+      return "double";
+    case STRING:
+      return "string";
+    case BINARY:
+      return "binary";
+    case TIMESTAMP:
+      return "timestamp";
+    case LIST:
+      return "array<" + subTypes[0]->toString() + ">";
+    case MAP:
+      return "map<" + subTypes[0]->toString() + "," +
+        subTypes[1]->toString() +  ">";
+    case STRUCT: {
+      std::string result = "struct<";
+      for(size_t i=0; i < subTypes.size(); ++i) {
+        if (i != 0) {
+          result += ",";
+        }
+        result += fieldNames[i];
+        result += ":";
+        result += subTypes[i]->toString();
+      }
+      result += ">";
+      return result;
+    }
+    case UNION: {
+      std::string result = "uniontype<";
+      for(size_t i=0; i < subTypes.size(); ++i) {
+        if (i != 0) {
+          result += ",";
+        }
+        result += subTypes[i]->toString();
+      }
+      result += ">";
+      return result;
+    }
+    case DECIMAL: {
+      std::stringstream result;
+      result << "decimal(" << precision << "," << scale << ")";
+      return result.str();
+    }
+    case DATE:
+      return "date";
+    case VARCHAR: {
+      std::stringstream result;
+      result << "varchar(" << maxLength << ")";
+      return result.str();
+    }
+    case CHAR: {
+      std::stringstream result;
+      result << "char(" << maxLength << ")";
+      return result.str();
+    }
+    default:
+      throw NotImplementedYet("Unknown type");
+    }
+  }
+
+  std::unique_ptr<Type> createPrimitiveType(TypeKind kind) {
+    return std::unique_ptr<Type>(new TypeImpl(kind));
+  }
+
+  std::unique_ptr<Type> createCharType(TypeKind kind,
+                                       uint64_t maxLength) {
+    return std::unique_ptr<Type>(new TypeImpl(kind, maxLength));
+  }
+
+  std::unique_ptr<Type> createDecimalType(uint64_t precision,
+                                          uint64_t scale) {
+    return std::unique_ptr<Type>(new TypeImpl(DECIMAL, precision, scale));
+  }
+
+  std::unique_ptr<Type> createStructType() {
+    return std::unique_ptr<Type>(new TypeImpl(STRUCT));
+  }
+
+  std::unique_ptr<Type>
+      createStructType(std::vector<Type*> types,
+                       std::vector<std::string> fieldNames) {
+    std::vector<Type*> typeVector(types.begin(), types.end());
+    std::vector<std::string> fieldVector(fieldNames.begin(), fieldNames.end());
+
+    return std::unique_ptr<Type>(new TypeImpl(STRUCT, typeVector,
+                                              fieldVector));
+  }
+
+#ifdef ORC_CXX_HAS_INITIALIZER_LIST
+  std::unique_ptr<Type> createStructType(
+      std::initializer_list<std::unique_ptr<Type> > types,
+      std::initializer_list<std::string> fieldNames) {
+    std::vector<Type*> typeVector(types.size());
+    std::vector<std::string> fieldVector(types.size());
+    auto currentType = types.begin();
+    auto endType = types.end();
+    size_t current = 0;
+    while (currentType != endType) {
+      typeVector[current++] =
+          const_cast<std::unique_ptr<Type>*>(currentType)->release();
+      ++currentType;
+    }
+    fieldVector.insert(fieldVector.end(), fieldNames.begin(),
+        fieldNames.end());
+    return std::unique_ptr<Type>(new TypeImpl(STRUCT, typeVector,
+        fieldVector));
+  }
+#endif
+
+  std::unique_ptr<Type> createListType(std::unique_ptr<Type> elements) {
+    std::vector<Type*> subtypes(1);
+    subtypes[0] = elements.release();
+    return std::unique_ptr<Type>(new TypeImpl(LIST, subtypes));
+  }
+
+  std::unique_ptr<Type> createMapType(std::unique_ptr<Type> key,
+                                      std::unique_ptr<Type> value) {
+    std::vector<Type*> subtypes(2);
+    subtypes[0] = key.release();
+    subtypes[1] = value.release();
+    return std::unique_ptr<Type>(new TypeImpl(MAP, subtypes));
+  }
+
+  std::unique_ptr<Type>
+      createUnionType(std::vector<Type*> types) {
+    std::vector<Type*> typeVector(types.begin(), types.end());
+    return std::unique_ptr<Type>(new TypeImpl(UNION, typeVector));
+  }
+
+  std::string printProtobufMessage(const google::protobuf::Message& message);
+  std::unique_ptr<Type> convertType(const proto::Type& type,
+                                    const proto::Footer& footer) {
+    switch (static_cast<int64_t>(type.kind())) {
+
+    case proto::Type_Kind_BOOLEAN:
+    case proto::Type_Kind_BYTE:
+    case proto::Type_Kind_SHORT:
+    case proto::Type_Kind_INT:
+    case proto::Type_Kind_LONG:
+    case proto::Type_Kind_FLOAT:
+    case proto::Type_Kind_DOUBLE:
+    case proto::Type_Kind_STRING:
+    case proto::Type_Kind_BINARY:
+    case proto::Type_Kind_TIMESTAMP:
+    case proto::Type_Kind_DATE:
+      return std::unique_ptr<Type>
+        (new TypeImpl(static_cast<TypeKind>(type.kind())));
+
+    case proto::Type_Kind_CHAR:
+    case proto::Type_Kind_VARCHAR:
+      return std::unique_ptr<Type>
+        (new TypeImpl(static_cast<TypeKind>(type.kind()),
+                      type.maximumlength()));
+
+    case proto::Type_Kind_DECIMAL:
+      return std::unique_ptr<Type>
+        (new TypeImpl(DECIMAL, type.precision(), type.scale()));
+
+    case proto::Type_Kind_LIST:
+    case proto::Type_Kind_MAP:
+    case proto::Type_Kind_UNION: {
+      uint64_t size = static_cast<uint64_t>(type.subtypes_size());
+      std::vector<Type*> typeList(size);
+      for(int i=0; i < type.subtypes_size(); ++i) {
+        typeList[static_cast<uint64_t>(i)] =
+          convertType(footer.types(static_cast<int>(type.subtypes(i))),
+                      footer).release();
+      }
+      return std::unique_ptr<Type>
+        (new TypeImpl(static_cast<TypeKind>(type.kind()), typeList));
+    }
+
+    case proto::Type_Kind_STRUCT: {
+      uint64_t size = static_cast<uint64_t>(type.subtypes_size());
+      std::vector<Type*> typeList(size);
+      std::vector<std::string> fieldList(size);
+      for(int i=0; i < type.subtypes_size(); ++i) {
+        typeList[static_cast<uint64_t>(i)] =
+          convertType(footer.types(static_cast<int>(type.subtypes(i))),
+                      footer).release();
+        fieldList[static_cast<uint64_t>(i)] = type.fieldnames(i);
+      }
+      return std::unique_ptr<Type>
+        (new TypeImpl(STRUCT, typeList, fieldList));
+    }
+    default:
+      throw NotImplementedYet("Unknown type kind");
+    }
+  }
+
+  std::string kind2String(TypeKind t) {
+      std::string name ;
+      switch(static_cast<int64_t>(t)) {
+        case BOOLEAN: { name = "BOOLEAN"; break; }
+        case BYTE: { name = "TINYINT"; break; }
+        case SHORT: { name = "SMALLINT"; break; }
+        case INT: { name = "INT"; break; }
+        case LONG: { name = "BIGINT"; break; }
+        case FLOAT: { name = "FLOAT"; break; }
+        case DOUBLE: { name = "DOUBLE"; break; }
+        case STRING: { name = "STRING"; break; }
+        case BINARY: { name = "BINARY"; break; }
+        case TIMESTAMP: { name = "TIMESTAMP"; break; }
+        case LIST: { name = "LIST"; break; }
+        case MAP: { name = "MAP"; break; }
+        case STRUCT: { name = "STRUCT"; break; }
+        case UNION: { name = "UNION"; break; }
+        case DECIMAL: { name = "DECIMAL"; break; }
+        case DATE: { name = "DATE"; break; }
+        case VARCHAR: { name = "VARCHAR"; break; }
+        case CHAR: { name = "CHAR"; break; }
+        default: { name = "UNKNOWN"; break; }
+      }
+      return name ;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/TypeImpl.hh
----------------------------------------------------------------------
diff --git a/c++/src/TypeImpl.hh b/c++/src/TypeImpl.hh
new file mode 100644
index 0000000..756375f
--- /dev/null
+++ b/c++/src/TypeImpl.hh
@@ -0,0 +1,101 @@
+/**
+ * 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 TYPE_IMPL_HH
+#define TYPE_IMPL_HH
+
+#include "orc/Vector.hh"
+
+#include "Adaptor.hh"
+#include "wrap/orc-proto-wrapper.hh"
+
+#include <vector>
+
+namespace orc {
+
+  class TypeImpl: public Type {
+  private:
+    int64_t columnId;
+    TypeKind kind;
+    std::vector<Type*> subTypes;
+    std::vector<std::string> fieldNames;
+    uint64_t subtypeCount;
+    uint64_t maxLength;
+    uint64_t precision;
+    uint64_t scale;
+
+  public:
+    /**
+     * Create most of the primitive types.
+     */
+    TypeImpl(TypeKind kind);
+
+    /**
+     * Create char and varchar type.
+     */
+    TypeImpl(TypeKind kind, uint64_t maxLength);
+
+    /**
+     * Create decimal type.
+     */
+    TypeImpl(TypeKind kind, uint64_t precision,
+             uint64_t scale);
+
+    /**
+     * Create struct type.
+     */
+    TypeImpl(TypeKind kind,
+             const std::vector<Type*>& types,
+             const std::vector<std::string>& fieldNames);
+
+    /**
+     * Create list, map, and union type.
+     */
+    TypeImpl(TypeKind kind, const std::vector<Type*>& types);
+
+    virtual ~TypeImpl();
+
+    int64_t assignIds(int64_t root) override;
+
+    int64_t getColumnId() const override;
+
+    TypeKind getKind() const override;
+
+    uint64_t getSubtypeCount() const override;
+
+    const Type& getSubtype(uint64_t i) const override;
+
+    const std::string& getFieldName(uint64_t i) const override;
+
+    uint64_t getMaximumLength() const override;
+
+    uint64_t getPrecision() const override;
+
+    uint64_t getScale() const override;
+
+    std::string toString() const override;
+
+    Type& addStructField(std::unique_ptr<Type> fieldType,
+                         const std::string& fieldName) override;
+  };
+
+  std::unique_ptr<Type> convertType(const proto::Type& type,
+                                    const proto::Footer& footer);
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/Vector.cc
----------------------------------------------------------------------
diff --git a/c++/src/Vector.cc b/c++/src/Vector.cc
new file mode 100644
index 0000000..e65c69b
--- /dev/null
+++ b/c++/src/Vector.cc
@@ -0,0 +1,306 @@
+/**
+ * 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 "orc/Vector.hh"
+
+#include "Adaptor.hh"
+#include "Exceptions.hh"
+
+#include <iostream>
+#include <sstream>
+#include <cstdlib>
+
+namespace orc {
+
+  ColumnVectorBatch::ColumnVectorBatch(uint64_t cap,
+                                       MemoryPool& pool
+                                       ): capacity(cap),
+                                          numElements(0),
+                                          notNull(pool, cap),
+                                          hasNulls(false),
+                                          memoryPool(pool) {
+    // PASS
+  }
+
+  ColumnVectorBatch::~ColumnVectorBatch() {
+    // PASS
+  }
+
+  void ColumnVectorBatch::resize(uint64_t cap) {
+    if (capacity < cap) {
+      capacity = cap;
+      notNull.resize(cap);
+    }
+  }
+
+  LongVectorBatch::LongVectorBatch(uint64_t capacity, MemoryPool& pool
+                     ): ColumnVectorBatch(capacity, pool),
+                        data(pool, capacity) {
+    // PASS
+  }
+
+  LongVectorBatch::~LongVectorBatch() {
+    // PASS
+  }
+
+  std::string LongVectorBatch::toString() const {
+    std::ostringstream buffer;
+    buffer << "Long vector <" << numElements << " of " << capacity << ">";
+    return buffer.str();
+  }
+
+  void LongVectorBatch::resize(uint64_t cap) {
+    if (capacity < cap) {
+      ColumnVectorBatch::resize(cap);
+      data.resize(cap);
+    }
+  }
+
+  DoubleVectorBatch::DoubleVectorBatch(uint64_t capacity, MemoryPool& pool
+                   ): ColumnVectorBatch(capacity, pool),
+                      data(pool, capacity) {
+    // PASS
+  }
+
+  DoubleVectorBatch::~DoubleVectorBatch() {
+    // PASS
+  }
+
+  std::string DoubleVectorBatch::toString() const {
+    std::ostringstream buffer;
+    buffer << "Double vector <" << numElements << " of " << capacity << ">";
+    return buffer.str();
+  }
+
+  void DoubleVectorBatch::resize(uint64_t cap) {
+    if (capacity < cap) {
+      ColumnVectorBatch::resize(cap);
+      data.resize(cap);
+    }
+  }
+
+  StringVectorBatch::StringVectorBatch(uint64_t capacity, MemoryPool& pool
+               ): ColumnVectorBatch(capacity, pool),
+                  data(pool, capacity),
+                  length(pool, capacity) {
+    // PASS
+  }
+
+  StringVectorBatch::~StringVectorBatch() {
+    // PASS
+  }
+
+  std::string StringVectorBatch::toString() const {
+    std::ostringstream buffer;
+    buffer << "Byte vector <" << numElements << " of " << capacity << ">";
+    return buffer.str();
+  }
+
+  void StringVectorBatch::resize(uint64_t cap) {
+    if (capacity < cap) {
+      ColumnVectorBatch::resize(cap);
+      data.resize(cap);
+      length.resize(cap);
+    }
+  }
+
+  StructVectorBatch::StructVectorBatch(uint64_t cap, MemoryPool& pool
+                                        ): ColumnVectorBatch(cap, pool) {
+    // PASS
+  }
+
+  StructVectorBatch::~StructVectorBatch() {
+    for (uint64_t i=0; i<this->fields.size(); i++) {
+      delete this->fields[i];
+    }
+  }
+
+  std::string StructVectorBatch::toString() const {
+    std::ostringstream buffer;
+    buffer << "Struct vector <" << numElements << " of " << capacity
+           << "; ";
+    for(std::vector<ColumnVectorBatch*>::const_iterator ptr=fields.begin();
+        ptr != fields.end(); ++ptr) {
+      buffer << (*ptr)->toString() << "; ";
+    }
+    buffer << ">";
+    return buffer.str();
+  }
+
+
+  void StructVectorBatch::resize(uint64_t cap) {
+    ColumnVectorBatch::resize(cap);
+  }
+
+  ListVectorBatch::ListVectorBatch(uint64_t cap, MemoryPool& pool
+                   ): ColumnVectorBatch(cap, pool),
+                      offsets(pool, cap+1) {
+    // PASS
+  }
+
+  ListVectorBatch::~ListVectorBatch() {
+    // PASS
+  }
+
+  std::string ListVectorBatch::toString() const {
+    std::ostringstream buffer;
+    buffer << "List vector <" << elements->toString() << " with "
+           << numElements << " of " << capacity << ">";
+    return buffer.str();
+  }
+
+  void ListVectorBatch::resize(uint64_t cap) {
+    if (capacity < cap) {
+      ColumnVectorBatch::resize(cap);
+      offsets.resize(cap + 1);
+    }
+  }
+
+  MapVectorBatch::MapVectorBatch(uint64_t cap, MemoryPool& pool
+                 ): ColumnVectorBatch(cap, pool),
+                    offsets(pool, cap+1) {
+    // PASS
+  }
+
+  MapVectorBatch::~MapVectorBatch() {
+    // PASS
+  }
+
+  std::string MapVectorBatch::toString() const {
+    std::ostringstream buffer;
+    buffer << "Map vector <" << keys->toString() << ", "
+           << elements->toString() << " with "
+           << numElements << " of " << capacity << ">";
+    return buffer.str();
+  }
+
+  void MapVectorBatch::resize(uint64_t cap) {
+    if (capacity < cap) {
+      ColumnVectorBatch::resize(cap);
+      offsets.resize(cap + 1);
+    }
+  }
+
+  UnionVectorBatch::UnionVectorBatch(uint64_t cap, MemoryPool& pool
+                                     ): ColumnVectorBatch(cap, pool),
+                                        tags(pool, cap),
+                                        offsets(pool, cap) {
+    // PASS
+  }
+
+  UnionVectorBatch::~UnionVectorBatch() {
+    for (uint64_t i=0; i < children.size(); i++) {
+      delete children[i];
+    }
+  }
+
+  std::string UnionVectorBatch::toString() const {
+    std::ostringstream buffer;
+    buffer << "Union vector <";
+    for(size_t i=0; i < children.size(); ++i) {
+      if (i != 0) {
+        buffer << ", ";
+      }
+      buffer << children[i]->toString();
+    }
+    buffer << "; with " << numElements << " of " << capacity << ">";
+    return buffer.str();
+  }
+
+  void UnionVectorBatch::resize(uint64_t cap) {
+    if (capacity < cap) {
+      ColumnVectorBatch::resize(cap);
+      tags.resize(cap);
+      offsets.resize(cap);
+    }
+  }
+
+  Decimal64VectorBatch::Decimal64VectorBatch(uint64_t cap, MemoryPool& pool
+                 ): ColumnVectorBatch(cap, pool),
+                    values(pool, cap),
+                    readScales(pool, cap) {
+    // PASS
+  }
+
+  Decimal64VectorBatch::~Decimal64VectorBatch() {
+    // PASS
+  }
+
+  std::string Decimal64VectorBatch::toString() const {
+    std::ostringstream buffer;
+    buffer << "Decimal64 vector  with "
+           << numElements << " of " << capacity << ">";
+    return buffer.str();
+  }
+
+  void Decimal64VectorBatch::resize(uint64_t cap) {
+    if (capacity < cap) {
+      ColumnVectorBatch::resize(cap);
+      values.resize(cap);
+      readScales.resize(cap);
+    }
+  }
+
+  Decimal128VectorBatch::Decimal128VectorBatch(uint64_t cap, MemoryPool& pool
+               ): ColumnVectorBatch(cap, pool),
+                  values(pool, cap),
+                  readScales(pool, cap) {
+    // PASS
+  }
+
+  Decimal128VectorBatch::~Decimal128VectorBatch() {
+    // PASS
+  }
+
+  std::string Decimal128VectorBatch::toString() const {
+    std::ostringstream buffer;
+    buffer << "Decimal128 vector  with "
+           << numElements << " of " << capacity << ">";
+    return buffer.str();
+  }
+
+  void Decimal128VectorBatch::resize(uint64_t cap) {
+    if (capacity < cap) {
+      ColumnVectorBatch::resize(cap);
+      values.resize(cap);
+      readScales.resize(cap);
+    }
+  }
+
+  Decimal::Decimal(const Int128& _value,
+                   int32_t _scale): value(_value), scale(_scale) {
+    // PASS
+  }
+
+  Decimal::Decimal(const std::string& str) {
+    std::size_t foundPoint = str.find(".");
+    // no decimal point, it is int
+    if(foundPoint == std::string::npos){
+      value = Int128(str);
+      scale = 0;
+    }else{
+      std::string copy(str);
+      scale = static_cast<int32_t>(str.length() - foundPoint);
+      value = Int128(copy.replace(foundPoint, 1, ""));
+    }
+  }
+
+  std::string Decimal::toString() const {
+    return value.toDecimalString(scale);
+  }
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/Adaptor.hh.in
----------------------------------------------------------------------
diff --git a/c++/src/orc/Adaptor.hh.in b/c++/src/orc/Adaptor.hh.in
deleted file mode 100644
index 5b3a677..0000000
--- a/c++/src/orc/Adaptor.hh.in
+++ /dev/null
@@ -1,92 +0,0 @@
-/**
- * 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 ADAPTER_HH
-#define ADAPTER_HH
-
-#cmakedefine INT64_IS_LL
-#cmakedefine HAS_PREAD
-#cmakedefine HAS_STOLL
-#cmakedefine HAS_DIAGNOSTIC_PUSH
-#cmakedefine HAS_PRE_1970
-#cmakedefine HAS_POST_2038
-
-#include "orc/orc-config.hh"
-#include <string>
-
-#ifndef HAS_STOLL
-  // A poor man's stoll that converts str to a long long int base 10
-  namespace std {
-    int64_t stoll(std::string str);
-  }
-#endif
-
-#ifndef HAS_PREAD
-  ssize_t pread(int fd, void *buf, size_t count, off_t offset);
-#endif
-
-#ifdef INT64_IS_LL
-  #define INT64_FORMAT_STRING "ll"
-#else
-  #define INT64_FORMAT_STRING "l"
-#endif
-
-#ifndef ORC_CXX_HAS_NOEXCEPT
-  #define noexcept ORC_NOEXCEPT
-#endif
-
-#ifndef ORC_CXX_HAS_OVERRIDE
-  #define override ORC_OVERRIDE
-#endif
-
-#ifdef HAS_DIAGNOSTIC_PUSH
-  #ifdef __clang__
-    #define DIAGNOSTIC_PUSH _Pragma("clang diagnostic push")
-    #define DIAGNOSTIC_POP _Pragma("clang diagnostic pop")
-  #elif defined(__GNUC__)
-    #define DIAGNOSTIC_PUSH _Pragma("GCC diagnostic push")
-    #define DIAGNOSTIC_POP _Pragma("GCC diagnostic pop")
-  #else
-    #error("Unknown compiler")
-  #endif
-#else
-  #define DIAGNOSTIC_PUSH
-  #define DIAGNOSTIC_POP
-#endif
-
-#define PRAGMA(TXT) _Pragma(#TXT)
-
-#ifdef __clang__
-  #define DIAGNOSTIC_IGNORE(XXX) PRAGMA(clang diagnostic ignored XXX)
-#elif defined(__GNUC__)
-  #define DIAGNOSTIC_IGNORE(XXX) PRAGMA(GCC diagnostic ignored XXX)
-#else
-  #define DIAGNOSTIC_IGNORE(XXX)
-#endif
-
-#ifndef ORC_CXX_HAS_UNIQUE_PTR
-  #define unique_ptr auto_ptr
-#endif
-
-#ifndef UINT32_MAX
-  #define UINT32_MAX 0xffffffff
-#endif
-
-#define GTEST_LANG_CXX11 0
-
-#endif /* ADAPTER_HH */


[13/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/test/TestColumnReader.cc
----------------------------------------------------------------------
diff --git a/c++/test/TestColumnReader.cc b/c++/test/TestColumnReader.cc
new file mode 100644
index 0000000..4c669d0
--- /dev/null
+++ b/c++/test/TestColumnReader.cc
@@ -0,0 +1,4308 @@
+/**
+ * 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 "ColumnReader.hh"
+#include "Exceptions.hh"
+#include "OrcTest.hh"
+
+#include "wrap/orc-proto-wrapper.hh"
+#include "wrap/gtest-wrapper.h"
+#include "wrap/gmock.h"
+
+#include <iostream>
+#include <vector>
+
+#ifdef __clang__
+  DIAGNOSTIC_IGNORE("-Winconsistent-missing-override")
+#endif
+
+namespace orc {
+
+class MockStripeStreams: public StripeStreams {
+public:
+  ~MockStripeStreams();
+  std::unique_ptr<SeekableInputStream> getStream(int64_t columnId,
+                                                 proto::Stream_Kind kind,
+                                                 bool stream) const override;
+  MOCK_CONST_METHOD0(getReaderOptions, const ReaderOptions&());
+  MOCK_CONST_METHOD0(getSelectedColumns, const std::vector<bool>());
+  MOCK_CONST_METHOD1(getEncoding, proto::ColumnEncoding (int64_t));
+  MOCK_CONST_METHOD3(getStreamProxy, SeekableInputStream*
+                     (int64_t, proto::Stream_Kind, bool));
+  MemoryPool& getMemoryPool() const {
+    return *getDefaultPool();
+  }
+};
+
+MockStripeStreams::~MockStripeStreams() {
+  // PASS
+}
+
+std::unique_ptr<SeekableInputStream>
+MockStripeStreams::getStream(int64_t columnId,
+                             proto::Stream_Kind kind,
+                             bool shouldStream) const {
+  return std::unique_ptr < SeekableInputStream >
+    (getStreamProxy(columnId, kind, shouldStream));
+}
+
+TEST(TestColumnReader, testBooleanWithNulls) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(2, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+  // alternate 4 non-null and 4 null via [0xf0 for x in range(512 / 8)]
+  const unsigned char buffer1[] = { 0x3d, 0xf0 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+
+  // [0x0f for x in range(256 / 8)]
+  const unsigned char buffer2[] = {0x1d, 0x0f};
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer2, ARRAY_SIZE(buffer2))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createPrimitiveType(BOOLEAN), "col0");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
+  LongVectorBatch *longBatch = new LongVectorBatch(1024, *getDefaultPool());
+  StructVectorBatch batch(1024, *getDefaultPool());
+  batch.fields.push_back(longBatch);
+  reader->next(batch, 512, 0);
+  ASSERT_EQ(512, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(512, longBatch->numElements);
+  ASSERT_EQ(true, longBatch->hasNulls);
+  unsigned int next = 0;
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    if (i & 4) {
+      EXPECT_EQ(0, longBatch->notNull[i]) << "Wrong value at " << i;
+    } else {
+      EXPECT_EQ(1, longBatch->notNull[i]) << "Wrong value at " << i;
+      EXPECT_EQ((next++ & 4) != 0, longBatch->data[i])
+          << "Wrong value at " << i;
+    }
+  }
+}
+
+TEST(TestColumnReader, testBooleanSkipsWithNulls) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(2, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+  // alternate 4 non-null and 4 null via [0xf0 for x in range(512 / 8)]
+  const unsigned char buffer1[] = {0x3d, 0xf0};
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+  // [0x0f for x in range(128 / 8)]
+  const unsigned char buffer2[] = {0x1d, 0x0f};
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer2, ARRAY_SIZE(buffer2))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createPrimitiveType(BOOLEAN), "col0");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
+  LongVectorBatch *longBatch = new LongVectorBatch(1024, *getDefaultPool());
+  StructVectorBatch batch(1024, *getDefaultPool());
+  batch.fields.push_back(longBatch);
+  reader->next(batch, 1, 0);
+  ASSERT_EQ(1, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(1, longBatch->numElements);
+  ASSERT_EQ(true, !longBatch->hasNulls);
+  EXPECT_EQ(0, longBatch->data[0]);
+  reader->skip(506);
+  reader->next(batch, 5, 0);
+  ASSERT_EQ(5, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(5, longBatch->numElements);
+  ASSERT_EQ(true, longBatch->hasNulls);
+  EXPECT_EQ(1, longBatch->data[0]);
+  EXPECT_EQ(true, !longBatch->notNull[1]);
+  EXPECT_EQ(true, !longBatch->notNull[2]);
+  EXPECT_EQ(true, !longBatch->notNull[3]);
+  EXPECT_EQ(true, !longBatch->notNull[4]);
+}
+
+TEST(TestColumnReader, testByteWithNulls) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(2, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+  // alternate 4 non-null and 4 null via [0xf0 for x in range(512 / 8)]
+  const unsigned char buffer1[] = {0x3d, 0xf0};
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+
+  // range(256)
+  char buffer[258];
+  buffer[0] = static_cast<char>(0x80);
+  for (unsigned int i = 0; i < 128; ++i) {
+    buffer[i + 1] = static_cast<char>(i);
+  }
+  buffer[129] = static_cast<char>(0x80);
+  for (unsigned int i = 128; i < 256; ++i) {
+    buffer[i + 2] = static_cast<char>(i);
+  }
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer, ARRAY_SIZE(buffer))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createPrimitiveType(BYTE), "col0");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader =
+      buildReader(*rowType, streams);
+  LongVectorBatch *longBatch = new LongVectorBatch(1024, *getDefaultPool());
+  StructVectorBatch batch(1024, *getDefaultPool());
+  batch.fields.push_back(longBatch);
+  reader->next(batch, 512, 0);
+  ASSERT_EQ(512, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(512, longBatch->numElements);
+  ASSERT_EQ(true, longBatch->hasNulls);
+  unsigned int next = 0;
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    if (i & 4) {
+      EXPECT_EQ(0, longBatch->notNull[i]) << "Wrong value at " << i;
+    } else {
+      EXPECT_EQ(1, longBatch->notNull[i]) << "Wrong value at " << i;
+      EXPECT_EQ(static_cast<char>(next++), longBatch->data[i])
+          << "Wrong value at " << i;
+    }
+  }
+}
+
+TEST(TestColumnReader, testByteSkipsWithNulls) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(2, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+  // alternate 4 non-null and 4 null via [0xf0 for x in range(512 / 8)]
+  const unsigned char buffer1[] = {0x3d, 0xf0};
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+
+  // range(256)
+  char buffer[258];
+  buffer[0] = static_cast<char>(0x80);
+  for (unsigned int i = 0; i < 128; ++i) {
+    buffer[i + 1] = static_cast<char>(i);
+  }
+  buffer[129] = static_cast<char>(0x80);
+  for (unsigned int i = 128; i < 256; ++i) {
+    buffer[i + 2] = static_cast<char>(i);
+  }
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer, ARRAY_SIZE(buffer))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createPrimitiveType(BYTE), "col0");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader =
+      buildReader(*rowType, streams);
+  LongVectorBatch *longBatch = new LongVectorBatch(1024, *getDefaultPool());
+  StructVectorBatch batch(1024, *getDefaultPool());
+  batch.fields.push_back(longBatch);
+  reader->next(batch, 1, 0);
+  ASSERT_EQ(1, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(1, longBatch->numElements);
+  ASSERT_EQ(true, !longBatch->hasNulls);
+  EXPECT_EQ(0, longBatch->data[0]);
+  reader->skip(506);
+  reader->next(batch, 5, 0);
+  ASSERT_EQ(5, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(5, longBatch->numElements);
+  ASSERT_EQ(true, longBatch->hasNulls);
+  EXPECT_EQ(static_cast<char>(-1), longBatch->data[0]);
+  EXPECT_EQ(true, !longBatch->notNull[1]);
+  EXPECT_EQ(true, !longBatch->notNull[2]);
+  EXPECT_EQ(true, !longBatch->notNull[3]);
+  EXPECT_EQ(true, !longBatch->notNull[4]);
+}
+
+TEST(TestColumnReader, testIntegerWithNulls) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(2, true);
+  
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+  const unsigned char buffer1[] = {0x16, 0xf0};
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+  const unsigned char buffer2[] = {0x64, 0x01, 0x00};
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer2, ARRAY_SIZE(buffer2))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createPrimitiveType(INT), "myInt" );
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader =
+      buildReader(*rowType, streams);
+  LongVectorBatch *longBatch = new LongVectorBatch(1024, *getDefaultPool());
+  StructVectorBatch batch(1024, *getDefaultPool());
+  batch.fields.push_back(longBatch);
+  reader->next(batch, 200, 0);
+  ASSERT_EQ(200, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(200, longBatch->numElements);
+  ASSERT_EQ(true, longBatch->hasNulls);
+  long next = 0;
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    if (i & 4) {
+      EXPECT_EQ(0, longBatch->notNull[i]);
+    } else {
+      EXPECT_EQ(1, longBatch->notNull[i]);
+      EXPECT_EQ(next++, longBatch->data[i]);
+    }
+  }
+}
+
+TEST(TestColumnReader, testDictionaryWithNulls) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(2, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(0))
+      .WillRepeatedly(testing::Return(directEncoding));
+  proto::ColumnEncoding dictionaryEncoding;
+  dictionaryEncoding.set_kind(proto::ColumnEncoding_Kind_DICTIONARY);
+  dictionaryEncoding.set_dictionarysize(2);
+  EXPECT_CALL(streams, getEncoding(1))
+      .WillRepeatedly(testing::Return(dictionaryEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+  const unsigned char buffer1[] =  { 0x19, 0xf0 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+  const unsigned char buffer2[] = { 0x2f, 0x00, 0x00, 0x2f, 0x00, 0x01 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer2, ARRAY_SIZE(buffer2))));
+  const unsigned char buffer3[] = { 0x4f, 0x52, 0x43, 0x4f, 0x77, 0x65, 0x6e };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DICTIONARY_DATA,
+                                      false))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer3, ARRAY_SIZE(buffer3))));
+  const unsigned char buffer4[] =  { 0x02, 0x01, 0x03 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, false))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer4, ARRAY_SIZE(buffer4))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createPrimitiveType(STRING), "myString");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader =
+      buildReader(*rowType, streams);
+  StringVectorBatch *stringBatch = new StringVectorBatch(1024,
+                                                         *getDefaultPool());
+  StructVectorBatch batch(1024, *getDefaultPool());
+  batch.fields.push_back(stringBatch);
+  reader->next(batch, 200, 0);
+  ASSERT_EQ(200, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(200, stringBatch->numElements);
+  ASSERT_EQ(true, stringBatch->hasNulls);
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    if (i & 4) {
+      EXPECT_EQ(0, stringBatch->notNull[i]);
+    } else {
+      EXPECT_EQ(1, stringBatch->notNull[i]);
+      const char* expected = i < 98 ? "ORC" : "Owen";
+      ASSERT_EQ(strlen(expected), stringBatch->length[i])
+      << "Wrong length at " << i;
+      for (size_t letter = 0; letter < strlen(expected); ++letter) {
+        EXPECT_EQ(expected[letter], stringBatch->data[i][letter])
+            << "Wrong contents at " << i << ", " << letter;
+      }
+    }
+  }
+}
+
+TEST(TestColumnReader, testVarcharDictionaryWithNulls) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(3, true);
+  selectedColumns.push_back(false);
+
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(0))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  proto::ColumnEncoding dictionary2Encoding;
+  dictionary2Encoding.set_kind(proto::ColumnEncoding_Kind_DICTIONARY);
+  dictionary2Encoding.set_dictionarysize(2);
+  EXPECT_CALL(streams, getEncoding(1))
+      .WillRepeatedly(testing::Return(dictionary2Encoding));
+
+  proto::ColumnEncoding dictionary0Encoding;
+  dictionary0Encoding.set_kind(proto::ColumnEncoding_Kind_DICTIONARY);
+  dictionary0Encoding.set_dictionarysize(0);
+  EXPECT_CALL(streams, getEncoding(testing::Ge(2)))
+      .WillRepeatedly(testing::Return(dictionary0Encoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  const unsigned char buffer1[] = { 0x16, 0xff };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+  const unsigned char buffer2[] = { 0x61, 0x00, 0x01, 0x61, 0x00, 0x00 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer2, ARRAY_SIZE(buffer2))));
+  const unsigned char buffer3[] = { 0x4f, 0x52, 0x43, 0x4f, 0x77,
+          0x65, 0x6e };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DICTIONARY_DATA,
+                                      false))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer3, ARRAY_SIZE(buffer3))));
+  const unsigned char buffer4[] = { 0x02, 0x01, 0x03 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, false))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer4, ARRAY_SIZE(buffer4))));
+
+  const unsigned char buffer5[] = {0x16, 0x00};
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer5, ARRAY_SIZE(buffer5))));
+
+  // all three return an empty stream
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer5, 0)));
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DICTIONARY_DATA,
+                                      false))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer5, 0)));
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_LENGTH, false))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer5, 0)));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createPrimitiveType(VARCHAR), "col0");
+  rowType->addStructField(createPrimitiveType(CHAR), "col1");
+  rowType->addStructField(createPrimitiveType(STRING), "col2");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader =
+      buildReader(*rowType, streams);
+  StructVectorBatch batch(1024, *getDefaultPool());
+  StringVectorBatch *stringBatch = new StringVectorBatch(1024,
+                                                         *getDefaultPool());
+  StringVectorBatch *nullBatch = new StringVectorBatch(1024,
+                                                       *getDefaultPool());
+  batch.fields.push_back(stringBatch);
+  batch.fields.push_back(nullBatch);
+  reader->next(batch, 200, 0);
+  ASSERT_EQ(200, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(200, stringBatch->numElements);
+  ASSERT_EQ(true, !stringBatch->hasNulls);
+  ASSERT_EQ(200, nullBatch->numElements);
+  ASSERT_EQ(true, nullBatch->hasNulls);
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    EXPECT_EQ(true, stringBatch->notNull[i]);
+    EXPECT_EQ(true, !nullBatch->notNull[i]);
+    const char* expected = i < 100 ? "Owen" : "ORC";
+    ASSERT_EQ(strlen(expected), stringBatch->length[i])
+    << "Wrong length at " << i;
+    for (size_t letter = 0; letter < strlen(expected); ++letter) {
+      EXPECT_EQ(expected[letter], stringBatch->data[i][letter])
+          << "Wrong contents at " << i << ", " << letter;
+    }
+  }
+}
+
+TEST(TestColumnReader, testSubstructsWithNulls) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(4, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  const unsigned char buffer1[] = {0x16, 0x0f};
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+
+  const unsigned char buffer2[] = { 0x0a, 0x55 };
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer2, ARRAY_SIZE(buffer2))));
+
+  const unsigned char buffer3[] = { 0x04, 0xf0 };
+  EXPECT_CALL(streams, getStreamProxy(3, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer3, ARRAY_SIZE(buffer3))));
+
+  const unsigned char buffer4[] =  { 0x17, 0x01, 0x00 };
+  EXPECT_CALL(streams, getStreamProxy(3, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer4, ARRAY_SIZE(buffer4))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createStructType(), "col0")
+    .addStructField(createStructType(), "col1")
+    .addStructField(createPrimitiveType(LONG), "col2");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
+
+  StructVectorBatch batch(1024, *getDefaultPool());
+  StructVectorBatch *middle = new StructVectorBatch(1024, *getDefaultPool());
+  StructVectorBatch *inner = new StructVectorBatch(1024, *getDefaultPool());
+  LongVectorBatch *longs = new LongVectorBatch(1024, *getDefaultPool());
+  batch.fields.push_back(middle);
+  middle->fields.push_back(inner);
+  inner->fields.push_back(longs);
+  reader->next(batch, 200, 0);
+  ASSERT_EQ(200, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(200, middle->numElements);
+  ASSERT_EQ(true, middle->hasNulls);
+  ASSERT_EQ(200, inner->numElements);
+  ASSERT_EQ(true, inner->hasNulls);
+  ASSERT_EQ(200, longs->numElements);
+  ASSERT_EQ(true, longs->hasNulls);
+  long middleCount = 0;
+  long innerCount = 0;
+  long longCount = 0;
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    if (i & 4) {
+      EXPECT_EQ(true, middle->notNull[i]) << "Wrong at " << i;
+      if (middleCount++ & 1) {
+        EXPECT_EQ(true, inner->notNull[i]) << "Wrong at " << i;
+        if (innerCount++ & 4) {
+          EXPECT_EQ(true, !longs->notNull[i]) << "Wrong at " << i;
+        } else {
+          EXPECT_EQ(true, longs->notNull[i]) << "Wrong at " << i;
+          EXPECT_EQ(longCount++, longs->data[i]) << "Wrong at " << i;
+        }
+      } else {
+        EXPECT_EQ(true, !inner->notNull[i]) << "Wrong at " << i;
+        EXPECT_EQ(true, !longs->notNull[i]) << "Wrong at " << i;
+      }
+    } else {
+      EXPECT_EQ(true, !middle->notNull[i]) << "Wrong at " << i;
+      EXPECT_EQ(true, !inner->notNull[i]) << "Wrong at " << i;
+      EXPECT_EQ(true, !longs->notNull[i]) << "Wrong at " << i;
+    }
+  }
+}
+
+TEST(TestColumnReader, testSkipWithNulls) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(3, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+  proto::ColumnEncoding dictionaryEncoding;
+  dictionaryEncoding.set_kind(proto::ColumnEncoding_Kind_DICTIONARY);
+  dictionaryEncoding.set_dictionarysize(100);
+  EXPECT_CALL(streams, getEncoding(2))
+      .WillRepeatedly(testing::Return(dictionaryEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+  const unsigned char buffer1[] =  { 0x03, 0x00, 0xff, 0x3f, 0x08, 0xff,
+                                     0xff, 0xfc, 0x03, 0x00 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+
+  const unsigned char buffer2[] = { 0x61, 0x01, 0x00 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer2, ARRAY_SIZE(buffer2))));
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer2, ARRAY_SIZE(buffer2))));
+
+  // fill the dictionary with '00' to '99'
+  char digits[200];
+  for (int i = 0; i < 10; ++i) {
+    for (int j = 0; j < 10; ++j) {
+      digits[2 * (10 * i + j)] = static_cast<char>('0' + i);
+      digits[2 * (10 * i + j) + 1] = static_cast<char>('0' + j);
+    }
+  }
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DICTIONARY_DATA,
+                                      false))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (digits, ARRAY_SIZE(digits))));
+  const unsigned char buffer3[] = {0x61, 0x00, 0x02};
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_LENGTH, false))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer3, ARRAY_SIZE(buffer3))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createPrimitiveType(INT), "myInt");
+  rowType->addStructField(createPrimitiveType(STRING), "myString");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader =
+      buildReader(*rowType, streams);
+  StructVectorBatch batch(100, *getDefaultPool());
+  LongVectorBatch *longBatch = new LongVectorBatch(100, *getDefaultPool());
+  StringVectorBatch *stringBatch =
+    new StringVectorBatch(100, *getDefaultPool());
+  batch.fields.push_back(longBatch);
+  batch.fields.push_back(stringBatch);
+  reader->next(batch, 20, 0);
+  ASSERT_EQ(20, batch.numElements);
+  ASSERT_EQ(20, longBatch->numElements);
+  ASSERT_EQ(20, stringBatch->numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(true, longBatch->hasNulls);
+  ASSERT_EQ(true, stringBatch->hasNulls);
+  for (size_t i = 0; i < 20; ++i) {
+    EXPECT_EQ(true, !longBatch->notNull[i]) << "Wrong at " << i;
+    EXPECT_EQ(true, !stringBatch->notNull[i]) << "Wrong at " << i;
+  }
+  reader->skip(30);
+  reader->next(batch, 100, 0);
+  ASSERT_EQ(100, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(true, !longBatch->hasNulls);
+  ASSERT_EQ(true, !stringBatch->hasNulls);
+  for (size_t i = 0; i < 10; ++i) {
+    for (size_t j = 0; j < 10; ++j) {
+      size_t k = 10 * i + j;
+      EXPECT_EQ(1, longBatch->notNull[k]) << "Wrong at " << k;
+      ASSERT_EQ(2, stringBatch->length[k])<< "Wrong at " << k;
+      EXPECT_EQ('0' + static_cast<char>(i), stringBatch->data[k][0])
+          << "Wrong at " << k;
+      EXPECT_EQ('0' + static_cast<char>(j), stringBatch->data[k][1])
+          << "Wrong at " << k;
+    }
+  }
+  reader->skip(50);
+}
+
+TEST(TestColumnReader, testBinaryDirect) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(2, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  char blob[200];
+  for (size_t i = 0; i < 10; ++i) {
+    for (size_t j = 0; j < 10; ++j) {
+      blob[2 * (10 * i + j)] = static_cast<char>(i);
+      blob[2 * (10 * i + j) + 1] = static_cast<char>(j);
+    }
+  }
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (blob, ARRAY_SIZE(blob))));
+
+  const unsigned char buffer[] = { 0x61, 0x00, 0x02 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer, ARRAY_SIZE(buffer))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createPrimitiveType(BINARY), "col0");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
+
+  StructVectorBatch batch(1024, *getDefaultPool());
+  StringVectorBatch *strings = new StringVectorBatch(1024, *getDefaultPool());
+  batch.fields.push_back(strings);
+  for (size_t i = 0; i < 2; ++i) {
+    reader->next(batch, 50, 0);
+    ASSERT_EQ(50, batch.numElements);
+    ASSERT_EQ(true, !batch.hasNulls);
+    ASSERT_EQ(50, strings->numElements);
+    ASSERT_EQ(true, !strings->hasNulls);
+    for (size_t j = 0; j < batch.numElements; ++j) {
+      ASSERT_EQ(2, strings->length[j]);
+      EXPECT_EQ((50 * i + j) / 10, strings->data[j][0]);
+      EXPECT_EQ((50 * i + j) % 10, strings->data[j][1]);
+    }
+  }
+}
+
+TEST(TestColumnReader, testBinaryDirectWithNulls) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(2, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  const unsigned char buffer1[] = {0x1d, 0xf0};
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+
+  char blob[256];
+  for (size_t i = 0; i < 8; ++i) {
+    for (size_t j = 0; j < 16; ++j) {
+      blob[2 * (16 * i + j)] = static_cast<char>('A' + i);
+      blob[2 * (16 * i + j) + 1] = static_cast<char>('A' + j);
+    }
+  }
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (blob, ARRAY_SIZE(blob))));
+
+  const unsigned char buffer2[] = {0x7d, 0x00, 0x02};
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer2, ARRAY_SIZE(buffer2))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createPrimitiveType(BINARY), "col0");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
+
+  StructVectorBatch batch(1024, *getDefaultPool());
+  StringVectorBatch *strings = new StringVectorBatch(1024, *getDefaultPool());
+  batch.fields.push_back(strings);
+  size_t next = 0;
+  for (size_t i = 0; i < 2; ++i) {
+    reader->next(batch, 128, 0);
+    ASSERT_EQ(128, batch.numElements);
+    ASSERT_EQ(true, !batch.hasNulls);
+    ASSERT_EQ(128, strings->numElements);
+    ASSERT_EQ(true, strings->hasNulls);
+    for (size_t j = 0; j < batch.numElements; ++j) {
+      ASSERT_EQ(((128 * i + j) & 4) == 0, strings->notNull[j]);
+      if (strings->notNull[j]) {
+        ASSERT_EQ(2, strings->length[j]);
+        EXPECT_EQ('A' + static_cast<char>(next / 16), strings->data[j][0]);
+        EXPECT_EQ('A' + static_cast<char>(next % 16), strings->data[j][1]);
+        next += 1;
+      }
+    }
+  }
+}
+
+TEST(TestColumnReader, testShortBlobError) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(2, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  char blob[100];
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (blob, ARRAY_SIZE(blob))));
+  
+  const unsigned char buffer1[] = {0x61, 0x00, 0x02};
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createPrimitiveType(STRING), "col0");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
+
+  StructVectorBatch batch(1024, *getDefaultPool());
+  StringVectorBatch *strings = new StringVectorBatch(1024, *getDefaultPool());
+  batch.fields.push_back(strings);
+  EXPECT_THROW(reader->next(batch, 100, 0), ParseError);
+}
+
+TEST(TestColumnReader, testStringDirectShortBuffer) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(2, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  char blob[200];
+  for (size_t i = 0; i < 10; ++i) {
+    for (size_t j = 0; j < 10; ++j) {
+      blob[2 * (10 * i + j)] = static_cast<char>(i);
+      blob[2 * (10 * i + j) + 1] = static_cast<char>(j);
+    }
+  }
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (blob, ARRAY_SIZE(blob), 3)));
+
+  const unsigned char buffer1[] = {0x61, 0x00, 0x02};
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createPrimitiveType(STRING), "col0");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
+
+  StructVectorBatch batch(25, *getDefaultPool());
+  StringVectorBatch *strings = new StringVectorBatch(25, *getDefaultPool());
+  batch.fields.push_back(strings);
+  for (size_t i = 0; i < 4; ++i) {
+    reader->next(batch, 25, 0);
+    ASSERT_EQ(25, batch.numElements);
+    ASSERT_EQ(true, !batch.hasNulls);
+    ASSERT_EQ(25, strings->numElements);
+    ASSERT_EQ(true, !strings->hasNulls);
+    for (size_t j = 0; j < batch.numElements; ++j) {
+      ASSERT_EQ(2, strings->length[j]);
+      EXPECT_EQ((25 * i + j) / 10, strings->data[j][0]);
+      EXPECT_EQ((25 * i + j) % 10, strings->data[j][1]);
+    }
+  }
+}
+
+TEST(TestColumnReader, testStringDirectShortBufferWithNulls) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(2, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  const unsigned char buffer1[] = {0x3d, 0xf0};
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+
+  char blob[512];
+  for (size_t i = 0; i < 16; ++i) {
+    for (size_t j = 0; j < 16; ++j) {
+      blob[2 * (16 * i + j)] = static_cast<char>('A' + i);
+      blob[2 * (16 * i + j) + 1] = static_cast<char>('A' + j);
+    }
+  }
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (blob, ARRAY_SIZE(blob), 30)));
+
+  const unsigned char buffer2[] =  { 0x7d, 0x00, 0x02, 0x7d, 0x00, 0x02 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer2, ARRAY_SIZE(buffer2))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createPrimitiveType(STRING), "col0");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
+
+  StructVectorBatch batch(64, *getDefaultPool());
+  StringVectorBatch *strings = new StringVectorBatch(64, *getDefaultPool());
+  batch.fields.push_back(strings);
+  size_t next = 0;
+  for (size_t i = 0; i < 8; ++i) {
+    reader->next(batch, 64, 0);
+    ASSERT_EQ(64, batch.numElements);
+    ASSERT_EQ(true, !batch.hasNulls);
+    ASSERT_EQ(64, strings->numElements);
+    ASSERT_EQ(true, strings->hasNulls);
+    for (size_t j = 0; j < batch.numElements; ++j) {
+      ASSERT_EQ((j & 4) == 0, strings->notNull[j]);
+      if (strings->notNull[j]) {
+        ASSERT_EQ(2, strings->length[j]);
+        EXPECT_EQ('A' + next / 16, strings->data[j][0]);
+        EXPECT_EQ('A' + next % 16, strings->data[j][1]);
+        next += 1;
+      }
+    }
+  }
+}
+
+TEST(TestColumnReader, testStringDirectSkip) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(2, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  // sum(0 to 1199)
+  const size_t BLOB_SIZE = 719400;
+  char blob[BLOB_SIZE];
+  size_t posn = 0;
+  for (size_t item = 0; item < 1200; ++item) {
+    for (size_t ch = 0; ch < item; ++ch) {
+      blob[posn++] = static_cast<char>(ch);
+    }
+  }
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+      (blob, BLOB_SIZE, 200)));
+
+  // the stream of 0 to 1199
+  const unsigned char buffer1[] = 
+    { 0x7f, 0x01, 0x00,
+      0x7f, 0x01, 0x82, 0x01,
+      0x7f, 0x01, 0x84, 0x02,
+      0x7f, 0x01, 0x86, 0x03,
+      0x7f, 0x01, 0x88, 0x04,
+      0x7f, 0x01, 0x8a, 0x05,
+      0x7f, 0x01, 0x8c, 0x06,
+      0x7f, 0x01, 0x8e, 0x07,
+      0x7f, 0x01, 0x90, 0x08,
+      0x1b, 0x01, 0x92, 0x09 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createPrimitiveType(STRING), "col0");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
+
+  StructVectorBatch batch(2, *getDefaultPool());
+  StringVectorBatch *strings = new StringVectorBatch(2, *getDefaultPool());
+  batch.fields.push_back(strings);
+  reader->next(batch, 2, 0);
+  ASSERT_EQ(2, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(2, strings->numElements);
+  ASSERT_EQ(true, !strings->hasNulls);
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    ASSERT_EQ(i, strings->length[i]);
+    for (size_t j = 0; j < i; ++j) {
+      EXPECT_EQ(static_cast<char>(j), strings->data[i][j]);
+    }
+  }
+  reader->skip(14);
+  reader->next(batch, 2, 0);
+  ASSERT_EQ(2, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(2, strings->numElements);
+  ASSERT_EQ(true, !strings->hasNulls);
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    ASSERT_EQ(16 + i, strings->length[i]);
+    for (size_t j = 0; j < 16 + i; ++j) {
+      EXPECT_EQ(static_cast<char>(j), strings->data[i][j]);
+    }
+  }
+  reader->skip(1180);
+  reader->next(batch, 2, 0);
+  ASSERT_EQ(2, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(2, strings->numElements);
+  ASSERT_EQ(true, !strings->hasNulls);
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    ASSERT_EQ(1198 + i, strings->length[i]);
+    for (size_t j = 0; j < 1198 + i; ++j) {
+      EXPECT_EQ(static_cast<char>(j), strings->data[i][j]);
+    }
+  }
+}
+
+TEST(TestColumnReader, testStringDirectSkipWithNulls) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(2, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  // alternate 4 non-null and 4 null via [0xf0 for x in range(2400 / 8)]
+  const unsigned char buffer1[] =  { 0x7f, 0xf0, 0x7f, 0xf0, 0x25, 0xf0 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+
+  // sum(range(1200))
+  const size_t BLOB_SIZE = 719400;
+
+  // each string is [x % 256 for x in range(r)]
+  char blob[BLOB_SIZE];
+  size_t posn = 0;
+  for (size_t item = 0; item < 1200; ++item) {
+    for (size_t ch = 0; ch < item; ++ch) {
+      blob[posn++] = static_cast<char>(ch);
+    }
+  }
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+      (blob, BLOB_SIZE, 200)));
+
+  // range(1200)
+  const unsigned char buffer2[] =  { 0x7f, 0x01, 0x00,
+                                     0x7f, 0x01, 0x82, 0x01,
+                                     0x7f, 0x01, 0x84, 0x02,
+                                     0x7f, 0x01, 0x86, 0x03,
+                                     0x7f, 0x01, 0x88, 0x04,
+                                     0x7f, 0x01, 0x8a, 0x05,
+                                     0x7f, 0x01, 0x8c, 0x06,
+                                     0x7f, 0x01, 0x8e, 0x07,
+                                     0x7f, 0x01, 0x90, 0x08,
+                                     0x1b, 0x01, 0x92, 0x09 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer2, ARRAY_SIZE(buffer2))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createPrimitiveType(STRING), "col0");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
+
+  StructVectorBatch batch(2, *getDefaultPool());
+  StringVectorBatch *strings = new StringVectorBatch(2, *getDefaultPool());
+  batch.fields.push_back(strings);
+  reader->next(batch, 2, 0);
+  ASSERT_EQ(2, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(2, strings->numElements);
+  ASSERT_EQ(true, !strings->hasNulls);
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    ASSERT_EQ(i, strings->length[i]);
+    for (size_t j = 0; j < i; ++j) {
+      EXPECT_EQ(static_cast<char>(j), strings->data[i][j]);
+    }
+  }
+  reader->skip(30);
+  reader->next(batch, 2, 0);
+  ASSERT_EQ(2, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(2, strings->numElements);
+  ASSERT_EQ(true, !strings->hasNulls);
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    ASSERT_EQ(16 + i, strings->length[i]);
+    for (size_t j = 0; j < 16 + i; ++j) {
+      EXPECT_EQ(static_cast<char>(j), strings->data[i][j]);
+    }
+  }
+  reader->skip(2364);
+  reader->next(batch, 2, 0);
+  ASSERT_EQ(2, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(2, strings->numElements);
+  ASSERT_EQ(true, strings->hasNulls);
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    EXPECT_EQ(true, !strings->notNull[i]);
+  }
+}
+
+TEST(TestColumnReader, testList) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(3, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(testing::_,
+          proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  // [2 for x in range(600)]
+  const unsigned char buffer1[] =  { 0x7f, 0x00, 0x02,
+                                     0x7f, 0x00, 0x02,
+                                     0x7f, 0x00, 0x02,
+                                     0x7f, 0x00, 0x02,
+                                     0x4d, 0x00, 0x02 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+
+  // range(1200)
+  const unsigned char buffer2[] = { 0x7f, 0x01, 0x00,
+                                    0x7f, 0x01, 0x84, 0x02,
+                                    0x7f, 0x01, 0x88, 0x04,
+                                    0x7f, 0x01, 0x8c, 0x06,
+                                    0x7f, 0x01, 0x90, 0x08,
+                                    0x7f, 0x01, 0x94, 0x0a,
+                                    0x7f, 0x01, 0x98, 0x0c,
+                                    0x7f, 0x01, 0x9c, 0x0e,
+                                    0x7f, 0x01, 0xa0, 0x10,
+                                    0x1b, 0x01, 0xa4, 0x12 };
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer2, ARRAY_SIZE(buffer2))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createListType(createPrimitiveType(LONG)), "col0");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
+
+  StructVectorBatch batch(512, *getDefaultPool());
+  ListVectorBatch *lists = new ListVectorBatch(512, *getDefaultPool());
+  LongVectorBatch *longs = new LongVectorBatch(512, *getDefaultPool());
+  batch.fields.push_back(lists);
+  lists->elements = std::unique_ptr < ColumnVectorBatch > (longs);
+  reader->next(batch, 512, 0);
+  ASSERT_EQ(512, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(512, lists->numElements);
+  ASSERT_EQ(true, !lists->hasNulls);
+  ASSERT_EQ(1024, longs->numElements);
+  ASSERT_EQ(true, !longs->hasNulls);
+  for (size_t i = 0; i <= batch.numElements; ++i) {
+    EXPECT_EQ(2 * i, lists->offsets[i]);
+  }
+  for (size_t i = 0; i < longs->numElements; ++i) {
+    EXPECT_EQ(i, longs->data[i]);
+  }
+}
+
+TEST(TestColumnReader, testListPropagateNulls) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(4, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createStructType(), "col0")
+    .addStructField(createListType(createPrimitiveType(LONG)), "col0_0");
+  rowType->assignIds(0);
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(testing::_,
+          proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  // set getStream
+  const unsigned char buffer[] = {0xff, 0x00};
+  EXPECT_CALL(streams, getStreamProxy(1,proto::Stream_Kind_PRESENT, true))
+    .WillRepeatedly(testing::Return
+                    (new SeekableArrayInputStream(buffer,
+                                                  ARRAY_SIZE(buffer))));
+
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_LENGTH, true))
+                    .WillRepeatedly(testing::Return
+                                    (new SeekableArrayInputStream(buffer, 0)));
+
+  EXPECT_CALL(streams, getStreamProxy(3, proto::Stream_Kind_DATA, true))
+                    .WillRepeatedly(testing::Return
+                                    (new SeekableArrayInputStream(buffer, 0)));
+
+  // create the row type
+  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
+
+  StructVectorBatch batch(512, *getDefaultPool());
+  StructVectorBatch *structs = new StructVectorBatch(512, *getDefaultPool());
+  ListVectorBatch *lists = new ListVectorBatch(512, *getDefaultPool());
+  LongVectorBatch *longs = new LongVectorBatch(512, *getDefaultPool());
+  batch.fields.push_back(structs);
+  structs->fields.push_back(lists);
+  lists->elements = std::unique_ptr < ColumnVectorBatch > (longs);
+  reader->next(batch, 8, 0);
+  ASSERT_EQ(8, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(8, structs->numElements);
+  ASSERT_EQ(true, structs->hasNulls);
+  ASSERT_EQ(8, lists->numElements);
+  ASSERT_EQ(true, lists->hasNulls);
+  ASSERT_EQ(0, longs->numElements);
+  ASSERT_EQ(true, !longs->hasNulls);
+  for(size_t i=0; i < 8; ++i) {
+    EXPECT_EQ(true, !structs->notNull[i]);
+  }
+}
+
+TEST(TestColumnReader, testListWithNulls) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(3, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  // [0xaa for x in range(2048/8)]
+  const unsigned char buffer1[] =  { 0x7f, 0xaa, 0x7b, 0xaa };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  // [1 for x in range(260)] +
+  // [4 for x in range(260)] +
+  // [0 for x in range(260)] +
+  // [3 for x in range(243)] +
+  // [19]
+  const unsigned char buffer2[] =  { 0x7f, 0x00, 0x01,
+                                     0x7f, 0x00, 0x01,
+                                     0x7f, 0x00, 0x04,
+                                     0x7f, 0x00, 0x04,
+                                     0x7f, 0x00, 0x00,
+                                     0x7f, 0x00, 0x00,
+                                     0x7f, 0x00, 0x03,
+                                     0x6e, 0x00, 0x03,
+                                     0xff, 0x13 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer2, ARRAY_SIZE(buffer2))));
+
+  // range(2048)
+  const unsigned char buffer3[] = { 0x7f, 0x01, 0x00,
+                                    0x7f, 0x01, 0x84, 0x02,
+                                    0x7f, 0x01, 0x88, 0x04,
+                                    0x7f, 0x01, 0x8c, 0x06,
+                                    0x7f, 0x01, 0x90, 0x08,
+                                    0x7f, 0x01, 0x94, 0x0a,
+                                    0x7f, 0x01, 0x98, 0x0c,
+                                    0x7f, 0x01, 0x9c, 0x0e,
+                                    0x7f, 0x01, 0xa0, 0x10,
+                                    0x7f, 0x01, 0xa4, 0x12,
+                                    0x7f, 0x01, 0xa8, 0x14,
+                                    0x7f, 0x01, 0xac, 0x16,
+                                    0x7f, 0x01, 0xb0, 0x18,
+                                    0x7f, 0x01, 0xb4, 0x1a,
+                                    0x7f, 0x01, 0xb8, 0x1c,
+                                    0x5f, 0x01, 0xbc, 0x1e };
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer3, ARRAY_SIZE(buffer3))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createListType(createPrimitiveType(LONG)), "col0");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
+
+  StructVectorBatch batch(512, *getDefaultPool());
+  ListVectorBatch *lists = new ListVectorBatch(512, *getDefaultPool());
+  LongVectorBatch *longs = new LongVectorBatch(512, *getDefaultPool());
+  batch.fields.push_back(lists);
+  lists->elements = std::unique_ptr < ColumnVectorBatch > (longs);
+  reader->next(batch, 512, 0);
+  ASSERT_EQ(512, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(512, lists->numElements);
+  ASSERT_EQ(true, lists->hasNulls);
+  ASSERT_EQ(256, longs->numElements);
+  ASSERT_EQ(true, !longs->hasNulls);
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    EXPECT_EQ(i % 2 == 0, lists->notNull[i]) << "Wrong value at " << i;
+    EXPECT_EQ((i + 1) / 2, lists->offsets[i]) << "Wrong value at " << i;
+  }
+  EXPECT_EQ(256, lists->offsets[512]);
+  for (size_t i = 0; i < longs->numElements; ++i) {
+    EXPECT_EQ(i, longs->data[i]);
+  }
+
+  reader->next(batch, 512, 0);
+  ASSERT_EQ(512, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(512, lists->numElements);
+  ASSERT_EQ(true, lists->hasNulls);
+  ASSERT_EQ(1012, longs->numElements);
+  ASSERT_EQ(true, !longs->hasNulls);
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    EXPECT_EQ(i % 2 == 0, lists->notNull[i]) << "Wrong value at " << i;
+    if (i < 8) {
+      EXPECT_EQ((i + 1) / 2, lists->offsets[i])
+          << "Wrong value at " << i;
+    } else {
+      EXPECT_EQ(4 * ((i + 1) / 2) - 12, lists->offsets[i])
+          << "Wrong value at " << i;
+    }
+  }
+  EXPECT_EQ(1012, lists->offsets[512]);
+  for (size_t i = 0; i < longs->numElements; ++i) {
+    EXPECT_EQ(256 + i, longs->data[i]);
+  }
+
+  reader->next(batch, 512, 0);
+  ASSERT_EQ(512, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(512, lists->numElements);
+  ASSERT_EQ(true, lists->hasNulls);
+  ASSERT_EQ(32, longs->numElements);
+  ASSERT_EQ(true, !longs->hasNulls);
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    EXPECT_EQ(i % 2 == 0, lists->notNull[i]) << "Wrong value at " << i;
+    if (i < 16) {
+      EXPECT_EQ(4 * ((i + 1) / 2), lists->offsets[i])
+          << "Wrong value at " << i;
+    } else {
+      EXPECT_EQ(32, lists->offsets[i]) << "Wrong value at " << i;
+    }
+  }
+  EXPECT_EQ(32, lists->offsets[512]);
+  for (size_t i = 0; i < longs->numElements; ++i) {
+    EXPECT_EQ(1268 + i, longs->data[i]);
+  }
+
+  reader->next(batch, 512, 0);
+  ASSERT_EQ(512, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(512, lists->numElements);
+  ASSERT_EQ(true, lists->hasNulls);
+  ASSERT_EQ(748, longs->numElements);
+  ASSERT_EQ(true, !longs->hasNulls);
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    EXPECT_EQ(i % 2 == 0, lists->notNull[i]) << "Wrong value at " << i;
+    if (i < 24) {
+      EXPECT_EQ(0, lists->offsets[i]) << "Wrong value at " << i;
+    } else if (i < 510) {
+      EXPECT_EQ(3 * ((i - 23) / 2), lists->offsets[i])
+          << "Wrong value at " << i;
+    } else if (i < 511) {
+      EXPECT_EQ(729, lists->offsets[i]) << "Wrong value at " << i;
+    } else {
+      EXPECT_EQ(748, lists->offsets[i]) << "Wrong value at " << i;
+    }
+  }
+  EXPECT_EQ(748, lists->offsets[512]);
+  for (size_t i = 0; i < longs->numElements; ++i) {
+    EXPECT_EQ(1300 + i, longs->data[i]);
+  }
+}
+
+TEST(TestColumnReader, testListSkipWithNulls) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(3, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  // [0xaa for x in range(2048/8)]
+  const unsigned char buffer1[] =  { 0x7f, 0xaa, 0x7b, 0xaa };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  // [1 for x in range(260)] +
+  // [4 for x in range(260)] +
+  // [0 for x in range(260)] +
+  // [3 for x in range(243)] +
+  // [19]
+  const unsigned char buffer2[] = { 0x7f, 0x00, 0x01,
+                                    0x7f, 0x00, 0x01,
+                                    0x7f, 0x00, 0x04,
+                                    0x7f, 0x00, 0x04,
+                                    0x7f, 0x00, 0x00,
+                                    0x7f, 0x00, 0x00,
+                                    0x7f, 0x00, 0x03,
+                                    0x6e, 0x00, 0x03,
+                                    0xff, 0x13 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer2, ARRAY_SIZE(buffer2))));
+
+  // range(2048)
+  const unsigned char buffer3[] = { 0x7f, 0x01, 0x00,
+                                    0x7f, 0x01, 0x84, 0x02,
+                                    0x7f, 0x01, 0x88, 0x04,
+                                    0x7f, 0x01, 0x8c, 0x06,
+                                    0x7f, 0x01, 0x90, 0x08,
+                                    0x7f, 0x01, 0x94, 0x0a,
+                                    0x7f, 0x01, 0x98, 0x0c,
+                                    0x7f, 0x01, 0x9c, 0x0e,
+                                    0x7f, 0x01, 0xa0, 0x10,
+                                    0x7f, 0x01, 0xa4, 0x12,
+                                    0x7f, 0x01, 0xa8, 0x14,
+                                    0x7f, 0x01, 0xac, 0x16,
+                                    0x7f, 0x01, 0xb0, 0x18,
+                                    0x7f, 0x01, 0xb4, 0x1a,
+                                    0x7f, 0x01, 0xb8, 0x1c,
+                                    0x5f, 0x01, 0xbc, 0x1e };
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer3, ARRAY_SIZE(buffer3))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createListType(createPrimitiveType(LONG)), "col0");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
+
+  StructVectorBatch batch(1, *getDefaultPool());
+  ListVectorBatch *lists = new ListVectorBatch(1, *getDefaultPool());
+  LongVectorBatch *longs = new LongVectorBatch(1, *getDefaultPool());
+  batch.fields.push_back(lists);
+  lists->elements = std::unique_ptr < ColumnVectorBatch > (longs);
+
+  reader->next(batch, 1, 0);
+  ASSERT_EQ(1, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(1, lists->numElements);
+  ASSERT_EQ(true, !lists->hasNulls);
+  ASSERT_EQ(1, longs->numElements);
+  ASSERT_EQ(true, !longs->hasNulls);
+  EXPECT_EQ(0, lists->offsets[0]);
+  EXPECT_EQ(1, lists->offsets[1]);
+  EXPECT_EQ(0, longs->data[0]);
+
+  reader->skip(13);
+  reader->next(batch, 1, 0);
+  ASSERT_EQ(1, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(1, lists->numElements);
+  ASSERT_EQ(true, !lists->hasNulls);
+  ASSERT_EQ(1, longs->numElements);
+  ASSERT_EQ(true, !longs->hasNulls);
+  EXPECT_EQ(0, lists->offsets[0]);
+  EXPECT_EQ(1, lists->offsets[1]);
+  EXPECT_EQ(7, longs->data[0]);
+
+  reader->skip(2031);
+  reader->next(batch, 2, 0);
+  ASSERT_EQ(2, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(2, lists->numElements);
+  ASSERT_EQ(true, lists->hasNulls);
+  ASSERT_EQ(19, longs->numElements);
+  ASSERT_EQ(true, !longs->hasNulls);
+  EXPECT_EQ(0, lists->offsets[0]);
+  EXPECT_EQ(19, lists->offsets[1]);
+  EXPECT_EQ(19, lists->offsets[2]);
+  for (size_t i = 0; i < longs->numElements; ++i) {
+    EXPECT_EQ(2029 + i, longs->data[i]);
+  }
+}
+
+TEST(TestColumnReader, testListSkipWithNullsNoData) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(2, true);
+  selectedColumns.push_back(false);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  // [0xaa for x in range(2048/8)]
+  const unsigned char buffer1[] = { 0x7f, 0xaa, 0x7b, 0xaa };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  // [1 for x in range(260)] +
+  // [4 for x in range(260)] +
+  // [0 for x in range(260)] +
+  // [3 for x in range(243)] +
+  // [19]
+  const unsigned char buffer2[] =  { 0x7f, 0x00, 0x01,
+                                     0x7f, 0x00, 0x01,
+                                     0x7f, 0x00, 0x04,
+                                     0x7f, 0x00, 0x04,
+                                     0x7f, 0x00, 0x00,
+                                     0x7f, 0x00, 0x00,
+                                     0x7f, 0x00, 0x03,
+                                     0x6e, 0x00, 0x03,
+                                     0xff, 0x13 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer2, ARRAY_SIZE(buffer2))));
+
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createListType(createPrimitiveType(LONG)), "col0");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
+
+  StructVectorBatch batch(1, *getDefaultPool());
+  ListVectorBatch *lists = new ListVectorBatch(1, *getDefaultPool());
+  batch.fields.push_back(lists);
+
+  reader->next(batch, 1, 0);
+  ASSERT_EQ(1, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(1, lists->numElements);
+  ASSERT_EQ(true, !lists->hasNulls);
+  EXPECT_EQ(0, lists->offsets[0]);
+  EXPECT_EQ(1, lists->offsets[1]);
+
+  reader->skip(13);
+  reader->next(batch, 1, 0);
+  ASSERT_EQ(1, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(1, lists->numElements);
+  ASSERT_EQ(true, !lists->hasNulls);
+  EXPECT_EQ(0, lists->offsets[0]);
+  EXPECT_EQ(1, lists->offsets[1]);
+
+  reader->skip(2031);
+  reader->next(batch, 2, 0);
+  ASSERT_EQ(2, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(2, lists->numElements);
+  ASSERT_EQ(true, lists->hasNulls);
+  EXPECT_EQ(0, lists->offsets[0]);
+  EXPECT_EQ(19, lists->offsets[1]);
+  EXPECT_EQ(19, lists->offsets[2]);
+}
+
+TEST(TestColumnReader, testMap) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(4, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(testing::_,
+          proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  // [2 for x in range(600)]
+  const unsigned char buffer1[] = { 0x7f, 0x00, 0x02,
+                                    0x7f, 0x00, 0x02,
+                                    0x7f, 0x00, 0x02,
+                                    0x7f, 0x00, 0x02,
+                                    0x4d, 0x00, 0x02 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+
+  // range(1200)
+  const unsigned char buffer2[] =  { 0x7f, 0x01, 0x00,
+                                     0x7f, 0x01, 0x84, 0x02,
+                                     0x7f, 0x01, 0x88, 0x04,
+                                     0x7f, 0x01, 0x8c, 0x06,
+                                     0x7f, 0x01, 0x90, 0x08,
+                                     0x7f, 0x01, 0x94, 0x0a,
+                                     0x7f, 0x01, 0x98, 0x0c,
+                                     0x7f, 0x01, 0x9c, 0x0e,
+                                     0x7f, 0x01, 0xa0, 0x10,
+                                     0x1b, 0x01, 0xa4, 0x12 };
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer2, ARRAY_SIZE(buffer2))));
+
+  // range(8, 1208)
+  const unsigned char buffer3[] =  { 0x7f, 0x01, 0x10,
+                                     0x7f, 0x01, 0x94, 0x02,
+                                     0x7f, 0x01, 0x98, 0x04,
+                                     0x7f, 0x01, 0x9c, 0x06,
+                                     0x7f, 0x01, 0xa0, 0x08,
+                                     0x7f, 0x01, 0xa4, 0x0a,
+                                     0x7f, 0x01, 0xa8, 0x0c,
+                                     0x7f, 0x01, 0xac, 0x0e,
+                                     0x7f, 0x01, 0xb0, 0x10,
+                                     0x1b, 0x01, 0xb4, 0x12 };
+  EXPECT_CALL(streams, getStreamProxy(3, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer3, ARRAY_SIZE(buffer3))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createMapType(createPrimitiveType(LONG),
+                                        createPrimitiveType(LONG)),
+                          "col0");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
+
+  StructVectorBatch batch(512, *getDefaultPool());
+  MapVectorBatch *maps = new MapVectorBatch(512, *getDefaultPool());
+  LongVectorBatch *keys = new LongVectorBatch(512, *getDefaultPool());
+  LongVectorBatch *elements = new LongVectorBatch(512, *getDefaultPool());
+  batch.fields.push_back(maps);
+  maps->keys = std::unique_ptr < ColumnVectorBatch > (keys);
+  maps->elements = std::unique_ptr < ColumnVectorBatch > (elements);
+  reader->next(batch, 512, 0);
+  ASSERT_EQ(512, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(512, maps->numElements);
+  ASSERT_EQ(true, !maps->hasNulls);
+  ASSERT_EQ(1024, keys->numElements);
+  ASSERT_EQ(true, !keys->hasNulls);
+  ASSERT_EQ(1024, elements->numElements);
+  ASSERT_EQ(true, !elements->hasNulls);
+  for (size_t i = 0; i <= batch.numElements; ++i) {
+    EXPECT_EQ(2 * i, maps->offsets[i]);
+  }
+  for (size_t i = 0; i < keys->numElements; ++i) {
+    EXPECT_EQ(i, keys->data[i]);
+    EXPECT_EQ(i + 8, elements->data[i]);
+  }
+}
+
+TEST(TestColumnReader, testMapWithNulls) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(4, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  // [0xaa for x in range(2048/8)]
+  const unsigned char buffer1[] = { 0x7f, 0xaa, 0x7b, 0xaa };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  // [0x55 for x in range(2048/8)]
+  const unsigned char buffer2[] =  { 0x7f, 0x55, 0x7b, 0x55 };
+  EXPECT_CALL(streams, getStreamProxy(3, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer2, ARRAY_SIZE(buffer2))));
+
+  // [1 for x in range(260)] +
+  // [4 for x in range(260)] +
+  // [0 for x in range(260)] +
+  // [3 for x in range(243)] +
+  // [19]
+  const unsigned char buffer3[] = { 0x7f, 0x00, 0x01,
+                                    0x7f, 0x00, 0x01,
+                                    0x7f, 0x00, 0x04,
+                                    0x7f, 0x00, 0x04,
+                                    0x7f, 0x00, 0x00,
+                                    0x7f, 0x00, 0x00,
+                                    0x7f, 0x00, 0x03,
+                                    0x6e, 0x00, 0x03,
+                                    0xff, 0x13 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer3, ARRAY_SIZE(buffer3))));
+
+  // range(2048)
+  const unsigned char buffer4[] =  { 0x7f, 0x01, 0x00,
+                                     0x7f, 0x01, 0x84, 0x02,
+                                     0x7f, 0x01, 0x88, 0x04,
+                                     0x7f, 0x01, 0x8c, 0x06,
+                                     0x7f, 0x01, 0x90, 0x08,
+                                     0x7f, 0x01, 0x94, 0x0a,
+                                     0x7f, 0x01, 0x98, 0x0c,
+                                     0x7f, 0x01, 0x9c, 0x0e,
+                                     0x7f, 0x01, 0xa0, 0x10,
+                                     0x7f, 0x01, 0xa4, 0x12,
+                                     0x7f, 0x01, 0xa8, 0x14,
+                                     0x7f, 0x01, 0xac, 0x16,
+                                     0x7f, 0x01, 0xb0, 0x18,
+                                     0x7f, 0x01, 0xb4, 0x1a,
+                                     0x7f, 0x01, 0xb8, 0x1c,
+                                     0x5f, 0x01, 0xbc, 0x1e };
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer4, ARRAY_SIZE(buffer4))));
+
+  // range(8, 1032)
+  const unsigned char buffer5[] = { 0x7f, 0x01, 0x10,
+                                    0x7f, 0x01, 0x94, 0x02,
+                                    0x7f, 0x01, 0x98, 0x04,
+                                    0x7f, 0x01, 0x9c, 0x06,
+                                    0x7f, 0x01, 0xa0, 0x08,
+                                    0x7f, 0x01, 0xa4, 0x0a,
+                                    0x7f, 0x01, 0xa8, 0x0c,
+                                    0x6f, 0x01, 0xac, 0x0e };
+  EXPECT_CALL(streams, getStreamProxy(3, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer5, ARRAY_SIZE(buffer5))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createMapType(createPrimitiveType(LONG),
+                                        createPrimitiveType(LONG)),
+                          "col0");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
+
+  StructVectorBatch batch(512, *getDefaultPool());
+  MapVectorBatch *maps = new MapVectorBatch(512, *getDefaultPool());
+  LongVectorBatch *keys = new LongVectorBatch(512, *getDefaultPool());
+  LongVectorBatch *elements = new LongVectorBatch(512, *getDefaultPool());
+  batch.fields.push_back(maps);
+  maps->keys = std::unique_ptr < ColumnVectorBatch > (keys);
+  maps->elements = std::unique_ptr < ColumnVectorBatch > (elements);
+  reader->next(batch, 512, 0);
+  ASSERT_EQ(512, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(512, maps->numElements);
+  ASSERT_EQ(true, maps->hasNulls);
+  ASSERT_EQ(256, keys->numElements);
+  ASSERT_EQ(true, !keys->hasNulls);
+  ASSERT_EQ(256, elements->numElements);
+  ASSERT_EQ(true, elements->hasNulls);
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    EXPECT_EQ(i % 2 == 0, maps->notNull[i]) << "Wrong value at " << i;
+    EXPECT_EQ((i + 1) / 2, maps->offsets[i]) << "Wrong value at " << i;
+  }
+  EXPECT_EQ(256, maps->offsets[512]);
+  for (size_t i = 0; i < keys->numElements; ++i) {
+    EXPECT_EQ(i, keys->data[i]);
+    EXPECT_EQ(i & 1, elements->notNull[i]);
+    if (elements->notNull[i]) {
+      EXPECT_EQ(i / 2 + 8, elements->data[i]);
+    }
+  }
+
+  reader->next(batch, 512, 0);
+  ASSERT_EQ(512, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(512, maps->numElements);
+  ASSERT_EQ(true, maps->hasNulls);
+  ASSERT_EQ(1012, keys->numElements);
+  ASSERT_EQ(true, !keys->hasNulls);
+  ASSERT_EQ(1012, elements->numElements);
+  ASSERT_EQ(true, elements->hasNulls);
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    EXPECT_EQ(i % 2 == 0, maps->notNull[i]) << "Wrong value at " << i;
+    if (i < 8) {
+      EXPECT_EQ((i + 1) / 2, maps->offsets[i])
+          << "Wrong value at " << i;
+    } else {
+      EXPECT_EQ(4 * ((i + 1) / 2) - 12, maps->offsets[i])
+          << "Wrong value at " << i;
+    }
+  }
+  EXPECT_EQ(1012, maps->offsets[512]);
+  for (size_t i = 0; i < keys->numElements; ++i) {
+    EXPECT_EQ(256 + i, keys->data[i]);
+    EXPECT_EQ(i & 1, elements->notNull[i]);
+    if (elements->notNull[i]) {
+      EXPECT_EQ(128 + 8 + i / 2, elements->data[i]);
+    }
+  }
+
+  reader->next(batch, 512, 0);
+  ASSERT_EQ(512, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(512, maps->numElements);
+  ASSERT_EQ(true, maps->hasNulls);
+  ASSERT_EQ(32, keys->numElements);
+  ASSERT_EQ(true, !keys->hasNulls);
+  ASSERT_EQ(32, elements->numElements);
+  ASSERT_EQ(true, elements->hasNulls);
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    EXPECT_EQ(i % 2 == 0, maps->notNull[i]) << "Wrong value at " << i;
+    if (i < 16) {
+      EXPECT_EQ(4 * ((i + 1) / 2), maps->offsets[i])
+          << "Wrong value at " << i;
+    } else {
+      EXPECT_EQ(32, maps->offsets[i]) << "Wrong value at " << i;
+    }
+  }
+  EXPECT_EQ(32, maps->offsets[512]);
+  for (size_t i = 0; i < keys->numElements; ++i) {
+    EXPECT_EQ(1268 + i, keys->data[i]);
+    EXPECT_EQ(i & 1, elements->notNull[i]);
+    if (elements->notNull[i]) {
+      EXPECT_EQ(634 + 8 + i / 2, elements->data[i]);
+    }
+  }
+
+  reader->next(batch, 512, 0);
+  ASSERT_EQ(512, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(512, maps->numElements);
+  ASSERT_EQ(true, maps->hasNulls);
+  ASSERT_EQ(748, keys->numElements);
+  ASSERT_EQ(true, !keys->hasNulls);
+  ASSERT_EQ(748, elements->numElements);
+  ASSERT_EQ(true, elements->hasNulls);
+  for (size_t i = 0; i < batch.numElements; ++i) {
+    EXPECT_EQ(i % 2 == 0, maps->notNull[i]) << "Wrong value at " << i;
+    if (i < 24) {
+      EXPECT_EQ(0, maps->offsets[i]) << "Wrong value at " << i;
+    } else if (i < 510) {
+      EXPECT_EQ(3 * ((i - 23) / 2), maps->offsets[i])
+          << "Wrong value at " << i;
+    } else if (i < 511) {
+      EXPECT_EQ(729, maps->offsets[i]) << "Wrong value at " << i;
+    } else {
+      EXPECT_EQ(748, maps->offsets[i]) << "Wrong value at " << i;
+    }
+  }
+  EXPECT_EQ(748, maps->offsets[512]);
+  for (size_t i = 0; i < keys->numElements; ++i) {
+    EXPECT_EQ(1300 + i, keys->data[i]);
+    EXPECT_EQ(i & 1, elements->notNull[i]);
+    if (elements->notNull[i]) {
+      EXPECT_EQ(650 + 8 + i / 2, elements->data[i]);
+    }
+  }
+}
+
+TEST(TestColumnReader, testMapSkipWithNulls) {
+  MockStripeStreams streams;
+
+  // set getSelectedColumns()
+  std::vector<bool> selectedColumns(4, true);
+  EXPECT_CALL(streams, getSelectedColumns())
+      .WillRepeatedly(testing::Return(selectedColumns));
+
+  // set getEncoding
+  proto::ColumnEncoding directEncoding;
+  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
+  EXPECT_CALL(streams, getEncoding(testing::_))
+      .WillRepeatedly(testing::Return(directEncoding));
+
+  // set getStream
+  EXPECT_CALL(streams, getStreamProxy(testing::_,proto::Stream_Kind_PRESENT,
+                                      true))
+      .WillRepeatedly(testing::Return(nullptr));
+
+  // [0xaa for x in range(2048/8)]
+  const unsigned char buffer1[] = { 0x7f, 0xaa, 0x7b, 0xaa };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer1, ARRAY_SIZE(buffer1))));
+
+  // [1 for x in range(260)] +
+  // [4 for x in range(260)] +
+  // [0 for x in range(260)] +
+  // [3 for x in range(243)] +
+  // [19]
+  const unsigned char buffer2[] = { 0x7f, 0x00, 0x01,
+                                    0x7f, 0x00, 0x01,
+                                    0x7f, 0x00, 0x04,
+                                    0x7f, 0x00, 0x04,
+                                    0x7f, 0x00, 0x00,
+                                    0x7f, 0x00, 0x00,
+                                    0x7f, 0x00, 0x03,
+                                    0x6e, 0x00, 0x03,
+                                    0xff, 0x13 };
+  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer2, ARRAY_SIZE(buffer2))));
+
+  // range(2048)
+  const unsigned char buffer3[] =  { 0x7f, 0x01, 0x00,
+                                     0x7f, 0x01, 0x84, 0x02,
+                                     0x7f, 0x01, 0x88, 0x04,
+                                     0x7f, 0x01, 0x8c, 0x06,
+                                     0x7f, 0x01, 0x90, 0x08,
+                                     0x7f, 0x01, 0x94, 0x0a,
+                                     0x7f, 0x01, 0x98, 0x0c,
+                                     0x7f, 0x01, 0x9c, 0x0e,
+                                     0x7f, 0x01, 0xa0, 0x10,
+                                     0x7f, 0x01, 0xa4, 0x12,
+                                     0x7f, 0x01, 0xa8, 0x14,
+                                     0x7f, 0x01, 0xac, 0x16,
+                                     0x7f, 0x01, 0xb0, 0x18,
+                                     0x7f, 0x01, 0xb4, 0x1a,
+                                     0x7f, 0x01, 0xb8, 0x1c,
+                                     0x5f, 0x01, 0xbc, 0x1e };
+  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer3, ARRAY_SIZE(buffer3))));
+
+  // range(8, 2056)
+  const unsigned char buffer4[] =  { 0x7f, 0x01, 0x10,
+                                     0x7f, 0x01, 0x94, 0x02,
+                                     0x7f, 0x01, 0x98, 0x04,
+                                     0x7f, 0x01, 0x9c, 0x06,
+                                     0x7f, 0x01, 0xa0, 0x08,
+                                     0x7f, 0x01, 0xa4, 0x0a,
+                                     0x7f, 0x01, 0xa8, 0x0c,
+                                     0x7f, 0x01, 0xac, 0x0e,
+                                     0x7f, 0x01, 0xb0, 0x10,
+                                     0x7f, 0x01, 0xb4, 0x12,
+                                     0x7f, 0x01, 0xb8, 0x14,
+                                     0x7f, 0x01, 0xbc, 0x16,
+                                     0x7f, 0x01, 0xc0, 0x18,
+                                     0x7f, 0x01, 0xc4, 0x1a,
+                                     0x7f, 0x01, 0xc8, 0x1c,
+                                     0x5f, 0x01, 0xcc, 0x1e };
+  EXPECT_CALL(streams, getStreamProxy(3, proto::Stream_Kind_DATA, true))
+      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
+                                      (buffer4, ARRAY_SIZE(buffer4))));
+
+  // create the row type
+  std::unique_ptr<Type> rowType = createStructType();
+  rowType->addStructField(createMapType(createPrimitiveType(LONG),
+                                        createPrimitiveType(LONG)),
+                          "col0");
+  rowType->assignIds(0);
+
+  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
+
+  StructVectorBatch batch(1, *getDefaultPool());
+  MapVectorBatch *maps = new MapVectorBatch(1, *getDefaultPool());
+  LongVectorBatch *keys = new LongVectorBatch(1, *getDefaultPool());
+  LongVectorBatch *elements = new LongVectorBatch(1, *getDefaultPool());
+  batch.fields.push_back(maps);
+  maps->keys = std::unique_ptr < ColumnVectorBatch > (keys);
+  maps->elements = std::unique_ptr < ColumnVectorBatch > (elements);
+
+  reader->next(batch, 1, 0);
+  ASSERT_EQ(1, batch.numElements);
+  ASSERT_EQ(true, !batch.hasNulls);
+  ASSERT_EQ(1, maps->numElements);
+  ASSERT_EQ(true, !maps->hasNulls);
+  ASSERT_EQ(1, keys->numElements);
+  ASSERT_EQ(true, !keys->hasNulls);
+  ASSERT_EQ(1, elements->numElements);
+  ASSERT_EQ(true, !elements->hasNulls);
+  EXPECT_EQ(0, maps->offsets[0]);
+  EXPECT_EQ(1, maps->offsets[1]);
+  EXPECT_EQ(0, keys->data[0]);
+  EXPECT_EQ(8, elements->data[0]);
+
+  reader->skip(13);
+  rea

<TRUNCATED>

[14/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/test/TestColumnPrinter.cc
----------------------------------------------------------------------
diff --git a/c++/test/TestColumnPrinter.cc b/c++/test/TestColumnPrinter.cc
new file mode 100644
index 0000000..a25bf3b
--- /dev/null
+++ b/c++/test/TestColumnPrinter.cc
@@ -0,0 +1,588 @@
+/**
+ * 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 "orc/ColumnPrinter.hh"
+
+#include "Exceptions.hh"
+#include "wrap/gtest-wrapper.h"
+
+namespace orc {
+
+  TEST(TestColumnPrinter, BooleanColumnPrinter) {
+    std::string line;
+    std::unique_ptr<Type> type = createPrimitiveType(BOOLEAN);
+    std::unique_ptr<ColumnPrinter> printer =
+      createColumnPrinter(line, *type);
+    LongVectorBatch batch(1024, *getDefaultPool());
+    const char *expected[] = {"true", "false", "true"};
+    batch.numElements = 3;
+    batch.hasNulls = false;
+    batch.data[0] = 1;
+    batch.data[1] = 0;
+    batch.data[2] = 1;
+    printer->reset(batch);
+    for(uint64_t i=0; i < 3; ++i) {
+      line.clear();
+      printer->printRow(i);
+      EXPECT_EQ(expected[i], line);
+    }
+    const char *expected2[] = {"null", "null", "true", "false"};
+    batch.numElements = 4;
+    batch.data[3] = false;
+    batch.hasNulls = true;
+    batch.notNull[0] = false;
+    batch.notNull[1] = false;
+    batch.notNull[2] = true;
+    batch.notNull[3] = true;
+    printer->reset(batch);
+    for(uint64_t i=0; i < 4; ++i) {
+      line.clear();
+      printer->printRow(i);
+      EXPECT_EQ(expected2[i], line);
+    }
+  }
+
+  TEST(TestColumnPrinter, LongColumnPrinter) {
+    std::string line;
+    std::unique_ptr<Type> type = createPrimitiveType(LONG);
+    std::unique_ptr<ColumnPrinter> printer = createColumnPrinter(line, *type);
+    LongVectorBatch batch(1024, *getDefaultPool());
+    batch.numElements = 2;
+    batch.hasNulls = false;
+    batch.data[0] = 9223372036854775807LL;
+    batch.data[1] = -9223372036854775807LL;
+    printer->reset(batch);
+    const char *expected[] = {"9223372036854775807", "-9223372036854775807"};
+    for(uint64_t i=0; i < 2; ++i) {
+      line.clear();
+      printer->printRow(i);
+      EXPECT_EQ(expected[i], line);
+    }
+    batch.numElements = 3;
+    batch.hasNulls = true;
+    batch.data[0] = 127;
+    batch.data[1] = -127;
+    batch.data[2] = 123;
+    batch.notNull[0] = true;
+    batch.notNull[1] = true;
+    batch.notNull[2] = false;
+    printer->reset(batch);
+    const char *expected2[] = {"127", "-127", "null"};
+    for(uint64_t i=0; i < 3; ++i) {
+      line.clear();
+      printer->printRow(i);
+      EXPECT_EQ(expected2[i], line);
+    }
+  }
+
+  TEST(TestColumnPrinter, DoubleColumnPrinter) {
+    std::string line;
+    std::unique_ptr<Type> type = createPrimitiveType(DOUBLE);
+    std::unique_ptr<ColumnPrinter> printer =
+      createColumnPrinter(line, *type);
+    DoubleVectorBatch batch(1024, *getDefaultPool());
+    batch.numElements = 2;
+    batch.hasNulls = false;
+    batch.data[0] = 1234.5;
+    batch.data[1] = -1234.5;
+    printer->reset(batch);
+    const char *expected[] = {"1234.5", "-1234.5"};
+    for(uint64_t i=0; i < 2; ++i) {
+      line.clear();
+      printer->printRow(i);
+      EXPECT_EQ(expected[i], line);
+    }
+    batch.numElements = 3;
+    batch.hasNulls = true;
+    batch.data[0] = 9999.125;
+    batch.data[1] = -9999.125;
+    batch.data[2] = 100000;
+    batch.notNull[0] = true;
+    batch.notNull[1] = true;
+    batch.notNull[2] = false;
+    printer->reset(batch);
+    const char *expected2[] = {"9999.125", "-9999.125", "null"};
+    for(uint64_t i=0; i < 3; ++i) {
+      line.clear();
+      printer->printRow(i);
+      EXPECT_EQ(expected2[i], line);
+    }
+  }
+
+  TEST(TestColumnPrinter, TimestampColumnPrinter) {
+    std::string line;
+    std::unique_ptr<Type> type = createPrimitiveType(TIMESTAMP);
+    std::unique_ptr<ColumnPrinter> printer = createColumnPrinter(line, *type);
+    LongVectorBatch batch(1024, *getDefaultPool());
+    batch.numElements = 12;
+    batch.hasNulls = false;
+    batch.data[0]  = 1420070400000000000;
+    batch.data[1]  =  963270000000000000;
+    batch.data[2]  = 1426168859000000000;
+    batch.data[3]  = 1426168859000000001;
+    batch.data[4]  = 1426168859000000010;
+    batch.data[5]  = 1426168859000000100;
+    batch.data[6]  = 1426168859000001000;
+    batch.data[7]  = 1426168859000010000;
+    batch.data[8]  = 1426168859000100000;
+    batch.data[9]  = 1426168859001000000;
+    batch.data[10] = 1426168859010000000;
+    batch.data[11] = 1426168859100000000;
+    const char *expected[] = {"\"2015-01-01 00:00:00.0\"",
+                              "\"2000-07-11 00:00:00.0\"",
+                              "\"2015-03-12 15:00:59.0\"",
+                              "\"2015-03-12 15:00:59.000000001\"",
+                              "\"2015-03-12 15:00:59.00000001\"",
+                              "\"2015-03-12 15:00:59.0000001\"",
+                              "\"2015-03-12 15:00:59.000001\"",
+                              "\"2015-03-12 15:00:59.00001\"",
+                              "\"2015-03-12 15:00:59.0001\"",
+                              "\"2015-03-12 15:00:59.001\"",
+                              "\"2015-03-12 15:00:59.01\"",
+                              "\"2015-03-12 15:00:59.1\""};
+    printer->reset(batch);
+    for(uint64_t i=0; i < batch.numElements; ++i) {
+      line.clear();
+      printer->printRow(i);
+      EXPECT_EQ(expected[i], line) << "for i = " << i;
+    }
+    batch.hasNulls = true;
+    for(size_t i = 0; i < batch.numElements; ++i) {
+      batch.notNull[i] = i % 2;
+    }
+    printer->reset(batch);
+    for(uint64_t i=0; i < batch.numElements; ++i) {
+      line.clear();
+      printer->printRow(i);
+      if (i % 2) {
+        EXPECT_EQ(expected[i], line) << "for i = " << i;
+      } else {
+        EXPECT_EQ("null", line) << "for i = " << i;
+      }
+    }
+  }
+
+  TEST(TestColumnPrinter, DateColumnPrinter) {
+    std::string line;
+    std::unique_ptr<Type> type = createPrimitiveType(DATE);
+    std::unique_ptr<ColumnPrinter> printer =
+      createColumnPrinter(line, *type);
+    LongVectorBatch batch(1024, *getDefaultPool());
+    batch.numElements = 10;
+    batch.hasNulls = false;
+    batch.data[0]  = 0;
+    batch.data[1]  = 11738;
+    batch.data[2]  = -165;
+    batch.data[3]  = -33165;
+    batch.data[4]  = 10489;
+    batch.data[5]  = -5171;
+    batch.data[6]  = 11016;
+    batch.data[7]  = 5763;
+    batch.data[8]  = 16729;
+    batch.data[9]  = 12275;
+    const char *expected[] = {"\"1970-01-01\"",
+                              "\"2002-02-20\"",
+                              "\"1969-07-20\"",
+                              "\"1879-03-14\"",
+                              "\"1998-09-20\"",
+                              "\"1955-11-05\"",
+                              "\"2000-02-29\"",
+                              "\"1985-10-12\"",
+                              "\"2015-10-21\"",
+                              "\"2003-08-11\""};
+    printer->reset(batch);
+    for(uint64_t i=0; i < batch.numElements; ++i) {
+      line.clear();
+      printer->printRow(i);
+      EXPECT_EQ(expected[i], line) << "for i = " << i;
+    }
+    batch.hasNulls = true;
+    for(size_t i = 0; i < batch.numElements; ++i) {
+      batch.notNull[i] = i % 2;
+    }
+    printer->reset(batch);
+    for(uint64_t i=0; i < batch.numElements; ++i) {
+      line.clear();
+      printer->printRow(i);
+      if (i % 2) {
+        EXPECT_EQ(expected[i], line) << "for i = " << i;
+      } else {
+        EXPECT_EQ("null", line) << "for i = " << i;
+      }
+    }
+  }
+
+  TEST(TestColumnPrinter, Decimal64ColumnPrinter) {
+    std::string line;
+    std::unique_ptr<Type> type = createDecimalType(16, 5);
+    std::unique_ptr<ColumnPrinter> printer =
+      createColumnPrinter(line, *type);
+    Decimal64VectorBatch batch(1024, *getDefaultPool());
+    batch.numElements = 10;
+    batch.hasNulls = false;
+    batch.scale = 5;
+    batch.values[0] =  0;
+    batch.values[1] =  1;
+    batch.values[2] = -10;
+    batch.values[3] =  100;
+    batch.values[4] =  1000;
+    batch.values[5] =  10000;
+    batch.values[6] =  100000;
+    batch.values[7] =  1000000;
+    batch.values[8] = -10000000;
+    batch.values[9] =  100000000;
+    const char *expected[] = { "0.00000",
+                               "0.00001",
+                               "-0.00010",
+                               "0.00100",
+                               "0.01000",
+                               "0.10000",
+                               "1.00000",
+                               "10.00000",
+                               "-100.00000",
+                               "1000.00000"};
+    printer->reset(batch);
+    for(uint64_t i=0; i < batch.numElements; ++i) {
+      line.clear();
+      printer->printRow(i);
+      EXPECT_EQ(expected[i], line) << "for i = " << i;
+    }
+    batch.hasNulls = true;
+    for(size_t i = 0; i < batch.numElements; ++i) {
+      batch.notNull[i] = i % 2;
+    }
+    printer->reset(batch);
+    for(uint64_t i=0; i < batch.numElements; ++i) {
+      line.clear();
+      printer->printRow(i);
+      if (i % 2) {
+        EXPECT_EQ(expected[i], line) << "for i = " << i;
+      } else {
+        EXPECT_EQ("null", line) << "for i = " << i;
+      }
+    }
+  }
+
+  TEST(TestColumnPrinter, Decimal128ColumnPrinter) {
+    std::string line;
+    std::unique_ptr<Type> type = createDecimalType(30, 5);
+    std::unique_ptr<ColumnPrinter> printer =
+      createColumnPrinter(line, *type);
+    Decimal128VectorBatch batch(1024, *getDefaultPool());
+    batch.numElements = 10;
+    batch.hasNulls = false;
+    batch.scale = 5;
+    batch.values[0] =  0;
+    batch.values[1] =  1;
+    batch.values[2] = -10;
+    batch.values[3] =  100;
+    batch.values[4] =  1000;
+    batch.values[5] =  10000;
+    batch.values[6] =  100000;
+    batch.values[7] =  1000000;
+    batch.values[8] = -10000000;
+    batch.values[9] =  100000000;
+    const char *expected[] = { "0.00000",
+                               "0.00001",
+                               "-0.00010",
+                               "0.00100",
+                               "0.01000",
+                               "0.10000",
+                               "1.00000",
+                               "10.00000",
+                               "-100.00000",
+                               "1000.00000"};
+    printer->reset(batch);
+    for(uint64_t i=0; i < batch.numElements; ++i) {
+      line.clear();
+      printer->printRow(i);
+      EXPECT_EQ(expected[i], line) << "for i = " << i;
+    }
+    batch.hasNulls = true;
+    for(size_t i = 0; i < batch.numElements; ++i) {
+      batch.notNull[i] = i % 2;
+    }
+    printer->reset(batch);
+    for(uint64_t i=0; i < batch.numElements; ++i) {
+      line.clear();
+      printer->printRow(i);
+      if (i % 2) {
+        EXPECT_EQ(expected[i], line) << "for i = " << i;
+      } else {
+        EXPECT_EQ("null", line) << "for i = " << i;
+      }
+    }
+  }
+
+  TEST(TestColumnPrinter, StringColumnPrinter) {
+    std::string line;
+    std::unique_ptr<Type> type = createPrimitiveType(STRING);
+    std::unique_ptr<ColumnPrinter> printer =
+      createColumnPrinter(line, *type);
+    StringVectorBatch batch(1024, *getDefaultPool());
+    const char *blob= "thisisatest\b\f\n\r\t\\\"'";
+    batch.numElements = 5;
+    batch.hasNulls = false;
+    batch.data[0] = const_cast<char *>(blob);
+    batch.length[0] = 4;
+    batch.length[1] = 2;
+    batch.length[2] = 1;
+    batch.length[3] = 4;
+    batch.length[4] = 8;
+    for(size_t i=1; i < 5; ++i) {
+      batch.data[i] = batch.data.data()[i - 1] +
+        static_cast<size_t>(batch.length[i - 1]);
+    }
+    const char *expected[] = {"\"this\"", "\"is\"", "\"a\"", "\"test\"",
+                              "\"\\b\\f\\n\\r\\t\\\\\\\"'\""};
+    printer->reset(batch);
+    for(uint64_t i=0; i < batch.numElements; ++i) {
+      line.clear();
+      printer->printRow(i);
+      EXPECT_EQ(expected[i], line) << "for i = " << i;
+    }
+    batch.hasNulls = true;
+    for(size_t i = 0; i < batch.numElements; ++i) {
+      batch.notNull[i] = i % 2;
+    }
+    printer->reset(batch);
+    for(uint64_t i=0; i < batch.numElements; ++i) {
+      line.clear();
+      printer->printRow(i);
+      if (i % 2) {
+        EXPECT_EQ(expected[i], line) << "for i = " << i;
+      } else {
+        EXPECT_EQ("null", line) << "for i = " << i;
+      }
+    }
+  }
+
+  TEST(TestColumnPrinter, BinaryColumnPrinter) {
+    std::string line;
+    std::unique_ptr<Type> type = createPrimitiveType(BINARY);
+    std::unique_ptr<ColumnPrinter> printer =
+      createColumnPrinter(line, *type);
+    StringVectorBatch batch(1024, *getDefaultPool());
+    char blob[45];
+    for(size_t i=0; i < sizeof(blob); ++i) {
+      blob[i] = static_cast<char>(i);
+    }
+    batch.numElements = 10;
+    batch.hasNulls = false;
+    batch.data[0] = blob;
+    batch.length[0] = 0;
+    for(size_t i=1; i < batch.numElements; ++i) {
+      batch.length[i] = static_cast<int64_t>(i);
+      batch.data[i] = batch.data.data()[i - 1] + batch.length[i - 1];
+    }
+    printer->reset(batch);
+    const char* expected[] = {"[]",
+                              "[0]",
+                              "[1, 2]",
+                              "[3, 4, 5]",
+                              "[6, 7, 8, 9]",
+                              "[10, 11, 12, 13, 14]",
+                              "[15, 16, 17, 18, 19, 20]",
+                              "[21, 22, 23, 24, 25, 26, 27]",
+                              "[28, 29, 30, 31, 32, 33, 34, 35]",
+                              "[36, 37, 38, 39, 40, 41, 42, 43, 44]"};
+    for(uint64_t i=0; i < batch.numElements; ++i) {
+      line.clear();
+      printer->printRow(i);
+      EXPECT_EQ(expected[i], line) << "for i = " << i;
+    }
+    batch.hasNulls = true;
+    for(size_t i = 0; i < batch.numElements; ++i) {
+      batch.notNull[i] = i % 2;
+    }
+    printer->reset(batch);
+    for(uint64_t i=0; i < batch.numElements; ++i) {
+      line.clear();
+      printer->printRow(i);
+      if (i % 2) {
+        EXPECT_EQ(expected[i], line) << "for i = " << i;
+      } else {
+        EXPECT_EQ("null", line) << "for i = " << i;
+      }
+    }
+  }
+
+  TEST(TestColumnPrinter, ListColumnPrinter) {
+    std::string line;
+    std::unique_ptr<Type> type = createListType(createPrimitiveType(LONG));
+    std::unique_ptr<ColumnPrinter> printer =
+      createColumnPrinter(line, *type);
+    ListVectorBatch batch(1024, *getDefaultPool());
+    LongVectorBatch* longBatch = new LongVectorBatch(1024, *getDefaultPool());
+    batch.elements = std::unique_ptr<ColumnVectorBatch>(longBatch);
+    batch.numElements = 10;
+    batch.hasNulls = false;
+    batch.offsets[0] = 0;
+    for(size_t i = 1; i <= batch.numElements; ++i) {
+      batch.offsets[i] = batch.offsets[i-1] + static_cast<int64_t>(i-1);
+    }
+    longBatch->numElements = 45;
+    longBatch->hasNulls = false;
+    for(size_t i = 0; i < longBatch->numElements; ++i) {
+      longBatch->data[i] = static_cast<int64_t>(i);
+    }
+    const char* expected[] = {"[]",
+                              "[0]",
+                              "[1, 2]",
+                              "[3, 4, 5]",
+                              "[6, 7, 8, 9]",
+                              "[10, 11, 12, 13, 14]",
+                              "[15, 16, 17, 18, 19, 20]",
+                              "[21, 22, 23, 24, 25, 26, 27]",
+                              "[28, 29, 30, 31, 32, 33, 34, 35]",
+                              "[36, 37, 38, 39, 40, 41, 42, 43, 44]"};
+    printer->reset(batch);
+    for(uint64_t i=0; i < batch.numElements; ++i) {
+      line.clear();
+      printer->printRow(i);
+      EXPECT_EQ(expected[i], line) << "for i = " << i;
+    }
+    batch.hasNulls = true;
+    for(size_t i = 0; i < batch.numElements; ++i) {
+      batch.notNull[i] = i % 2;
+    }
+    printer->reset(batch);
+    for(uint64_t i=0; i < batch.numElements; ++i) {
+      line.clear();
+      printer->printRow(i);
+      if (i % 2) {
+        EXPECT_EQ(expected[i], line) << "for i = " << i;
+      } else {
+        EXPECT_EQ("null", line) << "for i = " << i;
+      }
+    }
+  }
+
+  TEST(TestColumnPrinter, MapColumnPrinter) {
+    std::string line;
+    std::unique_ptr<Type> type = createMapType(createPrimitiveType(LONG),
+                                               createPrimitiveType(LONG));
+    std::unique_ptr<ColumnPrinter> printer =
+      createColumnPrinter(line, *type);
+    MapVectorBatch batch(1024, *getDefaultPool());
+    LongVectorBatch* keyBatch = new LongVectorBatch(1024, *getDefaultPool());
+    LongVectorBatch* valueBatch = new LongVectorBatch(1024, *getDefaultPool());
+    batch.keys = std::unique_ptr<ColumnVectorBatch>(keyBatch);
+    batch.elements = std::unique_ptr<ColumnVectorBatch>(valueBatch);
+    batch.numElements = 4;
+    batch.hasNulls = false;
+    batch.offsets[0] = 0;
+    for(size_t i = 1; i <= batch.numElements; ++i) {
+      batch.offsets[i] = batch.offsets[i-1] + static_cast<int64_t>(i-1);
+    }
+    keyBatch->numElements = 6;
+    keyBatch->hasNulls = false;
+    valueBatch->numElements = 6;
+    valueBatch->hasNulls = false;
+    for(size_t i = 0; i < keyBatch->numElements; ++i) {
+      keyBatch->data[i] = static_cast<int64_t>(i);
+      valueBatch->data[i] = static_cast<int64_t>(2 * i);
+    }
+    const char* expected[] = {"[]",
+                              "[{\"key\": 0, \"value\": 0}]",
+                              ("[{\"key\": 1, \"value\": 2},"
+                               " {\"key\": 2, \"value\": 4}]"),
+                              ("[{\"key\": 3, \"value\": 6},"
+                               " {\"key\": 4, \"value\": 8},"
+                               " {\"key\": 5, \"value\": 10}]")};
+    printer->reset(batch);
+    for(uint64_t i=0; i < batch.numElements; ++i) {
+      line.clear();
+      printer->printRow(i);
+      EXPECT_EQ(expected[i], line) << "for i = " << i;
+    }
+    batch.hasNulls = true;
+    for(size_t i = 0; i < batch.numElements; ++i) {
+      batch.notNull[i] = i % 2;
+    }
+    printer->reset(batch);
+    for(uint64_t i=0; i < batch.numElements; ++i) {
+      line.clear();
+      printer->printRow(i);
+      if (i % 2) {
+        EXPECT_EQ(expected[i], line) << "for i = " << i;
+      } else {
+        EXPECT_EQ("null", line) << "for i = " << i;
+      }
+    }
+  }
+
+  TEST(TestColumnPrinter, StructColumnPrinter) {
+    std::string line;
+    std::vector<std::string> fieldNames;
+    std::vector<Type*> subtypes;
+    fieldNames.push_back("first");
+    fieldNames.push_back("second");
+    subtypes.push_back(createPrimitiveType(LONG).release());
+    subtypes.push_back(createPrimitiveType(LONG).release());
+    std::unique_ptr<Type> type = createStructType(subtypes, fieldNames);
+    std::unique_ptr<ColumnPrinter> printer =
+      createColumnPrinter(line, *type);
+    StructVectorBatch batch(1024, *getDefaultPool());
+    LongVectorBatch* firstBatch = new LongVectorBatch(1024, *getDefaultPool());
+    LongVectorBatch* secondBatch =
+      new LongVectorBatch(1024, *getDefaultPool());
+    batch.fields.push_back(firstBatch);
+    batch.fields.push_back(secondBatch);
+    batch.numElements = 10;
+    batch.hasNulls = false;
+    firstBatch->numElements = 10;
+    firstBatch->hasNulls = false;
+    secondBatch->numElements = 10;
+    secondBatch->hasNulls = false;
+    for(size_t i = 0; i < firstBatch->numElements; ++i) {
+      firstBatch->data[i] = static_cast<int64_t>(i);
+      secondBatch->data[i] = static_cast<int64_t>(2 * i);
+    }
+    const char* expected[] = {"{\"first\": 0, \"second\": 0}",
+                              "{\"first\": 1, \"second\": 2}",
+                              "{\"first\": 2, \"second\": 4}",
+                              "{\"first\": 3, \"second\": 6}",
+                              "{\"first\": 4, \"second\": 8}",
+                              "{\"first\": 5, \"second\": 10}",
+                              "{\"first\": 6, \"second\": 12}",
+                              "{\"first\": 7, \"second\": 14}",
+                              "{\"first\": 8, \"second\": 16}",
+                              "{\"first\": 9, \"second\": 18}"};
+    printer->reset(batch);
+    for(uint64_t i=0; i < batch.numElements; ++i) {
+      line.clear();
+      printer->printRow(i);
+      EXPECT_EQ(expected[i], line) << "for i = " << i;
+    }
+    batch.hasNulls = true;
+    for(size_t i = 0; i < batch.numElements; ++i) {
+      batch.notNull[i] = i % 2;
+    }
+    printer->reset(batch);
+    for(uint64_t i=0; i < batch.numElements; ++i) {
+      line.clear();
+      printer->printRow(i);
+      if (i % 2) {
+        EXPECT_EQ(expected[i], line) << "for i = " << i;
+      } else {
+        EXPECT_EQ("null", line) << "for i = " << i;
+      }
+    }
+  }
+}  // namespace orc


[18/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/ColumnReader.cc
----------------------------------------------------------------------
diff --git a/c++/src/orc/ColumnReader.cc b/c++/src/orc/ColumnReader.cc
deleted file mode 100644
index 0b6a9cb..0000000
--- a/c++/src/orc/ColumnReader.cc
+++ /dev/null
@@ -1,1557 +0,0 @@
-/**
- * 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 "orc/Adaptor.hh"
-#include "ByteRLE.hh"
-#include "ColumnReader.hh"
-#include "Exceptions.hh"
-#include "orc/Int128.hh"
-#include "RLE.hh"
-
-#include <math.h>
-#include <iostream>
-
-namespace orc {
-
-  StripeStreams::~StripeStreams() {
-    // PASS
-  }
-
-  inline RleVersion convertRleVersion(proto::ColumnEncoding_Kind kind) {
-    switch (static_cast<int64_t>(kind)) {
-    case proto::ColumnEncoding_Kind_DIRECT:
-    case proto::ColumnEncoding_Kind_DICTIONARY:
-      return RleVersion_1;
-    case proto::ColumnEncoding_Kind_DIRECT_V2:
-    case proto::ColumnEncoding_Kind_DICTIONARY_V2:
-      return RleVersion_2;
-    default:
-      throw ParseError("Unknown encoding in convertRleVersion");
-    }
-  }
-
-  ColumnReader::ColumnReader(const Type& type,
-                             StripeStreams& stripe
-                             ): columnId(type.getColumnId()),
-                                memoryPool(stripe.getMemoryPool()) {
-    std::unique_ptr<SeekableInputStream> stream =
-      stripe.getStream(columnId, proto::Stream_Kind_PRESENT, true);
-    if (stream.get()) {
-      notNullDecoder = createBooleanRleDecoder(std::move(stream));
-    }
-  }
-
-  ColumnReader::~ColumnReader() {
-    // PASS
-  }
-
-  uint64_t ColumnReader::skip(uint64_t numValues) {
-    ByteRleDecoder* decoder = notNullDecoder.get();
-    if (decoder) {
-      // page through the values that we want to skip
-      // and count how many are non-null
-      const size_t MAX_BUFFER_SIZE = 32768;
-      size_t bufferSize = std::min(MAX_BUFFER_SIZE,
-                                   static_cast<size_t>(numValues));
-      char buffer[MAX_BUFFER_SIZE];
-      uint64_t remaining = numValues;
-      while (remaining > 0) {
-        uint64_t chunkSize =
-          std::min(remaining,
-                   static_cast<uint64_t>(bufferSize));
-        decoder->next(buffer, chunkSize, 0);
-        remaining -= chunkSize;
-        for(uint64_t i=0; i < chunkSize; ++i) {
-          if (!buffer[i]) {
-            numValues -= 1;
-          }
-        }
-      }
-    }
-    return numValues;
-  }
-
-  void ColumnReader::next(ColumnVectorBatch& rowBatch,
-                          uint64_t numValues,
-                          char* incomingMask) {
-    if (numValues > rowBatch.capacity) {
-      rowBatch.resize(numValues);
-    }
-    rowBatch.numElements = numValues;
-    ByteRleDecoder* decoder = notNullDecoder.get();
-    if (decoder) {
-      char* notNullArray = rowBatch.notNull.data();
-      decoder->next(notNullArray, numValues, incomingMask);
-      // check to see if there are nulls in this batch
-      for(uint64_t i=0; i < numValues; ++i) {
-        if (!notNullArray[i]) {
-          rowBatch.hasNulls = true;
-          return;
-        }
-      }
-    } else if (incomingMask) {
-      // If we don't have a notNull stream, copy the incomingMask
-      rowBatch.hasNulls = true;
-      memcpy(rowBatch.notNull.data(), incomingMask, numValues);
-      return;
-    }
-    rowBatch.hasNulls = false;
-  }
-
-  /**
-   * Expand an array of bytes in place to the corresponding array of longs.
-   * Has to work backwards so that they data isn't clobbered during the
-   * expansion.
-   * @param buffer the array of chars and array of longs that need to be
-   *        expanded
-   * @param numValues the number of bytes to convert to longs
-   */
-  void expandBytesToLongs(int64_t* buffer, uint64_t numValues) {
-    for(size_t i=numValues - 1; i < numValues; --i) {
-      buffer[i] = reinterpret_cast<char *>(buffer)[i];
-    }
-  }
-
-  class BooleanColumnReader: public ColumnReader {
-  private:
-    std::unique_ptr<orc::ByteRleDecoder> rle;
-
-  public:
-    BooleanColumnReader(const Type& type, StripeStreams& stipe);
-    ~BooleanColumnReader();
-
-    uint64_t skip(uint64_t numValues) override;
-
-    void next(ColumnVectorBatch& rowBatch,
-              uint64_t numValues,
-              char* notNull) override;
-  };
-
-  BooleanColumnReader::BooleanColumnReader(const Type& type,
-                                           StripeStreams& stripe
-                                           ): ColumnReader(type, stripe){
-    rle = createBooleanRleDecoder(stripe.getStream(columnId,
-                                                   proto::Stream_Kind_DATA,
-                                                   true));
-  }
-
-  BooleanColumnReader::~BooleanColumnReader() {
-    // PASS
-  }
-
-  uint64_t BooleanColumnReader::skip(uint64_t numValues) {
-    numValues = ColumnReader::skip(numValues);
-    rle->skip(numValues);
-    return numValues;
-  }
-
-  void BooleanColumnReader::next(ColumnVectorBatch& rowBatch,
-                                 uint64_t numValues,
-                                 char *notNull) {
-    ColumnReader::next(rowBatch, numValues, notNull);
-    // Since the byte rle places the output in a char* instead of long*,
-    // 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);
-    expandBytesToLongs(ptr, numValues);
-  }
-
-  class ByteColumnReader: public ColumnReader {
-  private:
-    std::unique_ptr<orc::ByteRleDecoder> rle;
-
-  public:
-    ByteColumnReader(const Type& type, StripeStreams& stipe);
-    ~ByteColumnReader();
-
-    uint64_t skip(uint64_t numValues) override;
-
-    void next(ColumnVectorBatch& rowBatch,
-              uint64_t numValues,
-              char* notNull) override;
-  };
-
-  ByteColumnReader::ByteColumnReader(const Type& type,
-                                           StripeStreams& stripe
-                                           ): ColumnReader(type, stripe){
-    rle = createByteRleDecoder(stripe.getStream(columnId,
-                                                proto::Stream_Kind_DATA,
-                                                true));
-  }
-
-  ByteColumnReader::~ByteColumnReader() {
-    // PASS
-  }
-
-  uint64_t ByteColumnReader::skip(uint64_t numValues) {
-    numValues = ColumnReader::skip(numValues);
-    rle->skip(numValues);
-    return numValues;
-  }
-
-  void ByteColumnReader::next(ColumnVectorBatch& rowBatch,
-                              uint64_t numValues,
-                              char *notNull) {
-    ColumnReader::next(rowBatch, numValues, notNull);
-    // Since the byte rle places the output in a char* instead of long*,
-    // 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);
-    expandBytesToLongs(ptr, numValues);
-  }
-
-  class IntegerColumnReader: public ColumnReader {
-  protected:
-    std::unique_ptr<orc::RleDecoder> rle;
-
-  public:
-    IntegerColumnReader(const Type& type, StripeStreams& stripe);
-    ~IntegerColumnReader();
-
-    uint64_t skip(uint64_t numValues) override;
-
-    void next(ColumnVectorBatch& rowBatch,
-              uint64_t numValues,
-              char* notNull) override;
-  };
-
-  IntegerColumnReader::IntegerColumnReader(const Type& type,
-                                           StripeStreams& stripe
-                                           ): ColumnReader(type, stripe) {
-    RleVersion vers = convertRleVersion(stripe.getEncoding(columnId).kind());
-    rle = createRleDecoder(stripe.getStream(columnId,
-                                            proto::Stream_Kind_DATA,
-                                            true),
-                           true, vers, memoryPool);
-  }
-
-  IntegerColumnReader::~IntegerColumnReader() {
-    // PASS
-  }
-
-  uint64_t IntegerColumnReader::skip(uint64_t numValues) {
-    numValues = ColumnReader::skip(numValues);
-    rle->skip(numValues);
-    return numValues;
-  }
-
-  void IntegerColumnReader::next(ColumnVectorBatch& rowBatch,
-                                 uint64_t numValues,
-                                 char *notNull) {
-    ColumnReader::next(rowBatch, numValues, notNull);
-    rle->next(dynamic_cast<LongVectorBatch&>(rowBatch).data.data(),
-              numValues, rowBatch.hasNulls ? rowBatch.notNull.data() : 0);
-  }
-
-  class TimestampColumnReader: public IntegerColumnReader {
-  private:
-    std::unique_ptr<orc::RleDecoder> nanoRle;
-    DataBuffer<int64_t> nanoBuffer;
-
-  public:
-    TimestampColumnReader(const Type& type, StripeStreams& stripe);
-    ~TimestampColumnReader();
-
-    uint64_t skip(uint64_t numValues) override;
-
-    void next(ColumnVectorBatch& rowBatch,
-              uint64_t numValues,
-              char* notNull) override;
-  };
-
-
-  TimestampColumnReader::TimestampColumnReader(const Type& type,
-                                               StripeStreams& stripe
-                                               ): IntegerColumnReader(type,
-                                                                      stripe),
-                                                  nanoBuffer(memoryPool, 1024){
-    RleVersion vers = convertRleVersion(stripe.getEncoding(columnId).kind());
-    nanoRle = createRleDecoder(stripe.getStream(columnId,
-                                                proto::Stream_Kind_SECONDARY,
-                                                true),
-                               false, vers, memoryPool);
-  }
-
-  TimestampColumnReader::~TimestampColumnReader() {
-    // PASS
-  }
-
-  uint64_t TimestampColumnReader::skip(uint64_t numValues) {
-    numValues = IntegerColumnReader::skip(numValues);
-    nanoRle->skip(numValues);
-    return numValues;
-  }
-
-  void TimestampColumnReader::next(ColumnVectorBatch& rowBatch,
-                                 uint64_t numValues,
-                                 char *notNull) {
-    ColumnReader::next(rowBatch, numValues, notNull);
-    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : nullptr;
-    int64_t* pStamp = dynamic_cast<LongVectorBatch&>(rowBatch).data.data();
-
-    // make sure that nanoBuffer is large enough
-    if (numValues > nanoBuffer.size()) {
-      nanoBuffer.resize(numValues);
-    }
-
-    rle->next(pStamp, numValues, notNull);
-    nanoRle->next(nanoBuffer.data(), numValues, notNull);
-
-    // Construct the values
-    for(uint64_t i=0; i < numValues; i++) {
-      if (notNull == nullptr || notNull[i]) {
-        int64_t nanosec =  nanoBuffer[i] >> 3;
-        uint64_t zeros = nanoBuffer[i] & 0x7;
-        if (zeros != 0) {
-          for(uint64_t j = 0; j <= zeros; ++j) {
-            nanosec *= 10;
-          }
-        }
-        pStamp[i] =  pStamp[i] * 1000000000 + 1420070400000000000;
-        if (pStamp[i] >= 0) {
-          pStamp[i] += nanosec;
-        } else {
-          pStamp[i] -= nanosec;
-        }
-      }
-    }
-  }
-
-  class DoubleColumnReader: public ColumnReader {
-  public:
-    DoubleColumnReader(const Type& type, StripeStreams& stripe);
-    ~DoubleColumnReader();
-
-    uint64_t skip(uint64_t numValues) override;
-
-    void next(ColumnVectorBatch& rowBatch,
-              uint64_t numValues,
-              char* notNull) override;
-
-  private:
-    std::unique_ptr<SeekableInputStream> inputStream;
-    TypeKind columnKind;
-    const uint64_t bytesPerValue ;
-    const char *bufferPointer;
-    const char *bufferEnd;
-
-    unsigned char readByte() {
-      if (bufferPointer == bufferEnd) {
-        int length;
-        if (!inputStream->Next
-            (reinterpret_cast<const void**>(&bufferPointer), &length)) {
-          throw ParseError("bad read in DoubleColumnReader::next()");
-        }
-        bufferEnd = bufferPointer + length;
-      }
-      return static_cast<unsigned char>(*(bufferPointer++));
-    }
-
-    double readDouble() {
-      int64_t bits = 0;
-      for (uint64_t i=0; i < 8; i++) {
-        bits |= static_cast<int64_t>(readByte()) << (i*8);
-      }
-      double *result = reinterpret_cast<double*>(&bits);
-      return *result;
-    }
-
-    double readFloat() {
-      int32_t bits = 0;
-      for (uint64_t i=0; i < 4; i++) {
-        bits |= readByte() << (i*8);
-      }
-      float *result = reinterpret_cast<float*>(&bits);
-      return *result;
-    }
-  };
-
-  DoubleColumnReader::DoubleColumnReader(const Type& type,
-                                         StripeStreams& stripe
-                                         ): ColumnReader(type, stripe),
-                                            inputStream
-                                               (stripe.getStream
-                                                (columnId,
-                                                 proto::Stream_Kind_DATA,
-                                                 true)),
-                                            columnKind(type.getKind()),
-                                            bytesPerValue((type.getKind() ==
-                                                           FLOAT) ? 4 : 8),
-                                            bufferPointer(NULL),
-                                            bufferEnd(NULL) {
-    // PASS
-  }
-
-  DoubleColumnReader::~DoubleColumnReader() {
-    // PASS
-  }
-
-  uint64_t DoubleColumnReader::skip(uint64_t numValues) {
-    numValues = ColumnReader::skip(numValues);
-
-    if (static_cast<size_t>(bufferEnd - bufferPointer) >=
-        bytesPerValue * numValues) {
-      bufferPointer+= bytesPerValue*numValues;
-    } else {
-      inputStream->Skip(static_cast<int>(bytesPerValue*numValues -
-                                         static_cast<size_t>(bufferEnd -
-                                                             bufferPointer)));
-      bufferEnd = NULL;
-      bufferPointer = NULL;
-    }
-
-    return numValues;
-  }
-
-  void DoubleColumnReader::next(ColumnVectorBatch& rowBatch,
-                                uint64_t numValues,
-                                char *notNull) {
-    ColumnReader::next(rowBatch, numValues, notNull);
-    // update the notNull from the parent class
-    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : 0;
-    double* outArray = dynamic_cast<DoubleVectorBatch&>(rowBatch).data.data();
-
-    if (columnKind == FLOAT) {
-      if (notNull) {
-        for(size_t i=0; i < numValues; ++i) {
-          if (notNull[i]) {
-            outArray[i] = readFloat();
-          }
-        }
-      } else {
-        for(size_t i=0; i < numValues; ++i) {
-          outArray[i] = readFloat();
-        }
-      }
-    } else {
-      if (notNull) {
-        for(size_t i=0; i < numValues; ++i) {
-          if (notNull[i]) {
-            outArray[i] = readDouble();
-          }
-        }
-      } else {
-        for(size_t i=0; i < numValues; ++i) {
-          outArray[i] = readDouble();
-        }
-      }
-    }
-  }
-
-  void readFully(char* buffer, int64_t bufferSize, SeekableInputStream* stream) {
-    int64_t posn = 0;
-    while (posn < bufferSize) {
-      const void* chunk;
-      int length;
-      if (!stream->Next(&chunk, &length)) {
-        throw ParseError("bad read in readFully");
-      }
-      memcpy(buffer + posn, chunk, static_cast<size_t>(length));
-      posn += length;
-    }
-  }
-
-  class StringDictionaryColumnReader: public ColumnReader {
-  private:
-    DataBuffer<char> dictionaryBlob;
-    DataBuffer<int64_t> dictionaryOffset;
-    std::unique_ptr<RleDecoder> rle;
-    uint64_t dictionaryCount;
-
-  public:
-    StringDictionaryColumnReader(const Type& type, StripeStreams& stipe);
-    ~StringDictionaryColumnReader();
-
-    uint64_t skip(uint64_t numValues) override;
-
-    void next(ColumnVectorBatch& rowBatch,
-              uint64_t numValues,
-              char *notNull) override;
-  };
-
-  StringDictionaryColumnReader::StringDictionaryColumnReader
-             (const Type& type,
-              StripeStreams& stripe
-              ): ColumnReader(type, stripe),
-                 dictionaryBlob(stripe.getMemoryPool()),
-                 dictionaryOffset(stripe.getMemoryPool()) {
-    RleVersion rleVersion = convertRleVersion(stripe.getEncoding(columnId)
-                                                .kind());
-    dictionaryCount = stripe.getEncoding(columnId).dictionarysize();
-    rle = createRleDecoder(stripe.getStream(columnId,
-                                            proto::Stream_Kind_DATA,
-                                            true),
-                           false, rleVersion, memoryPool);
-    std::unique_ptr<RleDecoder> lengthDecoder =
-      createRleDecoder(stripe.getStream(columnId,
-                                        proto::Stream_Kind_LENGTH,
-                                        false),
-                       false, rleVersion, memoryPool);
-    dictionaryOffset.resize(dictionaryCount+1);
-    int64_t* lengthArray = dictionaryOffset.data();
-    lengthDecoder->next(lengthArray + 1, dictionaryCount, 0);
-    lengthArray[0] = 0;
-    for(uint64_t i=1; i < dictionaryCount + 1; ++i) {
-      lengthArray[i] += lengthArray[i-1];
-    }
-    int64_t blobSize = lengthArray[dictionaryCount];
-    dictionaryBlob.resize(static_cast<uint64_t>(blobSize));
-    std::unique_ptr<SeekableInputStream> blobStream =
-      stripe.getStream(columnId, proto::Stream_Kind_DICTIONARY_DATA, false);
-    readFully(dictionaryBlob.data(), blobSize, blobStream.get());
-  }
-
-  StringDictionaryColumnReader::~StringDictionaryColumnReader() {
-    // PASS
-  }
-
-  uint64_t StringDictionaryColumnReader::skip(uint64_t numValues) {
-    numValues = ColumnReader::skip(numValues);
-    rle->skip(numValues);
-    return numValues;
-  }
-
-  void StringDictionaryColumnReader::next(ColumnVectorBatch& rowBatch,
-                                          uint64_t numValues,
-                                          char *notNull) {
-    ColumnReader::next(rowBatch, numValues, notNull);
-    // update the notNull from the parent class
-    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : 0;
-    StringVectorBatch& byteBatch = dynamic_cast<StringVectorBatch&>(rowBatch);
-    char *blob = dictionaryBlob.data();
-    int64_t *dictionaryOffsets = dictionaryOffset.data();
-    char **outputStarts = byteBatch.data.data();
-    int64_t *outputLengths = byteBatch.length.data();
-    rle->next(outputLengths, numValues, notNull);
-    if (notNull) {
-      for(uint64_t i=0; i < numValues; ++i) {
-        if (notNull[i]) {
-          int64_t entry = outputLengths[i];
-          outputStarts[i] = blob + dictionaryOffsets[entry];
-          outputLengths[i] = dictionaryOffsets[entry+1] -
-            dictionaryOffsets[entry];
-        }
-      }
-    } else {
-      for(uint64_t i=0; i < numValues; ++i) {
-        int64_t entry = outputLengths[i];
-        outputStarts[i] = blob + dictionaryOffsets[entry];
-        outputLengths[i] = dictionaryOffsets[entry+1] -
-          dictionaryOffsets[entry];
-      }
-    }
-  }
-
-  class StringDirectColumnReader: public ColumnReader {
-  private:
-    DataBuffer<char> blobBuffer;
-    std::unique_ptr<RleDecoder> lengthRle;
-    std::unique_ptr<SeekableInputStream> blobStream;
-    const char *lastBuffer;
-    size_t lastBufferLength;
-
-    /**
-     * Compute the total length of the values.
-     * @param lengths the array of lengths
-     * @param notNull the array of notNull flags
-     * @param numValues the lengths of the arrays
-     * @return the total number of bytes for the non-null values
-     */
-    size_t computeSize(const int64_t *lengths, const char *notNull,
-                       uint64_t numValues);
-
-  public:
-    StringDirectColumnReader(const Type& type, StripeStreams& stipe);
-    ~StringDirectColumnReader();
-
-    uint64_t skip(uint64_t numValues) override;
-
-    void next(ColumnVectorBatch& rowBatch,
-              uint64_t numValues,
-              char *notNull) override;
-  };
-
-  StringDirectColumnReader::StringDirectColumnReader
-                 (const Type& type,
-                  StripeStreams& stripe
-                  ): ColumnReader(type, stripe),
-                     blobBuffer(stripe.getMemoryPool()) {
-    RleVersion rleVersion = convertRleVersion(stripe.getEncoding(columnId)
-                                                .kind());
-    lengthRle = createRleDecoder(stripe.getStream(columnId,
-                                                  proto::Stream_Kind_LENGTH,
-                                                  true),
-                                 false, rleVersion, memoryPool);
-    blobStream = stripe.getStream(columnId, proto::Stream_Kind_DATA, true);
-    lastBuffer = 0;
-    lastBufferLength = 0;
-  }
-
-  StringDirectColumnReader::~StringDirectColumnReader() {
-    // PASS
-  }
-
-  uint64_t StringDirectColumnReader::skip(uint64_t numValues) {
-    const size_t BUFFER_SIZE = 1024;
-    numValues = ColumnReader::skip(numValues);
-    int64_t buffer[BUFFER_SIZE];
-    uint64_t done = 0;
-    size_t totalBytes = 0;
-    // read the lengths, so we know haw many bytes to skip
-    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);
-      done += step;
-    }
-    if (totalBytes <= lastBufferLength) {
-      // subtract the needed bytes from the ones left over
-      lastBufferLength -= totalBytes;
-      lastBuffer += totalBytes;
-    } else {
-      // move the stream forward after accounting for the buffered bytes
-      totalBytes -= lastBufferLength;
-      blobStream->Skip(static_cast<int>(totalBytes));
-      lastBufferLength = 0;
-      lastBuffer = 0;
-    }
-    return numValues;
-  }
-
-  size_t StringDirectColumnReader::computeSize(const int64_t* lengths,
-                                               const char* notNull,
-                                               uint64_t numValues) {
-    size_t totalLength = 0;
-    if (notNull) {
-      for(size_t i=0; i < numValues; ++i) {
-        if (notNull[i]) {
-          totalLength += static_cast<size_t>(lengths[i]);
-        }
-      }
-    } else {
-      for(size_t i=0; i < numValues; ++i) {
-        totalLength += static_cast<size_t>(lengths[i]);
-      }
-    }
-    return totalLength;
-  }
-
-  void StringDirectColumnReader::next(ColumnVectorBatch& rowBatch,
-                                      uint64_t numValues,
-                                      char *notNull) {
-    ColumnReader::next(rowBatch, numValues, notNull);
-    // update the notNull from the parent class
-    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : 0;
-    StringVectorBatch& byteBatch = dynamic_cast<StringVectorBatch&>(rowBatch);
-    char **startPtr = byteBatch.data.data();
-    int64_t *lengthPtr = byteBatch.length.data();
-
-    // read the length vector
-    lengthRle->next(lengthPtr, numValues, notNull);
-
-    // figure out the total length of data we need from the blob stream
-    const size_t totalLength = computeSize(lengthPtr, notNull, numValues);
-
-    // Load data from the blob stream into our buffer until we have enough
-    // to get the rest directly out of the stream's buffer.
-    size_t bytesBuffered = 0;
-    blobBuffer.resize(totalLength);
-    char *ptr= blobBuffer.data();
-    while (bytesBuffered + lastBufferLength < totalLength) {
-      blobBuffer.resize(bytesBuffered + lastBufferLength);
-      memcpy(ptr + bytesBuffered, lastBuffer, lastBufferLength);
-      bytesBuffered += lastBufferLength;
-      const void* readBuffer;
-      int readLength;
-      if (!blobStream->Next(&readBuffer, &readLength)) {
-        throw ParseError("failed to read in StringDirectColumnReader.next");
-      }
-      lastBuffer = static_cast<const char*>(readBuffer);
-      lastBufferLength = static_cast<size_t>(readLength);
-    }
-
-    // Set up the start pointers for the ones that will come out of the buffer.
-    size_t filledSlots = 0;
-    size_t usedBytes = 0;
-    ptr = blobBuffer.data();
-    if (notNull) {
-      while (filledSlots < numValues &&
-             (usedBytes + static_cast<size_t>(lengthPtr[filledSlots]) <=
-              bytesBuffered)) {
-        if (notNull[filledSlots]) {
-          startPtr[filledSlots] = ptr + usedBytes;
-          usedBytes += static_cast<size_t>(lengthPtr[filledSlots]);
-        }
-        filledSlots += 1;
-      }
-    } else {
-      while (filledSlots < numValues &&
-             (usedBytes + static_cast<size_t>(lengthPtr[filledSlots]) <=
-              bytesBuffered)) {
-        startPtr[filledSlots] = ptr + usedBytes;
-        usedBytes += static_cast<size_t>(lengthPtr[filledSlots]);
-        filledSlots += 1;
-      }
-    }
-
-    // do we need to complete the last value in the blob buffer?
-    if (usedBytes < bytesBuffered) {
-      size_t moreBytes = static_cast<size_t>(lengthPtr[filledSlots]) -
-        (bytesBuffered - usedBytes);
-      blobBuffer.resize(bytesBuffered + moreBytes);
-      ptr = blobBuffer.data();
-      memcpy(ptr + bytesBuffered, lastBuffer, moreBytes);
-      lastBuffer += moreBytes;
-      lastBufferLength -= moreBytes;
-      startPtr[filledSlots++] = ptr + usedBytes;
-    }
-
-    // Finally, set up any remaining entries into the stream buffer
-    if (notNull) {
-      while (filledSlots < numValues) {
-        if (notNull[filledSlots]) {
-          startPtr[filledSlots] = const_cast<char*>(lastBuffer);
-          lastBuffer += lengthPtr[filledSlots];
-          lastBufferLength -= static_cast<size_t>(lengthPtr[filledSlots]);
-        }
-        filledSlots += 1;
-      }
-    } else {
-      while (filledSlots < numValues) {
-        startPtr[filledSlots] = const_cast<char*>(lastBuffer);
-        lastBuffer += lengthPtr[filledSlots];
-        lastBufferLength -= static_cast<size_t>(lengthPtr[filledSlots]);
-        filledSlots += 1;
-      }
-    }
-  }
-
-  class StructColumnReader: public ColumnReader {
-  private:
-    std::vector<ColumnReader*> children;
-
-  public:
-    StructColumnReader(const Type& type, StripeStreams& stipe);
-    ~StructColumnReader();
-
-    uint64_t skip(uint64_t numValues) override;
-
-    void next(ColumnVectorBatch& rowBatch,
-              uint64_t numValues,
-              char *notNull) override;
-  };
-
-  StructColumnReader::StructColumnReader(const Type& type,
-                                         StripeStreams& stripe
-                                         ): ColumnReader(type, stripe) {
-    // count the number of selected sub-columns
-    const std::vector<bool> selectedColumns = stripe.getSelectedColumns();
-    switch (static_cast<int64_t>(stripe.getEncoding(columnId).kind())) {
-    case proto::ColumnEncoding_Kind_DIRECT:
-      for(unsigned int i=0; i < type.getSubtypeCount(); ++i) {
-        const Type& child = type.getSubtype(i);
-        if (selectedColumns[static_cast<uint64_t>(child.getColumnId())]) {
-          children.push_back(buildReader(child, stripe).release());
-        }
-      }
-      break;
-    case proto::ColumnEncoding_Kind_DIRECT_V2:
-    case proto::ColumnEncoding_Kind_DICTIONARY:
-    case proto::ColumnEncoding_Kind_DICTIONARY_V2:
-    default:
-      throw ParseError("Unknown encoding for StructColumnReader");
-    }
-  }
-
-  StructColumnReader::~StructColumnReader() {
-    for (size_t i=0; i<children.size(); i++) {
-      delete children[i];
-    }
-  }
-
-  uint64_t StructColumnReader::skip(uint64_t numValues) {
-    numValues = ColumnReader::skip(numValues);
-    for(std::vector<ColumnReader*>::iterator ptr=children.begin(); ptr != children.end(); ++ptr) {
-      (*ptr)->skip(numValues);
-    }
-    return numValues;
-  }
-
-  void StructColumnReader::next(ColumnVectorBatch& rowBatch,
-                                uint64_t numValues,
-                                char *notNull) {
-    ColumnReader::next(rowBatch, numValues, notNull);
-    uint64_t i=0;
-    notNull = rowBatch.hasNulls? rowBatch.notNull.data() : 0;
-    for(std::vector<ColumnReader*>::iterator ptr=children.begin();
-        ptr != children.end(); ++ptr, ++i) {
-      (*ptr)->next(*(dynamic_cast<StructVectorBatch&>(rowBatch).fields[i]),
-                   numValues, notNull);
-    }
-  }
-
-  class ListColumnReader: public ColumnReader {
-  private:
-    std::unique_ptr<ColumnReader> child;
-    std::unique_ptr<RleDecoder> rle;
-
-  public:
-    ListColumnReader(const Type& type, StripeStreams& stipe);
-    ~ListColumnReader();
-
-    uint64_t skip(uint64_t numValues) override;
-
-    void next(ColumnVectorBatch& rowBatch,
-              uint64_t numValues,
-              char *notNull) override;
-  };
-
-  ListColumnReader::ListColumnReader(const Type& type,
-                                     StripeStreams& stripe
-                                     ): ColumnReader(type, stripe) {
-    // count the number of selected sub-columns
-    const std::vector<bool> selectedColumns = stripe.getSelectedColumns();
-    RleVersion vers = convertRleVersion(stripe.getEncoding(columnId).kind());
-    rle = createRleDecoder(stripe.getStream(columnId,
-                                            proto::Stream_Kind_LENGTH,
-                                            true),
-                           false, vers, memoryPool);
-    const Type& childType = type.getSubtype(0);
-    if (selectedColumns[static_cast<uint64_t>(childType.getColumnId())]) {
-      child = buildReader(childType, stripe);
-    }
-  }
-
-  ListColumnReader::~ListColumnReader() {
-    // PASS
-  }
-
-  uint64_t ListColumnReader::skip(uint64_t numValues) {
-    numValues = ColumnReader::skip(numValues);
-    ColumnReader *childReader = child.get();
-    if (childReader) {
-      const uint64_t BUFFER_SIZE = 1024;
-      int64_t buffer[BUFFER_SIZE];
-      uint64_t childrenElements = 0;
-      uint64_t lengthsRead = 0;
-      while (lengthsRead < numValues) {
-        uint64_t chunk = std::min(numValues - lengthsRead, BUFFER_SIZE);
-        rle->next(buffer, chunk, 0);
-        for(size_t i=0; i < chunk; ++i) {
-          childrenElements += static_cast<size_t>(buffer[i]);
-        }
-        lengthsRead += chunk;
-      }
-      childReader->skip(childrenElements);
-    } else {
-      rle->skip(numValues);
-    }
-    return numValues;
-  }
-
-  void ListColumnReader::next(ColumnVectorBatch& rowBatch,
-                              uint64_t numValues,
-                              char *notNull) {
-    ColumnReader::next(rowBatch, numValues, notNull);
-    ListVectorBatch &listBatch = dynamic_cast<ListVectorBatch&>(rowBatch);
-    int64_t* offsets = listBatch.offsets.data();
-    notNull = listBatch.hasNulls ? listBatch.notNull.data() : 0;
-    rle->next(offsets, numValues, notNull);
-    uint64_t totalChildren = 0;
-    if (notNull) {
-      for(size_t i=0; i < numValues; ++i) {
-        if (notNull[i]) {
-          uint64_t tmp = static_cast<uint64_t>(offsets[i]);
-          offsets[i] = static_cast<int64_t>(totalChildren);
-          totalChildren += tmp;
-        } else {
-          offsets[i] = static_cast<int64_t>(totalChildren);
-        }
-      }
-    } else {
-      for(size_t i=0; i < numValues; ++i) {
-        uint64_t tmp = static_cast<uint64_t>(offsets[i]);
-        offsets[i] = static_cast<int64_t>(totalChildren);
-        totalChildren += tmp;
-      }
-    }
-    offsets[numValues] = static_cast<int64_t>(totalChildren);
-    ColumnReader *childReader = child.get();
-    if (childReader) {
-      childReader->next(*(listBatch.elements.get()), totalChildren, 0);
-    }
-  }
-
-  class MapColumnReader: public ColumnReader {
-  private:
-    std::unique_ptr<ColumnReader> keyReader;
-    std::unique_ptr<ColumnReader> elementReader;
-    std::unique_ptr<RleDecoder> rle;
-
-  public:
-    MapColumnReader(const Type& type, StripeStreams& stipe);
-    ~MapColumnReader();
-
-    uint64_t skip(uint64_t numValues) override;
-
-    void next(ColumnVectorBatch& rowBatch,
-              uint64_t numValues,
-              char *notNull) override;
-  };
-
-  MapColumnReader::MapColumnReader(const Type& type,
-                                   StripeStreams& stripe
-                                   ): ColumnReader(type, stripe) {
-    // Determine if the key and/or value columns are selected
-    const std::vector<bool> selectedColumns = stripe.getSelectedColumns();
-    RleVersion vers = convertRleVersion(stripe.getEncoding(columnId).kind());
-    rle = createRleDecoder(stripe.getStream(columnId,
-                                            proto::Stream_Kind_LENGTH,
-                                            true),
-                           false, vers, memoryPool);
-    const Type& keyType = type.getSubtype(0);
-    if (selectedColumns[static_cast<uint64_t>(keyType.getColumnId())]) {
-      keyReader = buildReader(keyType, stripe);
-    }
-    const Type& elementType = type.getSubtype(1);
-    if (selectedColumns[static_cast<uint64_t>(elementType.getColumnId())]) {
-      elementReader = buildReader(elementType, stripe);
-    }
-  }
-
-  MapColumnReader::~MapColumnReader() {
-    // PASS
-  }
-
-  uint64_t MapColumnReader::skip(uint64_t numValues) {
-    numValues = ColumnReader::skip(numValues);
-    ColumnReader *rawKeyReader = keyReader.get();
-    ColumnReader *rawElementReader = elementReader.get();
-    if (rawKeyReader || rawElementReader) {
-      const uint64_t BUFFER_SIZE = 1024;
-      int64_t buffer[BUFFER_SIZE];
-      uint64_t childrenElements = 0;
-      uint64_t lengthsRead = 0;
-      while (lengthsRead < numValues) {
-        uint64_t chunk = std::min(numValues - lengthsRead, BUFFER_SIZE);
-        rle->next(buffer, chunk, 0);
-        for(size_t i=0; i < chunk; ++i) {
-          childrenElements += static_cast<size_t>(buffer[i]);
-        }
-        lengthsRead += chunk;
-      }
-      if (rawKeyReader) {
-        rawKeyReader->skip(childrenElements);
-      }
-      if (rawElementReader) {
-        rawElementReader->skip(childrenElements);
-      }
-    } else {
-      rle->skip(numValues);
-    }
-    return numValues;
-  }
-
-  void MapColumnReader::next(ColumnVectorBatch& rowBatch,
-                             uint64_t numValues,
-                             char *notNull) {
-    ColumnReader::next(rowBatch, numValues, notNull);
-    MapVectorBatch &mapBatch = dynamic_cast<MapVectorBatch&>(rowBatch);
-    int64_t* offsets = mapBatch.offsets.data();
-    notNull = mapBatch.hasNulls ? mapBatch.notNull.data() : 0;
-    rle->next(offsets, numValues, notNull);
-    uint64_t totalChildren = 0;
-    if (notNull) {
-      for(size_t i=0; i < numValues; ++i) {
-        if (notNull[i]) {
-          uint64_t tmp = static_cast<uint64_t>(offsets[i]);
-          offsets[i] = static_cast<int64_t>(totalChildren);
-          totalChildren += tmp;
-        } else {
-          offsets[i] = static_cast<int64_t>(totalChildren);
-        }
-      }
-    } else {
-      for(size_t i=0; i < numValues; ++i) {
-        uint64_t tmp = static_cast<uint64_t>(offsets[i]);
-        offsets[i] = static_cast<int64_t>(totalChildren);
-        totalChildren += tmp;
-      }
-    }
-    offsets[numValues] = static_cast<int64_t>(totalChildren);
-    ColumnReader *rawKeyReader = keyReader.get();
-    if (rawKeyReader) {
-      rawKeyReader->next(*(mapBatch.keys.get()), totalChildren, 0);
-    }
-    ColumnReader *rawElementReader = elementReader.get();
-    if (rawElementReader) {
-      rawElementReader->next(*(mapBatch.elements.get()), totalChildren, 0);
-    }
-  }
-
-  class UnionColumnReader: public ColumnReader {
-  private:
-    std::unique_ptr<ByteRleDecoder> rle;
-    std::vector<ColumnReader*> childrenReader;
-    std::vector<int64_t> childrenCounts;
-    uint64_t numChildren;
-
-  public:
-    UnionColumnReader(const Type& type, StripeStreams& stipe);
-    ~UnionColumnReader();
-
-    uint64_t skip(uint64_t numValues) override;
-
-    void next(ColumnVectorBatch& rowBatch,
-              uint64_t numValues,
-              char *notNull) override;
-  };
-
-  UnionColumnReader::UnionColumnReader(const Type& type,
-                                       StripeStreams& stripe
-                                       ): ColumnReader(type, stripe) {
-    numChildren = type.getSubtypeCount();
-    childrenReader.resize(numChildren);
-    childrenCounts.resize(numChildren);
-
-    rle = createByteRleDecoder(stripe.getStream(columnId,
-                                                proto::Stream_Kind_DATA,
-                                                true));
-    // figure out which types are selected
-    const std::vector<bool> selectedColumns = stripe.getSelectedColumns();
-    for(unsigned int i=0; i < numChildren; ++i) {
-      const Type &child = type.getSubtype(i);
-      if (selectedColumns[static_cast<size_t>(child.getColumnId())]) {
-        childrenReader[i] = buildReader(child, stripe).release();
-      }
-    }
-  }
-
-  UnionColumnReader::~UnionColumnReader() {
-    for(std::vector<ColumnReader*>::iterator itr = childrenReader.begin();
-        itr != childrenReader.end(); ++itr) {
-      delete *itr;
-    }
-  }
-
-  uint64_t UnionColumnReader::skip(uint64_t numValues) {
-    numValues = ColumnReader::skip(numValues);
-    const uint64_t BUFFER_SIZE = 1024;
-    char buffer[BUFFER_SIZE];
-    uint64_t lengthsRead = 0;
-    int64_t *counts = childrenCounts.data();
-    memset(counts, 0, sizeof(int64_t) * numChildren);
-    while (lengthsRead < numValues) {
-      uint64_t chunk = std::min(numValues - lengthsRead, BUFFER_SIZE);
-      rle->next(buffer, chunk, 0);
-      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) {
-        childrenReader[i]->skip(static_cast<uint64_t>(counts[i]));
-      }
-    }
-    return numValues;
-  }
-
-  void UnionColumnReader::next(ColumnVectorBatch& rowBatch,
-                               uint64_t numValues,
-                               char *notNull) {
-    ColumnReader::next(rowBatch, numValues, notNull);
-    UnionVectorBatch &unionBatch = dynamic_cast<UnionVectorBatch&>(rowBatch);
-    uint64_t* offsets = unionBatch.offsets.data();
-    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;
-    rle->next(reinterpret_cast<char *>(tags), numValues, notNull);
-    // set the offsets for each row
-    if (notNull) {
-      for(size_t i=0; i < numValues; ++i) {
-        if (notNull[i]) {
-          offsets[i] =
-            static_cast<uint64_t>(counts[static_cast<size_t>(tags[i])]++);
-        }
-      }
-    } else {
-      for(size_t i=0; i < numValues; ++i) {
-        offsets[i] =
-          static_cast<uint64_t>(counts[static_cast<size_t>(tags[i])]++);
-      }
-    }
-    // read the right number of each child column
-    for(size_t i=0; i < numChildren; ++i) {
-      if (childrenReader[i] != nullptr) {
-        childrenReader[i]->next(*(unionBatch.children[i]),
-                                static_cast<uint64_t>(counts[i]), nullptr);
-      }
-    }
-  }
-
-  /**
-   * Destructively convert the number from zigzag encoding to the
-   * natural signed representation.
-   */
-  void unZigZagInt128(Int128& value) {
-    bool needsNegate = value.getLowBits() & 1;
-    value >>= 1;
-    if (needsNegate) {
-      value.negate();
-      value -= 1;
-    }
-  }
-
-  class Decimal64ColumnReader: public ColumnReader {
-  public:
-    static const uint32_t MAX_PRECISION_64 = 18;
-    static const uint32_t MAX_PRECISION_128 = 38;
-    static const int64_t POWERS_OF_TEN[MAX_PRECISION_64 + 1];
-
-  protected:
-    std::unique_ptr<SeekableInputStream> valueStream;
-    int32_t precision;
-    int32_t scale;
-    const char* buffer;
-    const char* bufferEnd;
-
-    std::unique_ptr<RleDecoder> scaleDecoder;
-
-    /**
-     * Read the valueStream for more bytes.
-     */
-    void readBuffer() {
-      while (buffer == bufferEnd) {
-        int length;
-        if (!valueStream->Next(reinterpret_cast<const void**>(&buffer),
-                               &length)) {
-          throw ParseError("Read past end of stream in Decimal64ColumnReader "+
-                           valueStream->getName());
-        }
-        bufferEnd = buffer + length;
-      }
-    }
-
-    void readInt64(int64_t& value, int32_t currentScale) {
-      value = 0;
-      size_t offset = 0;
-      while (true) {
-        readBuffer();
-        unsigned char ch = static_cast<unsigned char>(*(buffer++));
-        value |= static_cast<uint64_t>(ch & 0x7f) << offset;
-        offset += 7;
-        if (!(ch & 0x80)) {
-          break;
-        }
-      }
-      value = unZigZag(static_cast<uint64_t>(value));
-      if (scale > currentScale) {
-        value *= POWERS_OF_TEN[scale - currentScale];
-      } else if (scale < currentScale) {
-        value /= POWERS_OF_TEN[currentScale - scale];
-      }
-    }
-
-  public:
-    Decimal64ColumnReader(const Type& type, StripeStreams& stipe);
-    ~Decimal64ColumnReader();
-
-    uint64_t skip(uint64_t numValues) override;
-
-    void next(ColumnVectorBatch& rowBatch,
-              uint64_t numValues,
-              char *notNull) override;
-  };
-  const uint32_t Decimal64ColumnReader::MAX_PRECISION_64;
-  const uint32_t Decimal64ColumnReader::MAX_PRECISION_128;
-  const int64_t Decimal64ColumnReader::POWERS_OF_TEN[MAX_PRECISION_64 + 1]=
-    {1,
-     10,
-     100,
-     1000,
-     10000,
-     100000,
-     1000000,
-     10000000,
-     100000000,
-     1000000000,
-     10000000000,
-     100000000000,
-     1000000000000,
-     10000000000000,
-     100000000000000,
-     1000000000000000,
-     10000000000000000,
-     100000000000000000,
-     1000000000000000000};
-
-  Decimal64ColumnReader::Decimal64ColumnReader(const Type& type,
-                                               StripeStreams& stripe
-                                               ): ColumnReader(type, stripe) {
-    scale = static_cast<int32_t>(type.getScale());
-    precision = static_cast<int32_t>(type.getPrecision());
-    valueStream = stripe.getStream(columnId, proto::Stream_Kind_DATA, true);
-    buffer = nullptr;
-    bufferEnd = nullptr;
-    RleVersion vers = convertRleVersion(stripe.getEncoding(columnId).kind());
-    scaleDecoder = createRleDecoder(stripe.getStream
-                                    (columnId,
-                                     proto::Stream_Kind_SECONDARY,
-                                     true),
-                                    true, vers, memoryPool);
-  }
-
-  Decimal64ColumnReader::~Decimal64ColumnReader() {
-    // PASS
-  }
-
-  uint64_t Decimal64ColumnReader::skip(uint64_t numValues) {
-    numValues = ColumnReader::skip(numValues);
-    uint64_t skipped = 0;
-    while (skipped < numValues) {
-      readBuffer();
-      if (!(0x80 & *(buffer++))) {
-        skipped += 1;
-      }
-    }
-    scaleDecoder->skip(numValues);
-    return numValues;
-  }
-
-  void Decimal64ColumnReader::next(ColumnVectorBatch& rowBatch,
-                                   uint64_t numValues,
-                                   char *notNull) {
-    ColumnReader::next(rowBatch, numValues, notNull);
-    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : 0;
-    Decimal64VectorBatch &batch =
-      dynamic_cast<Decimal64VectorBatch&>(rowBatch);
-    int64_t* values = batch.values.data();
-    // read the next group of scales
-    int64_t* scaleBuffer = batch.readScales.data();
-    scaleDecoder->next(scaleBuffer, numValues, notNull);
-    batch.precision = precision;
-    batch.scale = scale;
-    if (notNull) {
-      for(size_t i=0; i < numValues; ++i) {
-        if (notNull[i]) {
-          readInt64(values[i], static_cast<int32_t>(scaleBuffer[i]));
-        }
-      }
-    } else {
-      for(size_t i=0; i < numValues; ++i) {
-        readInt64(values[i], static_cast<int32_t>(scaleBuffer[i]));
-      }
-    }
-  }
-
-  void scaleInt128(Int128& value, uint32_t scale, uint32_t currentScale) {
-    if (scale > currentScale) {
-      while(scale > currentScale) {
-        uint32_t scaleAdjust =
-          std::min(Decimal64ColumnReader::MAX_PRECISION_64,
-                   scale - currentScale);
-        value *= Decimal64ColumnReader::POWERS_OF_TEN[scaleAdjust];
-        currentScale += scaleAdjust;
-      }
-    } else if (scale < currentScale) {
-      Int128 remainder;
-      while(currentScale > scale) {
-        uint32_t scaleAdjust =
-          std::min(Decimal64ColumnReader::MAX_PRECISION_64,
-                   currentScale - scale);
-        value = value.divide(Decimal64ColumnReader::POWERS_OF_TEN[scaleAdjust],
-                             remainder);
-        currentScale -= scaleAdjust;
-      }
-    }
-  }
-
-  class Decimal128ColumnReader: public Decimal64ColumnReader {
-  public:
-    Decimal128ColumnReader(const Type& type, StripeStreams& stipe);
-    ~Decimal128ColumnReader();
-
-    void next(ColumnVectorBatch& rowBatch,
-              uint64_t numValues,
-              char *notNull) override;
-
-  private:
-    void readInt128(Int128& value, int32_t currentScale) {
-      value = 0;
-      Int128 work;
-      uint32_t offset = 0;
-      while (true) {
-        readBuffer();
-        unsigned char ch = static_cast<unsigned char>(*(buffer++));
-        work = ch & 0x7f;
-        work <<= offset;
-        value |=  work;
-        offset += 7;
-        if (!(ch & 0x80)) {
-          break;
-        }
-      }
-      unZigZagInt128(value);
-      scaleInt128(value, static_cast<uint32_t>(scale), 
-                  static_cast<uint32_t>(currentScale));
-    }
-  };
-
-  Decimal128ColumnReader::Decimal128ColumnReader
-                (const Type& type,
-                 StripeStreams& stripe
-                 ): Decimal64ColumnReader(type, stripe) {
-    // PASS
-  }
-
-  Decimal128ColumnReader::~Decimal128ColumnReader() {
-    // PASS
-  }
-
-  void Decimal128ColumnReader::next(ColumnVectorBatch& rowBatch,
-                                   uint64_t numValues,
-                                   char *notNull) {
-    ColumnReader::next(rowBatch, numValues, notNull);
-    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : 0;
-    Decimal128VectorBatch &batch =
-      dynamic_cast<Decimal128VectorBatch&>(rowBatch);
-    Int128* values = batch.values.data();
-    // read the next group of scales
-    int64_t* scaleBuffer = batch.readScales.data();
-    scaleDecoder->next(scaleBuffer, numValues, notNull);
-    batch.precision = precision;
-    batch.scale = scale;
-    if (notNull) {
-      for(size_t i=0; i < numValues; ++i) {
-        if (notNull[i]) {
-          readInt128(values[i], static_cast<int32_t>(scaleBuffer[i]));
-        }
-      }
-    } else {
-      for(size_t i=0; i < numValues; ++i) {
-        readInt128(values[i], static_cast<int32_t>(scaleBuffer[i]));
-      }
-    }
-  }
-
-  class DecimalHive11ColumnReader: public Decimal64ColumnReader {
-  private:
-    bool throwOnOverflow;
-    std::ostream* errorStream;
-
-    /**
-     * Read an Int128 from the stream and correct it to the desired scale.
-     */
-    bool readInt128(Int128& value, int32_t currentScale) {
-      // -/+ 99999999999999999999999999999999999999
-      static const Int128 MIN_VALUE(-0x4b3b4ca85a86c47b, 0xf675ddc000000001);
-      static const Int128 MAX_VALUE( 0x4b3b4ca85a86c47a, 0x098a223fffffffff);
-
-      value = 0;
-      Int128 work;
-      uint32_t offset = 0;
-      bool result = true;
-      while (true) {
-        readBuffer();
-        unsigned char ch = static_cast<unsigned char>(*(buffer++));
-        work = ch & 0x7f;
-        // If we have read more than 128 bits, we flag the error, but keep
-        // reading bytes so the stream isn't thrown off.
-        if (offset > 128 || (offset == 126 && work > 3)) {
-          result = false;
-        }
-        work <<= offset;
-        value |=  work;
-        offset += 7;
-        if (!(ch & 0x80)) {
-          break;
-        }
-      }
-
-      if (!result) {
-        return result;
-      }
-      unZigZagInt128(value);
-      scaleInt128(value, static_cast<uint32_t>(scale),
-                  static_cast<uint32_t>(currentScale));
-      return value >= MIN_VALUE && value <= MAX_VALUE;
-    }
-
-  public:
-    DecimalHive11ColumnReader(const Type& type, StripeStreams& stipe);
-    ~DecimalHive11ColumnReader();
-
-    void next(ColumnVectorBatch& rowBatch,
-              uint64_t numValues,
-              char *notNull) override;
-  };
-
-  DecimalHive11ColumnReader::DecimalHive11ColumnReader
-                    (const Type& type,
-                     StripeStreams& stripe
-                     ): Decimal64ColumnReader(type, stripe) {
-    const ReaderOptions options = stripe.getReaderOptions();
-    scale = options.getForcedScaleOnHive11Decimal();
-    throwOnOverflow = options.getThrowOnHive11DecimalOverflow();
-    errorStream = options.getErrorStream();
-  }
-
-  DecimalHive11ColumnReader::~DecimalHive11ColumnReader() {
-    // PASS
-  }
-
-  void DecimalHive11ColumnReader::next(ColumnVectorBatch& rowBatch,
-                                       uint64_t numValues,
-                                       char *notNull) {
-    ColumnReader::next(rowBatch, numValues, notNull);
-    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : 0;
-    Decimal128VectorBatch &batch =
-      dynamic_cast<Decimal128VectorBatch&>(rowBatch);
-    Int128* values = batch.values.data();
-    // read the next group of scales
-    int64_t* scaleBuffer = batch.readScales.data();
-
-    scaleDecoder->next(scaleBuffer, numValues, notNull);
-
-    batch.precision = precision;
-    batch.scale = scale;
-    if (notNull) {
-      for(size_t i=0; i < numValues; ++i) {
-        if (notNull[i]) {
-          if (!readInt128(values[i],
-                          static_cast<int32_t>(scaleBuffer[i]))) {
-            if (throwOnOverflow) {
-              throw ParseError("Hive 0.11 decimal was more than 38 digits.");
-            } else {
-              *errorStream << "Warning: "
-                           << "Hive 0.11 decimal with more than 38 digits "
-                           << "replaced by NULL.\n";
-              notNull[i] = false;
-            }
-          }
-        }
-      }
-    } else {
-      for(size_t i=0; i < numValues; ++i) {
-        if (!readInt128(values[i],
-                        static_cast<int32_t>(scaleBuffer[i]))) {
-          if (throwOnOverflow) {
-            throw ParseError("Hive 0.11 decimal was more than 38 digits.");
-          } else {
-            *errorStream << "Warning: "
-                         << "Hive 0.11 decimal with more than 38 digits "
-                         << "replaced by NULL.\n";
-            batch.hasNulls = true;
-            batch.notNull[i] = false;
-          }
-        }
-      }
-    }
-  }
-
-  /**
-   * Create a reader for the given stripe.
-   */
-  std::unique_ptr<ColumnReader> buildReader(const Type& type,
-                                            StripeStreams& stripe) {
-    switch (static_cast<int64_t>(type.getKind())) {
-    case DATE:
-    case INT:
-    case LONG:
-    case SHORT:
-      return std::unique_ptr<ColumnReader>(
-          new IntegerColumnReader(type, stripe));
-    case BINARY:
-    case CHAR:
-    case STRING:
-    case VARCHAR:
-      switch (static_cast<int64_t>(stripe.getEncoding(type.getColumnId()).kind())){
-      case proto::ColumnEncoding_Kind_DICTIONARY:
-      case proto::ColumnEncoding_Kind_DICTIONARY_V2:
-        return std::unique_ptr<ColumnReader>(
-            new StringDictionaryColumnReader(type, stripe));
-      case proto::ColumnEncoding_Kind_DIRECT:
-      case proto::ColumnEncoding_Kind_DIRECT_V2:
-        return std::unique_ptr<ColumnReader>(
-            new StringDirectColumnReader(type, stripe));
-      default:
-        throw NotImplementedYet("buildReader unhandled string encoding");
-      }
-
-    case BOOLEAN:
-      return std::unique_ptr<ColumnReader>(
-          new BooleanColumnReader(type, stripe));
-
-    case BYTE:
-      return std::unique_ptr<ColumnReader>(
-          new ByteColumnReader(type, stripe));
-
-    case LIST:
-      return std::unique_ptr<ColumnReader>(
-          new ListColumnReader(type, stripe));
-
-    case MAP:
-      return std::unique_ptr<ColumnReader>(
-          new MapColumnReader(type, stripe));
-
-    case UNION:
-      return std::unique_ptr<ColumnReader>(
-          new UnionColumnReader(type, stripe));
-
-    case STRUCT:
-      return std::unique_ptr<ColumnReader>(
-          new StructColumnReader(type, stripe));
-
-    case FLOAT:
-    case DOUBLE:
-      return std::unique_ptr<ColumnReader>(
-          new DoubleColumnReader(type, stripe));
-
-    case TIMESTAMP:
-      return std::unique_ptr<ColumnReader>
-        (new TimestampColumnReader(type, stripe));
-
-    case DECIMAL:
-      // is this a Hive 0.11 or 0.12 file?
-      if (type.getPrecision() == 0) {
-        return std::unique_ptr<ColumnReader>
-          (new DecimalHive11ColumnReader(type, stripe));
-
-      // can we represent the values using int64_t?
-      } else if (type.getPrecision() <=
-                 Decimal64ColumnReader::MAX_PRECISION_64) {
-        return std::unique_ptr<ColumnReader>
-          (new Decimal64ColumnReader(type, stripe));
-
-      // otherwise we use the Int128 implementation
-      } else {
-        return std::unique_ptr<ColumnReader>
-          (new Decimal128ColumnReader(type, stripe));
-      }
-
-    default:
-      throw NotImplementedYet("buildReader unhandled type");
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/ColumnReader.hh
----------------------------------------------------------------------
diff --git a/c++/src/orc/ColumnReader.hh b/c++/src/orc/ColumnReader.hh
deleted file mode 100644
index b90c942..0000000
--- a/c++/src/orc/ColumnReader.hh
+++ /dev/null
@@ -1,109 +0,0 @@
-/**
- * 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 ORC_COLUMN_READER_HH
-#define ORC_COLUMN_READER_HH
-
-#include "orc/Vector.hh"
-#include "ByteRLE.hh"
-#include "Compression.hh"
-#include "wrap/orc-proto-wrapper.hh"
-
-namespace orc {
-
-  class StripeStreams {
-  public:
-    virtual ~StripeStreams();
-
-    /**
-     * Get the reader options.
-     */
-    virtual const ReaderOptions& getReaderOptions() const = 0;
-
-    /**
-     * Get the array of booleans for which columns are selected.
-     * @return the address of an array which contains true at the index of
-     *    each columnId is selected.
-     */
-    virtual const std::vector<bool> getSelectedColumns() const = 0;
-
-    /**
-     * Get the encoding for the given column for this stripe.
-     */
-    virtual proto::ColumnEncoding getEncoding(int64_t columnId) const = 0;
-
-    /**
-     * Get the stream for the given column/kind in this stripe.
-     * @param columnId the id of the column
-     * @param kind the kind of the stream
-     * @param shouldStream should the reading page the stream in
-     * @return the new stream
-     */
-    virtual std::unique_ptr<SeekableInputStream>
-                    getStream(int64_t columnId,
-                              proto::Stream_Kind kind,
-                              bool shouldStream) const = 0;
-
-    /**
-     * Get the memory pool for this reader.
-     */
-    virtual MemoryPool& getMemoryPool() const = 0;
-  };
-
-  /**
-   * The interface for reading ORC data types.
-   */
-  class ColumnReader {
-  protected:
-    std::unique_ptr<ByteRleDecoder> notNullDecoder;
-    int64_t columnId;
-    MemoryPool& memoryPool;
-
-  public:
-    ColumnReader(const Type& type, StripeStreams& stipe);
-
-    virtual ~ColumnReader();
-
-    /**
-     * Skip number of specified rows.
-     * @param numValues the number of values to skip
-     * @return the number of non-null values skipped
-     */
-    virtual uint64_t skip(uint64_t numValues);
-
-    /**
-     * Read the next group of values into this rowBatch.
-     * @param rowBatch the memory to read into.
-     * @param numValues the number of values to read
-     * @param notNull if null, all values are not null. Otherwise, it is
-     *           a mask (with at least numValues bytes) for which values to
-     *           set.
-     */
-    virtual void next(ColumnVectorBatch& rowBatch,
-                      uint64_t numValues,
-                      char* notNull);
-  };
-
-  /**
-   * Create a reader for the given stripe.
-   */
-  std::unique_ptr<ColumnReader> buildReader(const Type& type,
-                                            StripeStreams& stripe);
-}
-
-#endif

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/Compression.cc
----------------------------------------------------------------------
diff --git a/c++/src/orc/Compression.cc b/c++/src/orc/Compression.cc
deleted file mode 100644
index 06e10e0..0000000
--- a/c++/src/orc/Compression.cc
+++ /dev/null
@@ -1,751 +0,0 @@
-/**
- * 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 "orc/Adaptor.hh"
-#include "Compression.hh"
-#include "Exceptions.hh"
-
-#include <algorithm>
-#include <iomanip>
-#include <iostream>
-#include <sstream>
-
-#include "zlib.h"
-
-#include "wrap/snappy-wrapper.h"
-
-namespace orc {
-
-  void printBuffer(std::ostream& out,
-                   const char *buffer,
-                   uint64_t length) {
-    const uint64_t width = 24;
-    out << std::hex;
-    for(uint64_t line = 0; line < (length + width - 1) / width; ++line) {
-      out << std::setfill('0') << std::setw(7) << (line * width);
-      for(uint64_t byte = 0;
-          byte < width && line * width + byte < length; ++byte) {
-        out << " " << std::setfill('0') << std::setw(2)
-                  << static_cast<uint64_t>(0xff & buffer[line * width +
-                                                             byte]);
-      }
-      out << "\n";
-    }
-    out << std::dec;
-  }
-
-  PositionProvider::PositionProvider(const std::list<uint64_t>& posns) {
-    position = posns.begin();
-  }
-
-  uint64_t PositionProvider::next() {
-    uint64_t result = *position;
-    ++position;
-    return result;
-  }
-
-  SeekableInputStream::~SeekableInputStream() {
-    // PASS
-  }
-
-  SeekableArrayInputStream::~SeekableArrayInputStream() {
-    // PASS
-  }
-
-  SeekableArrayInputStream::SeekableArrayInputStream
-               (const unsigned char* values,
-                uint64_t size,
-                int64_t blkSize
-                ): data(reinterpret_cast<const char*>(values)) {
-    length = size;
-    position = 0;
-    blockSize = blkSize == -1 ? length : static_cast<uint64_t>(blkSize);
-  }
-
-  SeekableArrayInputStream::SeekableArrayInputStream(const char* values,
-                                                     uint64_t size,
-                                                     int64_t blkSize
-                                                     ): data(values) {
-    length = size;
-    position = 0;
-    blockSize = blkSize == -1 ? length : static_cast<uint64_t>(blkSize);
-  }
-
-  bool SeekableArrayInputStream::Next(const void** buffer, int*size) {
-    uint64_t currentSize = std::min(length - position, blockSize);
-    if (currentSize > 0) {
-      *buffer = data + position;
-      *size = static_cast<int>(currentSize);
-      position += currentSize;
-      return true;
-    }
-    *size = 0;
-    return false;
-  }
-
-  void SeekableArrayInputStream::BackUp(int count) {
-    if (count >= 0) {
-      uint64_t unsignedCount = static_cast<uint64_t>(count);
-      if (unsignedCount <= blockSize && unsignedCount <= position) {
-        position -= unsignedCount;
-      } else {
-        throw std::logic_error("Can't backup that much!");
-      }
-    }
-  }
-
-  bool SeekableArrayInputStream::Skip(int count) {
-    if (count >= 0) {
-      uint64_t unsignedCount = static_cast<uint64_t>(count);
-      if (unsignedCount + position <= length) {
-        position += unsignedCount;
-        return true;
-      } else {
-        position = length;
-      }
-    }
-    return false;
-  }
-
-  google::protobuf::int64 SeekableArrayInputStream::ByteCount() const {
-    return static_cast<google::protobuf::int64>(position);
-  }
-
-  void SeekableArrayInputStream::seek(PositionProvider& seekPosition) {
-    position = seekPosition.next();
-  }
-
-  std::string SeekableArrayInputStream::getName() const {
-    std::ostringstream result;
-    result << "SeekableArrayInputStream " << position << " of " << length;
-    return result.str();
-  }
-
-  static uint64_t computeBlock(int64_t request, uint64_t length) {
-    return std::min(length,
-                    static_cast<uint64_t>(request < 0 ?
-                                          256 * 1024 : request));
-  }
-
-  SeekableFileInputStream::SeekableFileInputStream(InputStream* stream,
-                                                   uint64_t offset,
-                                                   uint64_t byteCount,
-                                                   MemoryPool& _pool,
-                                                   int64_t _blockSize
-                                                   ):pool(_pool),
-                                                     input(stream),
-                                                     start(offset),
-                                                     length(byteCount),
-                                                     blockSize(computeBlock
-                                                               (_blockSize,
-                                                                length)) {
-
-    position = 0;
-    buffer.reset(new DataBuffer<char>(pool));
-    pushBack = 0;
-  }
-
-  SeekableFileInputStream::~SeekableFileInputStream() {
-    // PASS
-  }
-
-  bool SeekableFileInputStream::Next(const void** data, int*size) {
-    uint64_t bytesRead;
-    if (pushBack != 0) {
-      *data = buffer->data() + (buffer->size() - pushBack);
-      bytesRead = pushBack;
-    } else {
-      bytesRead = std::min(length - position, blockSize);
-      buffer->resize(bytesRead);
-      if (bytesRead > 0) {
-        input->read(buffer->data(), bytesRead, start+position);
-        *data = static_cast<void*>(buffer->data());
-      }
-    }
-    position += bytesRead;
-    pushBack = 0;
-    *size = static_cast<int>(bytesRead);
-    return bytesRead != 0;
-  }
-
-  void SeekableFileInputStream::BackUp(int signedCount) {
-    if (signedCount < 0) {
-      throw std::logic_error("can't backup negative distances");
-    }
-    uint64_t count = static_cast<uint64_t>(signedCount);
-    if (pushBack > 0) {
-      throw std::logic_error("can't backup unless we just called Next");
-    }
-    if (count > blockSize || count > position) {
-      throw std::logic_error("can't backup that far");
-    }
-    pushBack = static_cast<uint64_t>(count);
-    position -= pushBack;
-  }
-
-  bool SeekableFileInputStream::Skip(int signedCount) {
-    if (signedCount < 0) {
-      return false;
-    }
-    uint64_t count = static_cast<uint64_t>(signedCount);
-    position = std::min(position + count, length);
-    pushBack = 0;
-    return position < length;
-  }
-
-  int64_t SeekableFileInputStream::ByteCount() const {
-    return static_cast<int64_t>(position);
-  }
-
-  void SeekableFileInputStream::seek(PositionProvider& location) {
-    position = location.next();
-    if (position > length) {
-      position = length;
-      throw std::logic_error("seek too far");
-    }
-    pushBack = 0;
-  }
-
-  std::string SeekableFileInputStream::getName() const {
-    std::ostringstream result;
-    result << input->getName() << " from " << start << " for "
-           << length;
-    return result.str();
-  }
-
-  enum DecompressState { DECOMPRESS_HEADER,
-                         DECOMPRESS_START,
-                         DECOMPRESS_CONTINUE,
-                         DECOMPRESS_ORIGINAL,
-                         DECOMPRESS_EOF};
-
-  class ZlibDecompressionStream: public SeekableInputStream {
-  public:
-    ZlibDecompressionStream(std::unique_ptr<SeekableInputStream> inStream,
-                            size_t blockSize,
-                            MemoryPool& pool);
-    virtual ~ZlibDecompressionStream();
-    virtual bool Next(const void** data, int*size) override;
-    virtual void BackUp(int count) override;
-    virtual bool Skip(int count) override;
-    virtual int64_t ByteCount() const override;
-    virtual void seek(PositionProvider& position) override;
-    virtual std::string getName() const override;
-
-  private:
-    void readBuffer(bool failOnEof) {
-      int length;
-      if (!input->Next(reinterpret_cast<const void**>(&inputBuffer),
-                       &length)) {
-        if (failOnEof) {
-          throw ParseError("Read past EOF in "
-                           "ZlibDecompressionStream::readBuffer");
-        }
-        state = DECOMPRESS_EOF;
-        inputBuffer = nullptr;
-        inputBufferEnd = nullptr;
-      } else {
-        inputBufferEnd = inputBuffer + length;
-      }
-    }
-
-    uint32_t readByte(bool failOnEof) {
-      if (inputBuffer == inputBufferEnd) {
-        readBuffer(failOnEof);
-        if (state == DECOMPRESS_EOF) {
-          return 0;
-        }
-      }
-      return static_cast<unsigned char>(*(inputBuffer++));
-    }
-
-    void readHeader() {
-      uint32_t header = readByte(false);
-      if (state != DECOMPRESS_EOF) {
-        header |= readByte(true) << 8;
-        header |= readByte(true) << 16;
-        if (header & 1) {
-          state = DECOMPRESS_ORIGINAL;
-        } else {
-          state = DECOMPRESS_START;
-        }
-        remainingLength = header >> 1;
-      } else {
-        remainingLength = 0;
-      }
-    }
-
-    MemoryPool& pool;
-    const size_t blockSize;
-    std::unique_ptr<SeekableInputStream> input;
-    z_stream zstream;
-    DataBuffer<char> buffer;
-
-    // the current state
-    DecompressState state;
-
-    // the start of the current buffer
-    // This pointer is not owned by us. It is either owned by zstream or
-    // the underlying stream.
-    const char* outputBuffer;
-    // the size of the current buffer
-    size_t outputBufferLength;
-    // the size of the current chunk
-    size_t remainingLength;
-
-    // the last buffer returned from the input
-    const char *inputBuffer;
-    const char *inputBufferEnd;
-
-    // roughly the number of bytes returned
-    off_t bytesReturned;
-  };
-
-DIAGNOSTIC_PUSH
-DIAGNOSTIC_IGNORE("-Wold-style-cast")
-
-  ZlibDecompressionStream::ZlibDecompressionStream
-                   (std::unique_ptr<SeekableInputStream> inStream,
-                    size_t _blockSize,
-                    MemoryPool& _pool
-                    ): pool(_pool),
-                       blockSize(_blockSize),
-                       buffer(pool, _blockSize) {
-    input.reset(inStream.release());
-    zstream.next_in = Z_NULL;
-    zstream.avail_in = 0;
-    zstream.zalloc = Z_NULL;
-    zstream.zfree = Z_NULL;
-    zstream.opaque = Z_NULL;
-    zstream.next_out = reinterpret_cast<Bytef*>(buffer.data());
-    zstream.avail_out = static_cast<uInt>(blockSize);
-    int64_t result = inflateInit2(&zstream, -15);
-    switch (result) {
-    case Z_OK:
-      break;
-    case Z_MEM_ERROR:
-      throw std::logic_error("Memory error from inflateInit2");
-    case Z_VERSION_ERROR:
-      throw std::logic_error("Version error from inflateInit2");
-    case Z_STREAM_ERROR:
-      throw std::logic_error("Stream error from inflateInit2");
-    default:
-      throw std::logic_error("Unknown error from inflateInit2");
-    }
-    outputBuffer = nullptr;
-    outputBufferLength = 0;
-    remainingLength = 0;
-    state = DECOMPRESS_HEADER;
-    inputBuffer = nullptr;
-    inputBufferEnd = nullptr;
-    bytesReturned = 0;
-  }
-
-DIAGNOSTIC_POP
-
-  ZlibDecompressionStream::~ZlibDecompressionStream() {
-    int64_t result = inflateEnd(&zstream);
-    if (result != Z_OK) {
-      // really can't throw in destructors
-      std::cout << "Error in ~ZlibDecompressionStream() " << result << "\n";
-    }
-  }
-
-  bool ZlibDecompressionStream::Next(const void** data, int*size) {
-    // if the user pushed back, return them the partial buffer
-    if (outputBufferLength) {
-      *data = outputBuffer;
-      *size = static_cast<int>(outputBufferLength);
-      outputBuffer += outputBufferLength;
-      outputBufferLength = 0;
-      return true;
-    }
-    if (state == DECOMPRESS_HEADER || remainingLength == 0) {
-      readHeader();
-    }
-    if (state == DECOMPRESS_EOF) {
-      return false;
-    }
-    if (inputBuffer == inputBufferEnd) {
-      readBuffer(true);
-    }
-    size_t availSize =
-      std::min(static_cast<size_t>(inputBufferEnd - inputBuffer),
-               remainingLength);
-    if (state == DECOMPRESS_ORIGINAL) {
-      *data = inputBuffer;
-      *size = static_cast<int>(availSize);
-      outputBuffer = inputBuffer + availSize;
-      outputBufferLength = 0;
-    } else if (state == DECOMPRESS_START) {
-      zstream.next_in =
-        reinterpret_cast<Bytef*>(const_cast<char*>(inputBuffer));
-      zstream.avail_in = static_cast<uInt>(availSize);
-      outputBuffer = buffer.data();
-      zstream.next_out =
-        reinterpret_cast<Bytef*>(const_cast<char*>(outputBuffer));
-      zstream.avail_out = static_cast<uInt>(blockSize);
-      if (inflateReset(&zstream) != Z_OK) {
-        throw std::logic_error("Bad inflateReset in "
-                               "ZlibDecompressionStream::Next");
-      }
-      int64_t result;
-      do {
-        result = inflate(&zstream, availSize == remainingLength ? Z_FINISH :
-                         Z_SYNC_FLUSH);
-        switch (result) {
-        case Z_OK:
-          remainingLength -= availSize;
-          inputBuffer += availSize;
-          readBuffer(true);
-          availSize =
-            std::min(static_cast<size_t>(inputBufferEnd - inputBuffer),
-                     remainingLength);
-          zstream.next_in =
-            reinterpret_cast<Bytef*>(const_cast<char*>(inputBuffer));
-          zstream.avail_in = static_cast<uInt>(availSize);
-          break;
-        case Z_STREAM_END:
-          break;
-        case Z_BUF_ERROR:
-          throw std::logic_error("Buffer error in "
-                                 "ZlibDecompressionStream::Next");
-        case Z_DATA_ERROR:
-          throw std::logic_error("Data error in "
-                                 "ZlibDecompressionStream::Next");
-        case Z_STREAM_ERROR:
-          throw std::logic_error("Stream error in "
-                                 "ZlibDecompressionStream::Next");
-        default:
-          throw std::logic_error("Unknown error in "
-                                 "ZlibDecompressionStream::Next");
-        }
-      } while (result != Z_STREAM_END);
-      *size = static_cast<int>(blockSize - zstream.avail_out);
-      *data = outputBuffer;
-      outputBufferLength = 0;
-      outputBuffer += *size;
-    } else {
-      throw std::logic_error("Unknown compression state in "
-                             "ZlibDecompressionStream::Next");
-    }
-    inputBuffer += availSize;
-    remainingLength -= availSize;
-    bytesReturned += *size;
-    return true;
-  }
-
-  void ZlibDecompressionStream::BackUp(int count) {
-    if (outputBuffer == nullptr || outputBufferLength != 0) {
-      throw std::logic_error("Backup without previous Next in "
-                             "ZlibDecompressionStream");
-    }
-    outputBuffer -= static_cast<size_t>(count);
-    outputBufferLength = static_cast<size_t>(count);
-    bytesReturned -= count;
-  }
-
-  bool ZlibDecompressionStream::Skip(int count) {
-    bytesReturned += count;
-    // this is a stupid implementation for now.
-    // should skip entire blocks without decompressing
-    while (count > 0) {
-      const void *ptr;
-      int len;
-      if (!Next(&ptr, &len)) {
-        return false;
-      }
-      if (len > count) {
-        BackUp(len - count);
-        count = 0;
-      } else {
-        count -= len;
-      }
-    }
-    return true;
-  }
-
-  int64_t ZlibDecompressionStream::ByteCount() const {
-    return bytesReturned;
-  }
-
-  void ZlibDecompressionStream::seek(PositionProvider& position) {
-    input->seek(position);
-    bytesReturned = input->ByteCount();
-    if (!Skip(static_cast<int>(position.next()))) {
-      throw ParseError("Bad skip in ZlibDecompressionStream::seek");
-    }
-  }
-
-  std::string ZlibDecompressionStream::getName() const {
-    std::ostringstream result;
-    result << "zlib(" << input->getName() << ")";
-    return result.str();
-  }
-
-  class SnappyDecompressionStream: public SeekableInputStream {
-  public:
-    SnappyDecompressionStream(std::unique_ptr<SeekableInputStream> inStream,
-                              size_t blockSize,
-                              MemoryPool& pool);
-
-    virtual ~SnappyDecompressionStream() {}
-    virtual bool Next(const void** data, int*size) override;
-    virtual void BackUp(int count) override;
-    virtual bool Skip(int count) override;
-    virtual int64_t ByteCount() const override;
-    virtual void seek(PositionProvider& position) override;
-    virtual std::string getName() const override;
-
-  private:
-    void readBuffer(bool failOnEof) {
-      int length;
-      if (!input->Next(reinterpret_cast<const void**>(&inputBufferPtr),
-                       &length)) {
-        if (failOnEof) {
-          throw ParseError("SnappyDecompressionStream read past EOF");
-        }
-        state = DECOMPRESS_EOF;
-        inputBufferPtr = nullptr;
-        inputBufferPtrEnd = nullptr;
-      } else {
-        inputBufferPtrEnd = inputBufferPtr + length;
-      }
-    }
-
-    uint32_t readByte(bool failOnEof) {
-      if (inputBufferPtr == inputBufferPtrEnd) {
-        readBuffer(failOnEof);
-        if (state == DECOMPRESS_EOF) {
-          return 0;
-        }
-      }
-      return static_cast<unsigned char>(*(inputBufferPtr++));
-    }
-
-    void readHeader() {
-      uint32_t header = readByte(false);
-      if (state != DECOMPRESS_EOF) {
-        header |= readByte(true) << 8;
-        header |= readByte(true) << 16;
-        if (header & 1) {
-          state = DECOMPRESS_ORIGINAL;
-        } else {
-          state = DECOMPRESS_START;
-        }
-        remainingLength = header >> 1;
-      } else {
-        remainingLength = 0;
-      }
-    }
-
-    std::unique_ptr<SeekableInputStream> input;
-    MemoryPool& pool;
-
-    // may need to stitch together multiple input buffers;
-    // to give snappy a contiguous block
-    DataBuffer<char> inputBuffer;
-
-    // uncompressed output
-    DataBuffer<char> outputBuffer;
-
-    // the current state
-    DecompressState state;
-
-    // the start of the current output buffer
-    const char* outputBufferPtr;
-    // the size of the current output buffer
-    size_t outputBufferLength;
-
-    // the size of the current chunk
-    size_t remainingLength;
-
-    // the last buffer returned from the input
-    const char *inputBufferPtr;
-    const char *inputBufferPtrEnd;
-
-    // bytes returned by this stream
-    off_t bytesReturned;
-  };
-
-  SnappyDecompressionStream::SnappyDecompressionStream
-                   (std::unique_ptr<SeekableInputStream> inStream,
-                    size_t bufferSize,
-                    MemoryPool& _pool
-                    ) : pool(_pool),
-                        inputBuffer(pool, bufferSize),
-                        outputBuffer(pool, bufferSize),
-                        state(DECOMPRESS_HEADER),
-                        outputBufferPtr(0),
-                        outputBufferLength(0),
-                        remainingLength(0),
-                        inputBufferPtr(0),
-                        inputBufferPtrEnd(0),
-                        bytesReturned(0) {
-    input.reset(inStream.release());
-  }
-
-  bool SnappyDecompressionStream::Next(const void** data, int*size) {
-    // if the user pushed back, return them the partial buffer
-    if (outputBufferLength) {
-      *data = outputBufferPtr;
-      *size = static_cast<int>(outputBufferLength);
-      outputBufferPtr += outputBufferLength;
-      bytesReturned += outputBufferLength;
-      outputBufferLength = 0;
-      return true;
-    }
-    if (state == DECOMPRESS_HEADER || remainingLength == 0) {
-      readHeader();
-    }
-    if (state == DECOMPRESS_EOF) {
-      return false;
-    }
-    if (inputBufferPtr == inputBufferPtrEnd) {
-      readBuffer(true);
-    }
-
-    size_t availSize =
-      std::min(static_cast<size_t>(inputBufferPtrEnd - inputBufferPtr),
-               remainingLength);
-    if (state == DECOMPRESS_ORIGINAL) {
-      *data = inputBufferPtr;
-      *size = static_cast<int>(availSize);
-      outputBufferPtr = inputBufferPtr + availSize;
-      outputBufferLength = 0;
-      inputBufferPtr += availSize;
-      remainingLength -= availSize;
-    } else if (state == DECOMPRESS_START) {
-      // Get contiguous bytes of compressed block.
-      const char *compressed = inputBufferPtr;
-      if (remainingLength == availSize) {
-          inputBufferPtr += availSize;
-      } else {
-        // Did not read enough from input.
-        if (inputBuffer.capacity() < remainingLength) {
-          inputBuffer.resize(remainingLength);
-        }
-        ::memcpy(inputBuffer.data(), inputBufferPtr, availSize);
-        inputBufferPtr += availSize;
-        compressed = inputBuffer.data();
-
-        for (size_t pos = availSize; pos < remainingLength; ) {
-          readBuffer(true);
-          size_t avail =
-              std::min(static_cast<size_t>(inputBufferPtrEnd - inputBufferPtr),
-                       remainingLength - pos);
-          ::memcpy(inputBuffer.data() + pos, inputBufferPtr, avail);
-          pos += avail;
-          inputBufferPtr += avail;
-        }
-      }
-
-      if (!snappy::GetUncompressedLength(compressed, remainingLength,
-                                         &outputBufferLength)) {
-        throw ParseError("SnappyDecompressionStream choked on corrupt input");
-      }
-
-      if (outputBufferLength > outputBuffer.capacity()) {
-        throw std::logic_error("uncompressed length exceeds block size");
-      }
-
-      if (!snappy::RawUncompress(compressed, remainingLength,
-                                 outputBuffer.data())) {
-        throw ParseError("SnappyDecompressionStream choked on corrupt input");
-      }
-
-      remainingLength = 0;
-      state = DECOMPRESS_HEADER;
-      *data = outputBuffer.data();
-      *size = static_cast<int>(outputBufferLength);
-      outputBufferPtr = outputBuffer.data() + outputBufferLength;
-      outputBufferLength = 0;
-    }
-
-    bytesReturned += *size;
-    return true;
-  }
-
-  void SnappyDecompressionStream::BackUp(int count) {
-    if (outputBufferPtr == nullptr || outputBufferLength != 0) {
-      throw std::logic_error("Backup without previous Next in "
-                             "SnappyDecompressionStream");
-    }
-    outputBufferPtr -= static_cast<size_t>(count);
-    outputBufferLength = static_cast<size_t>(count);
-    bytesReturned -= count;
-  }
-
-  bool SnappyDecompressionStream::Skip(int count) {
-    bytesReturned += count;
-    // this is a stupid implementation for now.
-    // should skip entire blocks without decompressing
-    while (count > 0) {
-      const void *ptr;
-      int len;
-      if (!Next(&ptr, &len)) {
-        return false;
-      }
-      if (len > count) {
-        BackUp(len - count);
-        count = 0;
-      } else {
-        count -= len;
-      }
-    }
-    return true;
-  }
-
-  int64_t SnappyDecompressionStream::ByteCount() const {
-    return bytesReturned;
-  }
-
-  void SnappyDecompressionStream::seek(PositionProvider& position) {
-    input->seek(position);
-    if (!Skip(static_cast<int>(position.next()))) {
-      throw ParseError("Bad skip in SnappyDecompressionStream::seek");
-    }
-  }
-
-  std::string SnappyDecompressionStream::getName() const {
-    std::ostringstream result;
-    result << "snappy(" << input->getName() << ")";
-    return result.str();
-  }
-
-  std::unique_ptr<SeekableInputStream>
-     createDecompressor(CompressionKind kind,
-                        std::unique_ptr<SeekableInputStream> input,
-                        uint64_t blockSize,
-                        MemoryPool& pool) {
-    switch (static_cast<int64_t>(kind)) {
-    case CompressionKind_NONE:
-      return std::move(input);
-    case CompressionKind_ZLIB:
-      return std::unique_ptr<SeekableInputStream>
-        (new ZlibDecompressionStream(std::move(input), blockSize, pool));
-    case CompressionKind_SNAPPY:
-      return std::unique_ptr<SeekableInputStream>
-        (new SnappyDecompressionStream(std::move(input), blockSize, pool));
-    case CompressionKind_LZO:
-    default:
-      throw NotImplementedYet("compression codec");
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/Compression.hh
----------------------------------------------------------------------
diff --git a/c++/src/orc/Compression.hh b/c++/src/orc/Compression.hh
deleted file mode 100644
index 222dc54..0000000
--- a/c++/src/orc/Compression.hh
+++ /dev/null
@@ -1,129 +0,0 @@
-/**
- * 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 ORC_COMPRESSION_HH
-#define ORC_COMPRESSION_HH
-
-#include "orc/Adaptor.hh"
-#include "orc/OrcFile.hh"
-#include "wrap/zero-copy-stream-wrapper.h"
-
-#include <list>
-#include <vector>
-#include <fstream>
-#include <iostream>
-#include <sstream>
-#include <memory>
-
-namespace orc {
-
-  void printBuffer(std::ostream& out,
-                   const char *buffer,
-                   uint64_t length);
-
-  class PositionProvider {
-  private:
-    std::list<uint64_t>::const_iterator position;
-  public:
-    PositionProvider(const std::list<uint64_t>& positions);
-    uint64_t next();
-  };
-
-  /**
-   * A subclass of Google's ZeroCopyInputStream that supports seek.
-   * By extending Google's class, we get the ability to pass it directly
-   * to the protobuf readers.
-   */
-  class SeekableInputStream: public google::protobuf::io::ZeroCopyInputStream {
-  public:
-    virtual ~SeekableInputStream();
-    virtual void seek(PositionProvider& position) = 0;
-    virtual std::string getName() const = 0;
-  };
-
-  /**
-   * Create a seekable input stream based on a memory range.
-   */
-  class SeekableArrayInputStream: public SeekableInputStream {
-  private:
-    const char* data;
-    uint64_t length;
-    uint64_t position;
-    uint64_t blockSize;
-
-  public:
-    SeekableArrayInputStream(const unsigned char* list,
-                             uint64_t length,
-                             int64_t block_size = -1);
-    SeekableArrayInputStream(const char* list,
-                             uint64_t length,
-                             int64_t block_size = -1);
-    virtual ~SeekableArrayInputStream();
-    virtual bool Next(const void** data, int*size) override;
-    virtual void BackUp(int count) override;
-    virtual bool Skip(int count) override;
-    virtual google::protobuf::int64 ByteCount() const override;
-    virtual void seek(PositionProvider& position) override;
-    virtual std::string getName() const override;
-  };
-
-  /**
-   * Create a seekable input stream based on an input stream.
-   */
-  class SeekableFileInputStream: public SeekableInputStream {
-  private:
-    MemoryPool& pool;
-    InputStream* const input;
-    const uint64_t start;
-    const uint64_t length;
-    const uint64_t blockSize;
-    std::unique_ptr<DataBuffer<char> > buffer;
-    uint64_t position;
-    uint64_t pushBack;
-
-  public:
-    SeekableFileInputStream(InputStream* input,
-                            uint64_t offset,
-                            uint64_t byteCount,
-                            MemoryPool& pool,
-                            int64_t blockSize = -1);
-    virtual ~SeekableFileInputStream();
-
-    virtual bool Next(const void** data, int*size) override;
-    virtual void BackUp(int count) override;
-    virtual bool Skip(int count) override;
-    virtual int64_t ByteCount() const override;
-    virtual void seek(PositionProvider& position) override;
-    virtual std::string getName() const override;
-  };
-
-  /**
-   * Create a decompressor for the given compression kind.
-   * @param kind the compression type to implement
-   * @param input the input stream that is the underlying source
-   * @param bufferSize the maximum size of the buffer
-   * @param pool the memory pool
-   */
-  std::unique_ptr<SeekableInputStream>
-     createDecompressor(CompressionKind kind,
-                        std::unique_ptr<SeekableInputStream> input,
-                        uint64_t bufferSize,
-                        MemoryPool& pool);
-}
-
-#endif


[06/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/test/orc/TestRle.cc
----------------------------------------------------------------------
diff --git a/c++/test/orc/TestRle.cc b/c++/test/orc/TestRle.cc
deleted file mode 100644
index af10de0..0000000
--- a/c++/test/orc/TestRle.cc
+++ /dev/null
@@ -1,2639 +0,0 @@
-/**
- * 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 "orc/Adaptor.hh"
-#include "orc/Compression.hh"
-#include "orc/RLE.hh"
-#include "wrap/gtest-wrapper.h"
-#include "OrcTest.hh"
-
-#include <iostream>
-#include <vector>
-
-namespace orc {
-
-std::vector<int64_t> decodeRLEv2(const unsigned char *bytes,
-                                 unsigned long l,
-                                 size_t n,
-                                 size_t count,
-                                 const char* notNull = nullptr) {
-  std::unique_ptr<RleDecoder> rle =
-    createRleDecoder(std::unique_ptr<SeekableInputStream>
-                     (new SeekableArrayInputStream(bytes,l)), true,
-                     RleVersion_2, *getDefaultPool());
-  std::vector<int64_t> results;
-  for (size_t i = 0; i < count; i+=n) {
-    size_t remaining = count - i;
-    size_t nread = std::min(n, remaining);
-    std::vector<int64_t> data(nread);
-    rle->next(data.data(), nread, notNull);
-    if (notNull) {
-      notNull += nread;
-    }
-    results.insert(results.end(), data.begin(), data.end());
-  }
-
-  return results;
-}
-
-void checkResults(const std::vector<int64_t> &e, const std::vector<int64_t> &a,
-                  size_t n, const char* notNull = nullptr) {
-  EXPECT_EQ(e.size(), a.size()) << "vectors differ in size";
-  for (size_t i = 0; i < e.size(); ++i) {
-    if (!notNull || notNull[i]) {
-      EXPECT_EQ(e[i], a[i]) << "Output wrong at " << i << ", n=" << n;
-    }
-  }
-}
-
-TEST(RLEv2, basicDelta0) {
-  const size_t count = 20;
-  std::vector<int64_t> values;
-  for (size_t i = 0; i < count; ++i) {
-    values.push_back(static_cast<int64_t>(i));
-  }
-
-  const unsigned char bytes[] = {0xc0,0x13,0x00,0x02};
-  unsigned long l = sizeof(bytes) / sizeof(char);
-  // Read 1 at a time, then 3 at a time, etc.
-  checkResults(values, decodeRLEv2(bytes, l, 1, count), 1);
-  checkResults(values, decodeRLEv2(bytes, l, 3, count), 3);
-  checkResults(values, decodeRLEv2(bytes, l, 7, count), 7);
-  checkResults(values, decodeRLEv2(bytes, l, count, count), count);
-};
-
-TEST(RLEv2, basicDelta1) {
-  std::vector<int64_t> values(5);
-  values[0] = -500;
-  values[1] = -400;
-  values[2] = -350;
-  values[3] = -325;
-  values[4] = -310;
-
-  const unsigned char bytes[] = {0xce,0x04,0xe7,0x07,0xc8,0x01,0x32,0x19,0x0f};
-  unsigned long l = sizeof(bytes) / sizeof(char);
-  // Read 1 at a time, then 3 at a time, etc.
-  checkResults(values, decodeRLEv2(bytes, l, 1, values.size()), 1);
-  checkResults(values, decodeRLEv2(bytes, l, 3, values.size()), 3);
-  checkResults(values, decodeRLEv2(bytes, l, 7, values.size()), 7);
-  checkResults(values, decodeRLEv2(bytes, l, values.size(), values.size()),
-               values.size());
-};
-
-TEST(RLEv2, basicDelta2) {
-  std::vector<int64_t> values(5);
-  values[0] = -500;
-  values[1] = -600;
-  values[2] = -650;
-  values[3] = -675;
-  values[4] = -710;
-
-  const unsigned char bytes[] = {0xce,0x04,0xe7,0x07,0xc7,0x01,0x32,0x19,0x23};
-  unsigned long l = sizeof(bytes) / sizeof(char);
-  // Read 1 at a time, then 3 at a time, etc.
-  checkResults(values, decodeRLEv2(bytes, l, 1, values.size()), 1);
-  checkResults(values, decodeRLEv2(bytes, l, 3, values.size()), 3);
-  checkResults(values, decodeRLEv2(bytes, l, 7, values.size()), 7);
-  checkResults(values, decodeRLEv2(bytes, l, values.size(), values.size()),
-               values.size());
-};
-
-TEST(RLEv2, basicDelta3) {
-  std::vector<int64_t> values(5);
-  values[0] = 500;
-  values[1] = 400;
-  values[2] = 350;
-  values[3] = 325;
-  values[4] = 310;
-
-  const unsigned char bytes[] = {0xce,0x04,0xe8,0x07,0xc7,0x01,0x32,0x19,0x0f};
-  unsigned long l = sizeof(bytes) / sizeof(char);
-  // Read 1 at a time, then 3 at a time, etc.
-  checkResults(values, decodeRLEv2(bytes, l, 1, values.size()), 1);
-  checkResults(values, decodeRLEv2(bytes, l, 3, values.size()), 3);
-  checkResults(values, decodeRLEv2(bytes, l, 7, values.size()), 7);
-  checkResults(values, decodeRLEv2(bytes, l, values.size(), values.size()),
-               values.size());
-};
-
-TEST(RLEv2, basicDelta4) {
-  std::vector<int64_t> values(5);
-  values[0] = 500;
-  values[1] = 600;
-  values[2] = 650;
-  values[3] = 675;
-  values[4] = 710;
-
-  const unsigned char bytes[] = {0xce,0x04,0xe8,0x07,0xc8,0x01,0x32,0x19,0x23};
-  unsigned long l = sizeof(bytes) / sizeof(char);
-  // Read 1 at a time, then 3 at a time, etc.
-  checkResults(values, decodeRLEv2(bytes, l, 1, values.size()), 1);
-  checkResults(values, decodeRLEv2(bytes, l, 3, values.size()), 3);
-  checkResults(values, decodeRLEv2(bytes, l, 7, values.size()), 7);
-  checkResults(values, decodeRLEv2(bytes, l, values.size(), values.size()),
-               values.size());
-};
-
-TEST(RLEv2, delta0Width) {
-  const unsigned char buffer[] = {0x4e, 0x2, 0x0, 0x1, 0x2, 0xc0, 0x2, 0x42,
-				  0x0};
-  std::unique_ptr<RleDecoder> decoder =
-    createRleDecoder(std::unique_ptr<SeekableInputStream>
-                     (new SeekableArrayInputStream
-                      (buffer, ARRAY_SIZE(buffer))),
-                     false, RleVersion_2, *getDefaultPool());
-  int64_t values[6];
-  decoder->next(values, 6, 0);
-  EXPECT_EQ(0, values[0]);
-  EXPECT_EQ(1, values[1]);
-  EXPECT_EQ(2, values[2]);
-  EXPECT_EQ(0x42, values[3]);
-  EXPECT_EQ(0x42, values[4]);
-  EXPECT_EQ(0x42, values[5]);
-}
-
-TEST(RLEv2, basicDelta0WithNulls) {
-  std::vector<int64_t> values;
-  std::vector<char> notNull;
-  for (size_t i = 0; i < 20; ++i) {
-    values.push_back(static_cast<int64_t>(i));
-    notNull.push_back(true);
-    // throw in a null every third value
-    bool addNull = (i % 3 == 0);
-    if (addNull) {
-      values.push_back(-1);
-      notNull.push_back(false);
-    }
-  }
-
-  const unsigned char bytes[] = {0xc0,0x13,0x00,0x02};
-  unsigned long l = sizeof(bytes) / sizeof(char);
-  const size_t count = values.size();
-  // Read 1 at a time, then 3 at a time, etc.
-  checkResults(values, decodeRLEv2(bytes, l, 1, count, notNull.data()),
-               1, notNull.data());
-  checkResults(values, decodeRLEv2(bytes, l, 3, count, notNull.data()),
-               3, notNull.data());
-  checkResults(values, decodeRLEv2(bytes, l, 7, count, notNull.data()),
-               7, notNull.data());
-  checkResults(values, decodeRLEv2(bytes, l, count, count, notNull.data()),
-               count, notNull.data());
-};
-
-TEST(RLEv2, shortRepeats) {
-  const size_t runLength = 7;
-  const size_t nVals = 10;
-  const size_t count = nVals * runLength;
-  std::vector<int64_t> values;
-  for (size_t i = 0; i < nVals; ++i) {
-    for (size_t j = 0; j < runLength; ++j) {
-      values.push_back(static_cast<int64_t>(i));
-    }
-  }
-
-  const unsigned char bytes[] = {0x04,0x00,0x04,0x02,0x04,0x04,0x04,
-                                 0x06,0x04,0x08,0x04,0x0a,0x04,0x0c,
-                                 0x04,0x0e,0x04,0x10,0x04,0x12};
-  unsigned long l = sizeof(bytes) / sizeof(char);
-  // Read 1 at a time, then 3 at a time, etc.
-  checkResults(values, decodeRLEv2(bytes, l, 1, count), 1);
-  checkResults(values, decodeRLEv2(bytes, l, 3, count), 3);
-  checkResults(values, decodeRLEv2(bytes, l, 7, count), 7);
-  checkResults(values, decodeRLEv2(bytes, l, count, count), count);
-};
-
-TEST(RLEv2, multiByteShortRepeats) {
-  const size_t runLength = 7;
-  const size_t nVals = 3;
-  const size_t count = nVals * runLength;
-  std::vector<int64_t> values;
-  for (size_t i = 0; i < nVals; ++i) {
-    for (size_t j = 0; j < runLength; ++j) {
-      values.push_back(static_cast<int64_t>(i)+(1L<<62));
-    }
-  }
-
-  const unsigned char bytes[] = {0x3c,0x80,0x00,0x00,0x00,0x00,0x00,0x00,0x00,
-                                 0x3c,0x80,0x00,0x00,0x00,0x00,0x00,0x00,0x02,
-                                 0x3c,0x80,0x00,0x00,0x00,0x00,0x00,0x00,0x04};
-  unsigned long l = sizeof(bytes) / sizeof(char);
-  // Read 1 at a time, then 3 at a time, etc.
-  checkResults(values, decodeRLEv2(bytes, l, 1, count), 1);
-  checkResults(values, decodeRLEv2(bytes, l, 3, count), 3);
-  checkResults(values, decodeRLEv2(bytes, l, 7, count), 7);
-  checkResults(values, decodeRLEv2(bytes, l, count, count), count);
-};
-
-TEST(RLEv2, 0to2Repeat1Direct) {
-  const unsigned char buffer[] = {0x46, 0x02, 0x02, 0x40};
-  std::unique_ptr<RleDecoder> rle =
-      createRleDecoder(std::unique_ptr<SeekableInputStream>
-		       (new SeekableArrayInputStream(buffer,
-						     ARRAY_SIZE(buffer))),
-		       true, RleVersion_2, *getDefaultPool());
-  std::vector<int64_t> data(3);
-  rle->next(data.data(), 3, nullptr);
-
-  for (size_t i = 0; i < data.size(); ++i) {
-    EXPECT_EQ(i, data[i]) << "Output wrong at " << i;
-  }
-};
-
-TEST(RLEv2, bitSize2Direct) {
- // 0,1 repeated 10 times (signed ints)
- const size_t count = 20;
- std::vector<int64_t> values;
- for (size_t i = 0; i < count; ++i) {
-     values.push_back(i%2);
- }
-
- const unsigned char bytes[] = {0x42, 0x13, 0x22, 0x22, 0x22, 0x22, 0x22};
- unsigned long l = sizeof(bytes) / sizeof(char);
- // Read 1 at a time, then 3 at a time, etc.
- checkResults(values, decodeRLEv2(bytes, l, 1, count), 1);
- checkResults(values, decodeRLEv2(bytes, l, 3, count), 3);
- checkResults(values, decodeRLEv2(bytes, l, 7, count), 7);
- checkResults(values, decodeRLEv2(bytes, l, count, count), count);
-};
-
-TEST(RLEv2, bitSize4Direct) {
- // 0,2 repeated 10 times (signed ints)
- const size_t count = 20;
- std::vector<int64_t> values;
- for (size_t i = 0; i < count; ++i) {
-     values.push_back((i%2)*2);
- }
-
- const unsigned char bytes[] = {0x46,0x13,0x04,0x04,0x04,0x04,
-                                0x04,0x04,0x04,0x04,0x04,0x04};
- unsigned long l = sizeof(bytes) / sizeof(char);
-
- // Read 1 at a time, then 3 at a time, etc.
- checkResults(values, decodeRLEv2(bytes, l, 1, count), 1);
- checkResults(values, decodeRLEv2(bytes, l, 3, count), 3);
- checkResults(values, decodeRLEv2(bytes, l, 7, count), 7);
- checkResults(values, decodeRLEv2(bytes, l, count, count), count);
-};
-
-TEST(RLEv2, multipleRunsDirect) {
- std::vector<int64_t> values;
- // 0,1 repeated 10 times (signed ints)
- for (size_t i = 0; i < 20; ++i) {
-     values.push_back(i%2);
- }
- // 0,2 repeated 10 times (signed ints)
- for (size_t i = 0; i < 20; ++i) {
-     values.push_back((i%2)*2);
- }
-
- const unsigned char bytes[] = {0x42,0x13,0x22,0x22,0x22,0x22,0x22,
-                                0x46,0x13,0x04,0x04,0x04,0x04,0x04,
-                                0x04,0x04,0x04,0x04,0x04};
- unsigned long l = sizeof(bytes) / sizeof(char);
-
- // Read 1 at a time, then 3 at a time, etc.
- checkResults(values, decodeRLEv2(bytes, l, 1, values.size()), 1);
- checkResults(values, decodeRLEv2(bytes, l, 3, values.size()), 3);
- checkResults(values, decodeRLEv2(bytes, l, 7, values.size()), 7);
- checkResults(values, decodeRLEv2(bytes, l, values.size(), values.size()),
-              values.size());
-};
-
-TEST(RLEv2, largeNegativesDirect) {
-  const unsigned char buffer[] =
-    {0x7e,0x04,0xcf,0xca,0xcc,0x91,0xba,0x38,0x93,0xab,0x00,0x00,
-     0x00,0x00,0x00,0x00,0x00,0x00,0x00,0x00,0x00,0x00,0x00,0x00,
-     0x00,0x02,0x00,0x00,0x00,0x00,0x00,0x00,0x00,0x02,0x99,0xa5,
-     0xcc,0x28,0x03,0xf7,0xe0,0xff};
-  std::unique_ptr<RleDecoder> rle =
-      createRleDecoder(
-          std::unique_ptr<SeekableInputStream>(
-             new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer))),
-          true, RleVersion_2, *getDefaultPool());
-  std::vector<int64_t> data(5);
-  rle->next(data.data(), 5, nullptr);
-
-  EXPECT_EQ(-7486502418706614742, data[0]) << "Output wrong at " << 0;
-  EXPECT_EQ(0, data[1]) << "Output wrong at " << 1;
-  EXPECT_EQ(1, data[2]) << "Output wrong at " << 2;
-  EXPECT_EQ(1, data[3]) << "Output wrong at " << 3;
-  EXPECT_EQ(-5535739865598783616, data[4]) << "Output wrong at " << 4;
-};
-
-TEST(RLEv2, overflowDirect) {
-  std::vector<int64_t> values(4);
-  values[0] = 4513343538618202719l;
-  values[1] = 4513343538618202711l;
-  values[2] = 2911390882471569739l;
-  values[3] = -9181829309989854913l;
-
-  const unsigned char bytes[] = {0x7e,0x03,0x7d,0x45,0x3c,0x12,0x41,0x48,0xf4,
-                                 0xbe,0x7d,0x45,0x3c,0x12,0x41,0x48,0xf4,0xae,
-                                 0x50,0xce,0xad,0x2a,0x30,0x0e,0xd2,0x96,0xfe,
-                                 0xd8,0xd2,0x38,0x54,0x6e,0x3d,0x81};
-  unsigned long l = sizeof(bytes) / sizeof(char);
-  // Read 1 at a time, then 3 at a time, etc.
-  checkResults(values, decodeRLEv2(bytes, l, 1, values.size()), 1);
-  checkResults(values, decodeRLEv2(bytes, l, 3, values.size()), 3);
-  checkResults(values, decodeRLEv2(bytes, l, 7, values.size()), 7);
-  checkResults(values, decodeRLEv2(bytes, l, values.size(), values.size()),
-               values.size());
-};
-
-TEST(RLEv2, basicPatched0) {
-  long v[] = {2030,2000,2020,1000000,2040,2050,2060,2070,2080,2090};
-  std::vector<int64_t> values;
-  for (size_t i = 0; i < sizeof(v) / sizeof(long); ++i) {
-      values.push_back(v[i]);
-  }
-
-  const unsigned char bytes[] = {0x8e,0x09,0x2b,0x21,0x07,0xd0,0x1e,0x00,0x14,
-                                 0x70,0x28,0x32,0x3c,0x46,0x50,0x5a,0xfc,0xe8};
-  unsigned long l = sizeof(bytes) / sizeof(char);
-  // Read 1 at a time, then 3 at a time, etc.
-  checkResults(values, decodeRLEv2(bytes, l, 1, values.size()), 1);
-  checkResults(values, decodeRLEv2(bytes, l, 3, values.size()), 3);
-  checkResults(values, decodeRLEv2(bytes, l, 7, values.size()), 7);
-  checkResults(values, decodeRLEv2(bytes, l, values.size(), values.size()),
-               values.size());
-};
-
-TEST(RLEv2, basicPatched1) {
-  long v[] = {20, 2, 3, 2, 1, 3, 17, 71, 35, 2, 1, 139, 2, 2, 3, 1783, 475, 2,
-              1, 1, 3, 1, 3, 2, 32, 1, 2, 3, 1, 8, 30, 1, 3, 414, 1, 1, 135, 3,
-              3, 1, 414, 2, 1, 2, 2, 594, 2, 5, 6, 4, 11, 1, 2, 2, 1, 1, 52, 4,
-              1, 2, 7, 1, 17, 334, 1, 2, 1, 2, 2, 6, 1, 266, 1, 2, 217, 2, 6, 2,
-              13, 2, 2, 1, 2, 3, 5, 1, 2, 1, 7244, 11813, 1, 33, 2, -13, 1, 2, 3,
-              13, 1, 92, 3, 13, 5, 14, 9, 141, 12, 6, 15, 25};
-  std::vector<int64_t> values;
-  for (size_t i = 0; i < sizeof(v) / sizeof(long); ++i) {
-    values.push_back(v[i]);
-  }
-
-  const unsigned char bytes[] = {0x90,0x6d,0x04,0xa4,0x8d,0x10,0x83,0xc2,0x00,
-                                 0xf0,0x70,0x40,0x3c,0x54,0x18,0x03,0xc1,0xc9,
-                                 0x80,0x78,0x3c,0x21,0x04,0xf4,0x03,0xc1,0xc0,
-                                 0xe0,0x80,0x38,0x20,0x0f,0x16,0x83,0x81,0xe1,
-                                 0x00,0x70,0x54,0x56,0x0e,0x08,0x6a,0xc1,0xc0,
-                                 0xe4,0xa0,0x40,0x20,0x0e,0xd5,0x83,0xc1,0xc0,
-                                 0xf0,0x79,0x7c,0x1e,0x12,0x09,0x84,0x43,0x00,
-                                 0xe0,0x78,0x3c,0x1c,0x0e,0x20,0x84,0x41,0xc0,
-                                 0xf0,0xa0,0x38,0x3d,0x5b,0x07,0x03,0xc1,0xc0,
-                                 0xf0,0x78,0x4c,0x1d,0x17,0x07,0x03,0xdc,0xc0,
-                                 0xf0,0x98,0x3c,0x34,0x0f,0x07,0x83,0x81,0xe1,
-                                 0x00,0x90,0x38,0x1e,0x0e,0x2c,0x8c,0x81,0xc2,
-                                 0xe0,0x78,0x00,0x1c,0x0f,0x08,0x06,0x81,0xc6,
-                                 0x90,0x80,0x68,0x24,0x1b,0x0b,0x26,0x83,0x21,
-                                 0x30,0xe0,0x98,0x3c,0x6f,0x06,0xb7,0x03,0x70};
-  unsigned long l = sizeof(bytes) / sizeof(char);
-  // Read 1 at a time, then 3 at a time, etc.
-  checkResults(values, decodeRLEv2(bytes, l, 1, values.size()), 1);
-  checkResults(values, decodeRLEv2(bytes, l, 3, values.size()), 3);
-  checkResults(values, decodeRLEv2(bytes, l, 7, values.size()), 7);
-  checkResults(values, decodeRLEv2(bytes, l, values.size(), values.size()),
-               values.size());
-};
-
-TEST(RLEv2, mixedPatchedAndShortRepeats) {
-  long v[] = {20, 2, 3, 2, 1, 3, 17, 71, 35, 2, 1, 139, 2, 2, 3, 1783, 475, 2, 1, 1, 3, 1, 3, 2, 32, 1, 2, 3, 1, 8, 30, 1, 3, 414, 1, 1, 135, 3, 3, 1, 414, 2, 1, 2, 2, 594, 2, 5, 6, 4, 11, 1, 2, 2, 1, 1, 52, 4, 1, 2, 7, 1, 17, 334, 1, 2, 1, 2, 2, 6, 1, 266, 1, 2, 217, 2, 6, 2, 13, 2, 2, 1, 2, 3, 5, 1, 2, 1, 7244, 11813, 1, 33, 2, -13, 1, 2, 3, 13, 1, 92, 3, 13, 5, 14, 9, 141, 12, 6, 15, 25, 1, 1, 1, 46, 2, 1, 1, 141, 3, 1, 1, 1, 1, 2, 1, 4, 34, 5, 78, 8, 1, 2, 2, 1, 9, 10, 2, 1, 4, 13, 1, 5, 4, 4, 19, 5, 1, 1, 1, 68, 33, 399, 1, 1885, 25, 5, 2, 4, 1, 1, 2, 16, 1, 2966, 3, 1, 1, 25501, 1, 1, 1, 66, 1, 3, 8, 131, 14, 5, 1, 2, 2, 1, 1, 8, 1, 1, 2, 1, 5, 9, 2, 3, 112, 13, 2, 2, 1, 5, 10, 3, 1, 1, 13, 2, 3, 4, 1, 3, 1, 1, 2, 1, 1, 2, 4, 2, 207, 1, 1, 2, 4, 3, 3, 2, 2, 16};
-  std::vector<int64_t> values;
-  for (size_t i = 0; i < sizeof(v) / sizeof(long); ++i) {
-    values.push_back(v[i]);
-  }
-
-  const unsigned char bytes[] = {0x90,0x6d,0x04,0xa4,0x8d,0x10,0x83,0xc2,0x00,0xf0,0x70,0x40,0x3c,0x54,0x18,0x03,0xc1,0xc9,0x80,0x78,0x3c,0x21,0x04,0xf4,0x03,0xc1,0xc0,0xe0,0x80,0x38,0x20,0x0f,0x16,0x83,0x81,0xe1,0x00,0x70,0x54,0x56,0x0e,0x08,0x6a,0xc1,0xc0,0xe4,0xa0,0x40,0x20,0x0e,0xd5,0x83,0xc1,0xc0,0xf0,0x79,0x7c,0x1e,0x12,0x09,0x84,0x43,0x00,0xe0,0x78,0x3c,0x1c,0x0e,0x20,0x84,0x41,0xc0,0xf0,0xa0,0x38,0x3d,0x5b,0x07,0x03,0xc1,0xc0,0xf0,0x78,0x4c,0x1d,0x17,0x07,0x03,0xdc,0xc0,0xf0,0x98,0x3c,0x34,0x0f,0x07,0x83,0x81,0xe1,0x00,0x90,0x38,0x1e,0x0e,0x2c,0x8c,0x81,0xc2,0xe0,0x78,0x00,0x1c,0x0f,0x08,0x06,0x81,0xc6,0x90,0x80,0x68,0x24,0x1b,0x0b,0x26,0x83,0x21,0x30,0xe0,0x98,0x3c,0x6f,0x06,0xb7,0x03,0x70,0x00,0x02,0x5e,0x05,0x00,0x5c,0x00,0x04,0x00,0x02,0x00,0x02,0x01,0x1a,0x00,0x06,0x01,0x02,0x8a,0x16,0x00,0x41,0x01,0x04,0x00,0xe1,0x10,0xd1,0xc0,0x04,0x10,0x08,0x24,0x10,0x03,0x30,0x01,0x03,0x0d,0x21,0x00,0xb0,0x00,0x02,0x5e,0x12,0x00,0x88,0x00,0x42,0x03,0x1e,0x00,0x02,0x0e,0xba,0x00,0x32
 ,0x00,0x0a,0x00,0x04,0x00,0x08,0x00,0x02,0x00,0x02,0x00,0x04,0x00,0x20,0x00,0x02,0x17,0x2c,0x00,0x06,0x00,0x02,0x00,0x02,0xc7,0x3a,0x00,0x02,0x8c,0x36,0x00,0xa2,0x01,0x82,0x00,0x10,0x70,0x43,0x42,0x00,0x02,0x04,0x00,0x00,0xe0,0x00,0x01,0x00,0x10,0x40,0x10,0x5b,0xc6,0x01,0x02,0x00,0x20,0x90,0x40,0x00,0x0c,0x02,0x08,0x18,0x00,0x40,0x00,0x01,0x00,0x00,0x08,0x30,0x33,0x80,0x00,0x02,0x0c,0x10,0x20,0x20,0x47,0x80,0x13,0x4c};
-  unsigned long l = sizeof(bytes) / sizeof(char);
-  // Read 1 at a time, then 3 at a time, etc.
-  checkResults(values, decodeRLEv2(bytes, l, 1, values.size()), 1);
-  checkResults(values, decodeRLEv2(bytes, l, 3, values.size()), 3);
-  checkResults(values, decodeRLEv2(bytes, l, 7, values.size()), 7);
-  checkResults(values, decodeRLEv2(bytes, l, values.size(), values.size()),
-               values.size());
-};
-
-TEST(RLEv2, basicDirectSeek) {
-  // 0,1 repeated 10 times (signed ints) followed by
-  // 0,2 repeated 10 times (signed ints)
-  const unsigned char bytes[] = {0x42,0x13,0x22,0x22,0x22,0x22,0x22,
-                                 0x46,0x13,0x04,0x04,0x04,0x04,0x04,
-                                 0x04,0x04,0x04,0x04,0x04};
-  unsigned long l = sizeof(bytes) / sizeof(char);
-
-  std::unique_ptr<RleDecoder> rle =
-      createRleDecoder(std::unique_ptr<SeekableInputStream>
-                       (new SeekableArrayInputStream(bytes,l)), true,
-                       RleVersion_2, *getDefaultPool());
-  std::list<uint64_t> position;
-  position.push_back(7); // byte position; skip first 20 [0 to 19]
-  position.push_back(13); // value position; skip 13 more [20 to 32]
-
-  PositionProvider location(position);
-  rle->seek(location);
-  std::vector<int64_t> data(3);
-  rle->next(data.data(), 3, nullptr);
-  EXPECT_EQ(2, data[0]);
-  EXPECT_EQ(0, data[1]);
-  EXPECT_EQ(2, data[2]);
-  rle->next(data.data(), 3, nullptr);
-  EXPECT_EQ(0, data[0]);
-  EXPECT_EQ(2, data[1]);
-  EXPECT_EQ(0, data[2]);
-  rle->next(data.data(), 1, nullptr);
-  EXPECT_EQ(2, data[0]);
-};
-
-TEST(RLEv2, bitsLeftByPreviousStream) {
-  // test for #109
-  // 118 DIRECT values, followed by PATHCED values
-  const unsigned char bytes[] = {
-    0x5a, 0x75, 0x92, 0x42, 0x49, 0x09, 0x2b, 0xa4,
-    0xae, 0x92, 0xc2, 0x4b, 0x89, 0x2f, 0x24, 0xbc,
-    0x93, 0x2a, 0x4c, 0xa9, 0x34, 0x24, 0xe0, 0x93,
-    0x92, 0x4e, 0xe9, 0x40, 0xa5, 0x04, 0x94, 0x12,
-    0x62, 0xa9, 0xc9, 0xa7, 0x26, 0x9c, 0xaa, 0x73,
-    0x09, 0xcd, 0x27, 0x34, 0x9c, 0xf2, 0x74, 0x49,
-    0xd3, 0x27, 0x50, 0x9d, 0x42, 0x75, 0x29, 0xd4,
-    0xa7, 0x5a, 0x9d, 0xaa, 0x79, 0x89, 0xe9, 0x27,
-    0xa4, 0x9e, 0xea, 0x7c, 0x29, 0xf6, 0x27, 0xdc,
-    0x9f, 0xb2, 0x7f, 0x4a, 0x00, 0xa8, 0x14, 0xa0,
-    0x72, 0x82, 0x8a, 0x19, 0x28, 0x6e, 0xa2, 0x52,
-    0x89, 0x4a, 0x28, 0x28, 0xa6, 0xa2, 0x9a, 0x8b,
-    0x6a, 0x2d, 0xa8, 0xb8, 0xa2, 0xe2, 0x8b, 0xaa,
-    0x53, 0xa9, 0x54, 0xa5, 0x92, 0x98, 0x6a, 0x62,
-    0xa9, 0x9c, 0xa6, 0x8a, 0x9b, 0xea, 0x70, 0x29,
-    0xd2, 0xa7, 0x52, 0x9d, 0x4a, 0x77, 0x29, 0xe0,
-    0xa7, 0xa2, 0x9e, 0xaa, 0x7b, 0x29, 0xf0, 0xa7,
-    0xd2, 0xa0, 0x0a, 0x84, 0x2a, 0x18, 0xa8, 0x72,
-    0xa1, 0xca, 0x89, 0x2a, 0x30, 0xa9, 0x4a, 0xa5,
-    0x4a, 0x96, 0x2a, 0xae, 0xab, 0x02, 0xac, 0x2b,
-    0x8d, 0x2e, 0x60, 0xb9, 0x82, 0xe7, 0x2b, 0x9f,
-    0xae, 0x84, 0xba, 0x52, 0xe9, 0xeb, 0xad, 0x2e,
-    0xb6, 0xbc, 0x32, 0xf1, 0xcb, 0xcc, 0x2f, 0x42,
-    0xbd, 0x8a, 0xf7, 0xcb, 0xe1, 0xaf, 0xa4, 0xbe,
-    0x9a, 0xfa, 0x6b, 0xeb, 0xaf, 0xba, 0xbe, 0xea,
-    0xfd, 0x2b, 0xf4, 0xaf, 0xd8, 0xbf, 0xfb, 0x00,
-    0x80, // <= end of DIRECT, start of PATCHED =>
-    0x90,0x6d,0x04,0xa4,0x8d,0x10,0x83,0xc2,0x00,
-    0xf0,0x70,0x40,0x3c,0x54,0x18,0x03,0xc1,0xc9,
-    0x80,0x78,0x3c,0x21,0x04,0xf4,0x03,0xc1,0xc0,
-    0xe0,0x80,0x38,0x20,0x0f,0x16,0x83,0x81,0xe1,
-    0x00,0x70,0x54,0x56,0x0e,0x08,0x6a,0xc1,0xc0,
-    0xe4,0xa0,0x40,0x20,0x0e,0xd5,0x83,0xc1,0xc0,
-    0xf0,0x79,0x7c,0x1e,0x12,0x09,0x84,0x43,0x00,
-    0xe0,0x78,0x3c,0x1c,0x0e,0x20,0x84,0x41,0xc0,
-    0xf0,0xa0,0x38,0x3d,0x5b,0x07,0x03,0xc1,0xc0,
-    0xf0,0x78,0x4c,0x1d,0x17,0x07,0x03,0xdc,0xc0,
-    0xf0,0x98,0x3c,0x34,0x0f,0x07,0x83,0x81,0xe1,
-    0x00,0x90,0x38,0x1e,0x0e,0x2c,0x8c,0x81,0xc2,
-    0xe0,0x78,0x00,0x1c,0x0f,0x08,0x06,0x81,0xc6,
-    0x90,0x80,0x68,0x24,0x1b,0x0b,0x26,0x83,0x21,
-    0x30,0xe0,0x98,0x3c,0x6f,0x06,0xb7,0x03,0x70
-  };
-  unsigned long l = sizeof(bytes) / sizeof(unsigned char);
-
-  // PATCHED values.
-  long v[] = {20, 2, 3, 2, 1, 3, 17, 71, 35, 2, 1, 139, 2, 2, 3, 1783, 475, 2,
-              1, 1, 3, 1, 3, 2, 32, 1, 2, 3, 1, 8, 30, 1, 3, 414, 1, 1, 135, 3,
-              3, 1, 414, 2, 1, 2, 2, 594, 2, 5, 6, 4, 11, 1, 2, 2, 1, 1, 52, 4,
-              1, 2, 7, 1, 17, 334, 1, 2, 1, 2, 2, 6, 1, 266, 1, 2, 217, 2, 6, 2,
-              13, 2, 2, 1, 2, 3, 5, 1, 2, 1, 7244, 11813, 1, 33, 2, -13, 1, 2, 3,
-              13, 1, 92, 3, 13, 5, 14, 9, 141, 12, 6, 15, 25};
-  unsigned long D = 118, P = sizeof(v) / sizeof(long), N = D + P;
-
-  std::unique_ptr<RleDecoder> rle =
-      createRleDecoder(std::unique_ptr<SeekableInputStream>
-                       (new SeekableArrayInputStream(bytes,l)), true,
-                       RleVersion_2, *getDefaultPool());
-
-  std::vector<int64_t> data(N);
-  rle->next(data.data(), N, nullptr);
-  // check patched values
-  for (size_t i=0; i < P; ++i) {
-      EXPECT_EQ(v[i], data[i+D]);
-  }
-};
-
-TEST(RLEv1, simpleTest) {
-  const unsigned char buffer[] = {0x61, 0xff, 0x64, 0xfb, 0x02, 0x03, 0x5, 0x7,
-				  0xb};
-  std::unique_ptr<RleDecoder> rle =
-      createRleDecoder(std::unique_ptr<SeekableInputStream>
-		       (new SeekableArrayInputStream(buffer,
-						     ARRAY_SIZE(buffer))),
-		       false, RleVersion_1, *getDefaultPool());
-  std::vector<int64_t> data(105);
-  rle->next(data.data(), 105, nullptr);
-
-  for (size_t i = 0; i < 100; ++i) {
-    EXPECT_EQ(100 - i, data[i]) << "Output wrong at " << i;
-  }
-  EXPECT_EQ(2, data[100]);
-  EXPECT_EQ(3, data[101]);
-  EXPECT_EQ(5, data[102]);
-  EXPECT_EQ(7, data[103]);
-  EXPECT_EQ(11, data[104]);
-};
-
-TEST(RLEv1, signedNullLiteralTest) {
-  const unsigned char buffer[]= {0xf8, 0x0, 0x1, 0x2, 0x3, 0x4, 0x5, 0x6, 0x7};
-  std::unique_ptr<RleDecoder> rle =
-      createRleDecoder(std::unique_ptr<SeekableInputStream>
-		       (new SeekableArrayInputStream(buffer,
-						     ARRAY_SIZE(buffer))),
-		       true, RleVersion_1, *getDefaultPool());
-  std::vector<int64_t> data(8);
-  std::vector<char> notNull(8, 1);
-  rle->next(data.data(), 8, notNull.data());
-
-  for (size_t i = 0; i < 8; ++i) {
-    EXPECT_EQ(i % 2 == 0 ? i/2 : -((i+1)/2),
-              data[i]);
-  }
-}
-
-TEST(RLEv1, splitHeader) {
-  const unsigned char buffer[] = {0x0, 0x00, 0xdc, 0xba, 0x98, 0x76};
-  std::unique_ptr<RleDecoder> rle =
-      createRleDecoder(
-          std::unique_ptr<SeekableInputStream>
-          (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer), 4)),
-          false, RleVersion_1, *getDefaultPool());
-  std::vector<int64_t> data(200);
-  rle->next(data.data(), 3, nullptr);
-
-  for (size_t i = 0; i < 3; ++i) {
-    EXPECT_EQ(247864668, data[i]) << "Output wrong at " << i;
-  }
-}
-
-TEST(RLEv1, splitRuns) {
-  const unsigned char buffer[] = {0x7d, 0x01, 0xff, 0x01, 0xfb, 0x01,
-				  0x02, 0x03, 0x04, 0x05};
-  SeekableInputStream* const stream =
-    new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer));
-  std::unique_ptr<RleDecoder> rle =
-      createRleDecoder(std::unique_ptr<SeekableInputStream>(stream),
-                       false, RleVersion_1, *getDefaultPool());
-  std::vector<int64_t> data(200);
-  for (size_t i = 0; i < 42; ++i) {
-    rle->next(data.data(), 3, nullptr);
-    for (size_t j = 0; j < 3; ++j) {
-      EXPECT_EQ(255 + i * 3 + j, data[j])
-      << "Wrong output at " << i << ", " << j;
-    }
-  }
-  rle->next(data.data(), 3, nullptr);
-  EXPECT_EQ(381, data[0]);
-  EXPECT_EQ(382, data[1]);
-  EXPECT_EQ(1, data[2]);
-  rle->next(data.data(), 3, nullptr);
-  EXPECT_EQ(2, data[0]);
-  EXPECT_EQ(3, data[1]);
-  EXPECT_EQ(4, data[2]);
-  rle->next(data.data(), 1, nullptr);
-  EXPECT_EQ(5, data[0]);
-}
-
-TEST(RLEv1, testSigned) {
-  const unsigned char buffer[] = {0x7f, 0xff, 0x20};
-  SeekableInputStream* const stream =
-    new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer));
-  std::unique_ptr<RleDecoder> rle =
-      createRleDecoder(std::unique_ptr<SeekableInputStream>(stream),
-                       true, RleVersion_1, *getDefaultPool());
-  std::vector<int64_t> data(100);
-  rle->next(data.data(), data.size(), nullptr);
-  for (size_t i = 0; i < data.size(); ++i) {
-    EXPECT_EQ(16 - i, data[i]) << "Wrong output at " << i;
-  }
-  rle->next(data.data(), 30, nullptr);
-  for(size_t i = 0; i < 30; ++i) {
-    EXPECT_EQ(16 - 100 - static_cast<long>(i), data[i])
-      << "Wrong output at " << (i + 100);
-  }
-}
-
-TEST(RLEv1, testNull) {
-  const unsigned char buffer[] = {0x75, 0x02, 0x00};
-  SeekableInputStream* const stream =
-    new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer));
-  std::unique_ptr<RleDecoder> rle =
-      createRleDecoder(std::unique_ptr<SeekableInputStream>(stream),
-                       true, RleVersion_1, *getDefaultPool());
-  std::vector<int64_t> data(24);
-  std::vector<char> notNull(24);
-  for (size_t i = 0; i < notNull.size(); ++i) {
-    notNull[i] = (i + 1) % 2;
-  }
-  for (size_t i = 0; i < 10; ++i) {
-    for(size_t j = 0; j < data.size(); ++j) {
-      data[j] = -1;
-    }
-    rle->next(data.data(), 24, notNull.data());
-    for (size_t j = 0; j < 24; ++j) {
-      if (notNull[j]) {
-        EXPECT_EQ(i * 24 + j, data[j]);
-      } else {
-        EXPECT_EQ(-1, data[j]);
-      }
-    }
-  }
-}
-
-TEST(RLEv1, testAllNulls) {
-  const unsigned char buffer[] = 
-    {0xf0, 0x00, 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07,
-     0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f,
-     0x3d, 0x00, 0x12};
-  SeekableInputStream* const stream =
-    new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer));
-  std::unique_ptr<RleDecoder> rle =
-      createRleDecoder(std::unique_ptr<SeekableInputStream>(stream),
-                       false, RleVersion_1, *getDefaultPool());
-  std::vector<int64_t> data(16, -1);
-  std::vector<char> allNull(16, 0);
-  std::vector<char> noNull(16, 1);
-  rle->next(data.data(), 16, allNull.data());
-  for (size_t i = 0; i < data.size(); ++i) {
-    EXPECT_EQ(-1, data[i]) << "Output wrong at " << i;
-  }
-  rle->next(data.data(), data.size(), noNull.data());
-  for (size_t i = 0; i < data.size(); ++i) {
-    EXPECT_EQ(i, data[i]) << "Output wrong at " << i;
-    data[i] = -1;
-  }
-  rle->next(data.data(), data.size(), allNull.data());
-  for (size_t i = 0; i < data.size(); ++i) {
-    EXPECT_EQ(-1, data[i]) << "Output wrong at " << i;
-  }
-  for (size_t i = 0; i < 4; ++i) {
-    rle->next(data.data(), data.size(), noNull.data());
-    for(size_t j = 0; j < data.size(); ++j) {
-      EXPECT_EQ(18, data[j]) << "Output wrong at " << i;
-    }
-  }
-  rle->next(data.data(), data.size(), allNull.data());
-}
-
-TEST(RLEv1, skipTest) {
-  // Create the RLE stream from Java's TestRunLengthIntegerEncoding.testSkips
-  // for (size_t i = 0; i < 1024; ++i)
-  //   out.write(i);
-  // for (size_t i = 1024; i < 2048; ++i)
-  //   out.write(i * 256);
-  // This causes the first half to be delta encoded and the second half to
-  // be literal encoded.
-  const unsigned char buffer[] = 
-{
-127,   1,   0, 127,   1, 132,   2, 127,   1, 136,   4, 127,   1, 140,   6, 127,
-  1, 144,   8, 127,   1, 148,  10, 127,   1, 152,  12, 111,   1, 156,  14, 128,
-128, 128,  32, 128, 132,  32, 128, 136,  32, 128, 140,  32, 128, 144,  32, 128,
-148,  32, 128, 152,  32, 128, 156,  32, 128, 160,  32, 128, 164,  32, 128, 168,
- 32, 128, 172,  32, 128, 176,  32, 128, 180,  32, 128, 184,  32, 128, 188,  32,
-128, 192,  32, 128, 196,  32, 128, 200,  32, 128, 204,  32, 128, 208,  32, 128,
-212,  32, 128, 216,  32, 128, 220,  32, 128, 224,  32, 128, 228,  32, 128, 232,
- 32, 128, 236,  32, 128, 240,  32, 128, 244,  32, 128, 248,  32, 128, 252,  32,
-128, 128,  33, 128, 132,  33, 128, 136,  33, 128, 140,  33, 128, 144,  33, 128,
-148,  33, 128, 152,  33, 128, 156,  33, 128, 160,  33, 128, 164,  33, 128, 168,
- 33, 128, 172,  33, 128, 176,  33, 128, 180,  33, 128, 184,  33, 128, 188,  33,
-128, 192,  33, 128, 196,  33, 128, 200,  33, 128, 204,  33, 128, 208,  33, 128,
-212,  33, 128, 216,  33, 128, 220,  33, 128, 224,  33, 128, 228,  33, 128, 232,
- 33, 128, 236,  33, 128, 240,  33, 128, 244,  33, 128, 248,  33, 128, 252,  33,
-128, 128,  34, 128, 132,  34, 128, 136,  34, 128, 140,  34, 128, 144,  34, 128,
-148,  34, 128, 152,  34, 128, 156,  34, 128, 160,  34, 128, 164,  34, 128, 168,
- 34, 128, 172,  34, 128, 176,  34, 128, 180,  34, 128, 184,  34, 128, 188,  34,
-128, 192,  34, 128, 196,  34, 128, 200,  34, 128, 204,  34, 128, 208,  34, 128,
-212,  34, 128, 216,  34, 128, 220,  34, 128, 224,  34, 128, 228,  34, 128, 232,
- 34, 128, 236,  34, 128, 240,  34, 128, 244,  34, 128, 248,  34, 128, 252,  34,
-128, 128,  35, 128, 132,  35, 128, 136,  35, 128, 140,  35, 128, 144,  35, 128,
-148,  35, 128, 152,  35, 128, 156,  35, 128, 160,  35, 128, 164,  35, 128, 168,
- 35, 128, 172,  35, 128, 176,  35, 128, 180,  35, 128, 184,  35, 128, 188,  35,
-128, 192,  35, 128, 196,  35, 128, 200,  35, 128, 204,  35, 128, 208,  35, 128,
-212,  35, 128, 216,  35, 128, 220,  35, 128, 224,  35, 128, 228,  35, 128, 232,
- 35, 128, 236,  35, 128, 240,  35, 128, 244,  35, 128, 248,  35, 128, 252,  35,
-128, 128, 128,  36, 128, 132,  36, 128, 136,  36, 128, 140,  36, 128, 144,  36,
-128, 148,  36, 128, 152,  36, 128, 156,  36, 128, 160,  36, 128, 164,  36, 128,
-168,  36, 128, 172,  36, 128, 176,  36, 128, 180,  36, 128, 184,  36, 128, 188,
- 36, 128, 192,  36, 128, 196,  36, 128, 200,  36, 128, 204,  36, 128, 208,  36,
-128, 212,  36, 128, 216,  36, 128, 220,  36, 128, 224,  36, 128, 228,  36, 128,
-232,  36, 128, 236,  36, 128, 240,  36, 128, 244,  36, 128, 248,  36, 128, 252,
- 36, 128, 128,  37, 128, 132,  37, 128, 136,  37, 128, 140,  37, 128, 144,  37,
-128, 148,  37, 128, 152,  37, 128, 156,  37, 128, 160,  37, 128, 164,  37, 128,
-168,  37, 128, 172,  37, 128, 176,  37, 128, 180,  37, 128, 184,  37, 128, 188,
- 37, 128, 192,  37, 128, 196,  37, 128, 200,  37, 128, 204,  37, 128, 208,  37,
-128, 212,  37, 128, 216,  37, 128, 220,  37, 128, 224,  37, 128, 228,  37, 128,
-232,  37, 128, 236,  37, 128, 240,  37, 128, 244,  37, 128, 248,  37, 128, 252,
- 37, 128, 128,  38, 128, 132,  38, 128, 136,  38, 128, 140,  38, 128, 144,  38,
-128, 148,  38, 128, 152,  38, 128, 156,  38, 128, 160,  38, 128, 164,  38, 128,
-168,  38, 128, 172,  38, 128, 176,  38, 128, 180,  38, 128, 184,  38, 128, 188,
- 38, 128, 192,  38, 128, 196,  38, 128, 200,  38, 128, 204,  38, 128, 208,  38,
-128, 212,  38, 128, 216,  38, 128, 220,  38, 128, 224,  38, 128, 228,  38, 128,
-232,  38, 128, 236,  38, 128, 240,  38, 128, 244,  38, 128, 248,  38, 128, 252,
- 38, 128, 128,  39, 128, 132,  39, 128, 136,  39, 128, 140,  39, 128, 144,  39,
-128, 148,  39, 128, 152,  39, 128, 156,  39, 128, 160,  39, 128, 164,  39, 128,
-168,  39, 128, 172,  39, 128, 176,  39, 128, 180,  39, 128, 184,  39, 128, 188,
- 39, 128, 192,  39, 128, 196,  39, 128, 200,  39, 128, 204,  39, 128, 208,  39,
-128, 212,  39, 128, 216,  39, 128, 220,  39, 128, 224,  39, 128, 228,  39, 128,
-232,  39, 128, 236,  39, 128, 240,  39, 128, 244,  39, 128, 248,  39, 128, 252,
- 39, 128, 128, 128,  40, 128, 132,  40, 128, 136,  40, 128, 140,  40, 128, 144,
- 40, 128, 148,  40, 128, 152,  40, 128, 156,  40, 128, 160,  40, 128, 164,  40,
-128, 168,  40, 128, 172,  40, 128, 176,  40, 128, 180,  40, 128, 184,  40, 128,
-188,  40, 128, 192,  40, 128, 196,  40, 128, 200,  40, 128, 204,  40, 128, 208,
- 40, 128, 212,  40, 128, 216,  40, 128, 220,  40, 128, 224,  40, 128, 228,  40,
-128, 232,  40, 128, 236,  40, 128, 240,  40, 128, 244,  40, 128, 248,  40, 128,
-252,  40, 128, 128,  41, 128, 132,  41, 128, 136,  41, 128, 140,  41, 128, 144,
- 41, 128, 148,  41, 128, 152,  41, 128, 156,  41, 128, 160,  41, 128, 164,  41,
-128, 168,  41, 128, 172,  41, 128, 176,  41, 128, 180,  41, 128, 184,  41, 128,
-188,  41, 128, 192,  41, 128, 196,  41, 128, 200,  41, 128, 204,  41, 128, 208,
- 41, 128, 212,  41, 128, 216,  41, 128, 220,  41, 128, 224,  41, 128, 228,  41,
-128, 232,  41, 128, 236,  41, 128, 240,  41, 128, 244,  41, 128, 248,  41, 128,
-252,  41, 128, 128,  42, 128, 132,  42, 128, 136,  42, 128, 140,  42, 128, 144,
- 42, 128, 148,  42, 128, 152,  42, 128, 156,  42, 128, 160,  42, 128, 164,  42,
-128, 168,  42, 128, 172,  42, 128, 176,  42, 128, 180,  42, 128, 184,  42, 128,
-188,  42, 128, 192,  42, 128, 196,  42, 128, 200,  42, 128, 204,  42, 128, 208,
- 42, 128, 212,  42, 128, 216,  42, 128, 220,  42, 128, 224,  42, 128, 228,  42,
-128, 232,  42, 128, 236,  42, 128, 240,  42, 128, 244,  42, 128, 248,  42, 128,
-252,  42, 128, 128,  43, 128, 132,  43, 128, 136,  43, 128, 140,  43, 128, 144,
- 43, 128, 148,  43, 128, 152,  43, 128, 156,  43, 128, 160,  43, 128, 164,  43,
-128, 168,  43, 128, 172,  43, 128, 176,  43, 128, 180,  43, 128, 184,  43, 128,
-188,  43, 128, 192,  43, 128, 196,  43, 128, 200,  43, 128, 204,  43, 128, 208,
- 43, 128, 212,  43, 128, 216,  43, 128, 220,  43, 128, 224,  43, 128, 228,  43,
-128, 232,  43, 128, 236,  43, 128, 240,  43, 128, 244,  43, 128, 248,  43, 128,
-252,  43, 128, 128, 128,  44, 128, 132,  44, 128, 136,  44, 128, 140,  44, 128,
-144,  44, 128, 148,  44, 128, 152,  44, 128, 156,  44, 128, 160,  44, 128, 164,
- 44, 128, 168,  44, 128, 172,  44, 128, 176,  44, 128, 180,  44, 128, 184,  44,
-128, 188,  44, 128, 192,  44, 128, 196,  44, 128, 200,  44, 128, 204,  44, 128,
-208,  44, 128, 212,  44, 128, 216,  44, 128, 220,  44, 128, 224,  44, 128, 228,
- 44, 128, 232,  44, 128, 236,  44, 128, 240,  44, 128, 244,  44, 128, 248,  44,
-128, 252,  44, 128, 128,  45, 128, 132,  45, 128, 136,  45, 128, 140,  45, 128,
-144,  45, 128, 148,  45, 128, 152,  45, 128, 156,  45, 128, 160,  45, 128, 164,
- 45, 128, 168,  45, 128, 172,  45, 128, 176,  45, 128, 180,  45, 128, 184,  45,
-128, 188,  45, 128, 192,  45, 128, 196,  45, 128, 200,  45, 128, 204,  45, 128,
-208,  45, 128, 212,  45, 128, 216,  45, 128, 220,  45, 128, 224,  45, 128, 228,
- 45, 128, 232,  45, 128, 236,  45, 128, 240,  45, 128, 244,  45, 128, 248,  45,
-128, 252,  45, 128, 128,  46, 128, 132,  46, 128, 136,  46, 128, 140,  46, 128,
-144,  46, 128, 148,  46, 128, 152,  46, 128, 156,  46, 128, 160,  46, 128, 164,
- 46, 128, 168,  46, 128, 172,  46, 128, 176,  46, 128, 180,  46, 128, 184,  46,
-128, 188,  46, 128, 192,  46, 128, 196,  46, 128, 200,  46, 128, 204,  46, 128,
-208,  46, 128, 212,  46, 128, 216,  46, 128, 220,  46, 128, 224,  46, 128, 228,
- 46, 128, 232,  46, 128, 236,  46, 128, 240,  46, 128, 244,  46, 128, 248,  46,
-128, 252,  46, 128, 128,  47, 128, 132,  47, 128, 136,  47, 128, 140,  47, 128,
-144,  47, 128, 148,  47, 128, 152,  47, 128, 156,  47, 128, 160,  47, 128, 164,
- 47, 128, 168,  47, 128, 172,  47, 128, 176,  47, 128, 180,  47, 128, 184,  47,
-128, 188,  47, 128, 192,  47, 128, 196,  47, 128, 200,  47, 128, 204,  47, 128,
-208,  47, 128, 212,  47, 128, 216,  47, 128, 220,  47, 128, 224,  47, 128, 228,
- 47, 128, 232,  47, 128, 236,  47, 128, 240,  47, 128, 244,  47, 128, 248,  47,
-128, 252,  47, 128, 128, 128,  48, 128, 132,  48, 128, 136,  48, 128, 140,  48,
-128, 144,  48, 128, 148,  48, 128, 152,  48, 128, 156,  48, 128, 160,  48, 128,
-164,  48, 128, 168,  48, 128, 172,  48, 128, 176,  48, 128, 180,  48, 128, 184,
- 48, 128, 188,  48, 128, 192,  48, 128, 196,  48, 128, 200,  48, 128, 204,  48,
-128, 208,  48, 128, 212,  48, 128, 216,  48, 128, 220,  48, 128, 224,  48, 128,
-228,  48, 128, 232,  48, 128, 236,  48, 128, 240,  48, 128, 244,  48, 128, 248,
- 48, 128, 252,  48, 128, 128,  49, 128, 132,  49, 128, 136,  49, 128, 140,  49,
-128, 144,  49, 128, 148,  49, 128, 152,  49, 128, 156,  49, 128, 160,  49, 128,
-164,  49, 128, 168,  49, 128, 172,  49, 128, 176,  49, 128, 180,  49, 128, 184,
- 49, 128, 188,  49, 128, 192,  49, 128, 196,  49, 128, 200,  49, 128, 204,  49,
-128, 208,  49, 128, 212,  49, 128, 216,  49, 128, 220,  49, 128, 224,  49, 128,
-228,  49, 128, 232,  49, 128, 236,  49, 128, 240,  49, 128, 244,  49, 128, 248,
- 49, 128, 252,  49, 128, 128,  50, 128, 132,  50, 128, 136,  50, 128, 140,  50,
-128, 144,  50, 128, 148,  50, 128, 152,  50, 128, 156,  50, 128, 160,  50, 128,
-164,  50, 128, 168,  50, 128, 172,  50, 128, 176,  50, 128, 180,  50, 128, 184,
- 50, 128, 188,  50, 128, 192,  50, 128, 196,  50, 128, 200,  50, 128, 204,  50,
-128, 208,  50, 128, 212,  50, 128, 216,  50, 128, 220,  50, 128, 224,  50, 128,
-228,  50, 128, 232,  50, 128, 236,  50, 128, 240,  50, 128, 244,  50, 128, 248,
- 50, 128, 252,  50, 128, 128,  51, 128, 132,  51, 128, 136,  51, 128, 140,  51,
-128, 144,  51, 128, 148,  51, 128, 152,  51, 128, 156,  51, 128, 160,  51, 128,
-164,  51, 128, 168,  51, 128, 172,  51, 128, 176,  51, 128, 180,  51, 128, 184,
- 51, 128, 188,  51, 128, 192,  51, 128, 196,  51, 128, 200,  51, 128, 204,  51,
-128, 208,  51, 128, 212,  51, 128, 216,  51, 128, 220,  51, 128, 224,  51, 128,
-228,  51, 128, 232,  51, 128, 236,  51, 128, 240,  51, 128, 244,  51, 128, 248,
- 51, 128, 252,  51, 128, 128, 128,  52, 128, 132,  52, 128, 136,  52, 128, 140,
- 52, 128, 144,  52, 128, 148,  52, 128, 152,  52, 128, 156,  52, 128, 160,  52,
-128, 164,  52, 128, 168,  52, 128, 172,  52, 128, 176,  52, 128, 180,  52, 128,
-184,  52, 128, 188,  52, 128, 192,  52, 128, 196,  52, 128, 200,  52, 128, 204,
- 52, 128, 208,  52, 128, 212,  52, 128, 216,  52, 128, 220,  52, 128, 224,  52,
-128, 228,  52, 128, 232,  52, 128, 236,  52, 128, 240,  52, 128, 244,  52, 128,
-248,  52, 128, 252,  52, 128, 128,  53, 128, 132,  53, 128, 136,  53, 128, 140,
- 53, 128, 144,  53, 128, 148,  53, 128, 152,  53, 128, 156,  53, 128, 160,  53,
-128, 164,  53, 128, 168,  53, 128, 172,  53, 128, 176,  53, 128, 180,  53, 128,
-184,  53, 128, 188,  53, 128, 192,  53, 128, 196,  53, 128, 200,  53, 128, 204,
- 53, 128, 208,  53, 128, 212,  53, 128, 216,  53, 128, 220,  53, 128, 224,  53,
-128, 228,  53, 128, 232,  53, 128, 236,  53, 128, 240,  53, 128, 244,  53, 128,
-248,  53, 128, 252,  53, 128, 128,  54, 128, 132,  54, 128, 136,  54, 128, 140,
- 54, 128, 144,  54, 128, 148,  54, 128, 152,  54, 128, 156,  54, 128, 160,  54,
-128, 164,  54, 128, 168,  54, 128, 172,  54, 128, 176,  54, 128, 180,  54, 128,
-184,  54, 128, 188,  54, 128, 192,  54, 128, 196,  54, 128, 200,  54, 128, 204,
- 54, 128, 208,  54, 128, 212,  54, 128, 216,  54, 128, 220,  54, 128, 224,  54,
-128, 228,  54, 128, 232,  54, 128, 236,  54, 128, 240,  54, 128, 244,  54, 128,
-248,  54, 128, 252,  54, 128, 128,  55, 128, 132,  55, 128, 136,  55, 128, 140,
- 55, 128, 144,  55, 128, 148,  55, 128, 152,  55, 128, 156,  55, 128, 160,  55,
-128, 164,  55, 128, 168,  55, 128, 172,  55, 128, 176,  55, 128, 180,  55, 128,
-184,  55, 128, 188,  55, 128, 192,  55, 128, 196,  55, 128, 200,  55, 128, 204,
- 55, 128, 208,  55, 128, 212,  55, 128, 216,  55, 128, 220,  55, 128, 224,  55,
-128, 228,  55, 128, 232,  55, 128, 236,  55, 128, 240,  55, 128, 244,  55, 128,
-248,  55, 128, 252,  55, 128, 128, 128,  56, 128, 132,  56, 128, 136,  56, 128,
-140,  56, 128, 144,  56, 128, 148,  56, 128, 152,  56, 128, 156,  56, 128, 160,
- 56, 128, 164,  56, 128, 168,  56, 128, 172,  56, 128, 176,  56, 128, 180,  56,
-128, 184,  56, 128, 188,  56, 128, 192,  56, 128, 196,  56, 128, 200,  56, 128,
-204,  56, 128, 208,  56, 128, 212,  56, 128, 216,  56, 128, 220,  56, 128, 224,
- 56, 128, 228,  56, 128, 232,  56, 128, 236,  56, 128, 240,  56, 128, 244,  56,
-128, 248,  56, 128, 252,  56, 128, 128,  57, 128, 132,  57, 128, 136,  57, 128,
-140,  57, 128, 144,  57, 128, 148,  57, 128, 152,  57, 128, 156,  57, 128, 160,
- 57, 128, 164,  57, 128, 168,  57, 128, 172,  57, 128, 176,  57, 128, 180,  57,
-128, 184,  57, 128, 188,  57, 128, 192,  57, 128, 196,  57, 128, 200,  57, 128,
-204,  57, 128, 208,  57, 128, 212,  57, 128, 216,  57, 128, 220,  57, 128, 224,
- 57, 128, 228,  57, 128, 232,  57, 128, 236,  57, 128, 240,  57, 128, 244,  57,
-128, 248,  57, 128, 252,  57, 128, 128,  58, 128, 132,  58, 128, 136,  58, 128,
-140,  58, 128, 144,  58, 128, 148,  58, 128, 152,  58, 128, 156,  58, 128, 160,
- 58, 128, 164,  58, 128, 168,  58, 128, 172,  58, 128, 176,  58, 128, 180,  58,
-128, 184,  58, 128, 188,  58, 128, 192,  58, 128, 196,  58, 128, 200,  58, 128,
-204,  58, 128, 208,  58, 128, 212,  58, 128, 216,  58, 128, 220,  58, 128, 224,
- 58, 128, 228,  58, 128, 232,  58, 128, 236,  58, 128, 240,  58, 128, 244,  58,
-128, 248,  58, 128, 252,  58, 128, 128,  59, 128, 132,  59, 128, 136,  59, 128,
-140,  59, 128, 144,  59, 128, 148,  59, 128, 152,  59, 128, 156,  59, 128, 160,
- 59, 128, 164,  59, 128, 168,  59, 128, 172,  59, 128, 176,  59, 128, 180,  59,
-128, 184,  59, 128, 188,  59, 128, 192,  59, 128, 196,  59, 128, 200,  59, 128,
-204,  59, 128, 208,  59, 128, 212,  59, 128, 216,  59, 128, 220,  59, 128, 224,
- 59, 128, 228,  59, 128, 232,  59, 128, 236,  59, 128, 240,  59, 128, 244,  59,
-128, 248,  59, 128, 252,  59, 128, 128, 128,  60, 128, 132,  60, 128, 136,  60,
-128, 140,  60, 128, 144,  60, 128, 148,  60, 128, 152,  60, 128, 156,  60, 128,
-160,  60, 128, 164,  60, 128, 168,  60, 128, 172,  60, 128, 176,  60, 128, 180,
- 60, 128, 184,  60, 128, 188,  60, 128, 192,  60, 128, 196,  60, 128, 200,  60,
-128, 204,  60, 128, 208,  60, 128, 212,  60, 128, 216,  60, 128, 220,  60, 128,
-224,  60, 128, 228,  60, 128, 232,  60, 128, 236,  60, 128, 240,  60, 128, 244,
- 60, 128, 248,  60, 128, 252,  60, 128, 128,  61, 128, 132,  61, 128, 136,  61,
-128, 140,  61, 128, 144,  61, 128, 148,  61, 128, 152,  61, 128, 156,  61, 128,
-160,  61, 128, 164,  61, 128, 168,  61, 128, 172,  61, 128, 176,  61, 128, 180,
- 61, 128, 184,  61, 128, 188,  61, 128, 192,  61, 128, 196,  61, 128, 200,  61,
-128, 204,  61, 128, 208,  61, 128, 212,  61, 128, 216,  61, 128, 220,  61, 128,
-224,  61, 128, 228,  61, 128, 232,  61, 128, 236,  61, 128, 240,  61, 128, 244,
- 61, 128, 248,  61, 128, 252,  61, 128, 128,  62, 128, 132,  62, 128, 136,  62,
-128, 140,  62, 128, 144,  62, 128, 148,  62, 128, 152,  62, 128, 156,  62, 128,
-160,  62, 128, 164,  62, 128, 168,  62, 128, 172,  62, 128, 176,  62, 128, 180,
- 62, 128, 184,  62, 128, 188,  62, 128, 192,  62, 128, 196,  62, 128, 200,  62,
-128, 204,  62, 128, 208,  62, 128, 212,  62, 128, 216,  62, 128, 220,  62, 128,
-224,  62, 128, 228,  62, 128, 232,  62, 128, 236,  62, 128, 240,  62, 128, 244,
- 62, 128, 248,  62, 128, 252,  62, 128, 128,  63, 128, 132,  63, 128, 136,  63,
-128, 140,  63, 128, 144,  63, 128, 148,  63, 128, 152,  63, 128, 156,  63, 128,
-160,  63, 128, 164,  63, 128, 168,  63, 128, 172,  63, 128, 176,  63, 128, 180,
- 63, 128, 184,  63, 128, 188,  63, 128, 192,  63, 128, 196,  63, 128, 200,  63,
-128, 204,  63, 128, 208,  63, 128, 212,  63, 128, 216,  63, 128, 220,  63, 128,
-224,  63, 128, 228,  63, 128, 232,  63, 128, 236,  63, 128, 240,  63, 128, 244,
-63, 128, 248,  63, 128, 252,  63};
-  SeekableInputStream* const stream =
-    new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer));
-  std::unique_ptr<RleDecoder> rle =
-      createRleDecoder(std::unique_ptr<SeekableInputStream>(stream),
-                       true, RleVersion_1, *getDefaultPool());
-  std::vector<int64_t> data(1);
-  for (size_t i = 0; i < 2048; i += 10) {
-    rle->next(data.data(), 1, nullptr);
-    if (i < 1024) {
-      EXPECT_EQ(i, data[0]) << "Wrong output at " << i;
-    } else {
-      EXPECT_EQ(256 * i, data[0]) << "Wrong output at " << i;
-    }
-    if (i < 2038) {
-      rle->skip(9);
-    }
-    rle->skip(0);
-  }
-}
-
-TEST(RLEv1, seekTest) {
-  // Create the RLE stream from Java's
-  // TestRunLengthIntegerEncoding.testUncompressedSeek
-  // for (size_t i = 0; i < 1024; ++i)
-  //   out.write(i / 4);
-  // for (size_t i = 1024; i < 2048; ++i)
-  //   out.write(2 * i);
-  // for (size_t i = 0; i < 2048; ++i)
-  //   out.write(junk[i]);
-  // This causes the first half to be delta encoded and the second half to
-  // be literal encoded.
-  const unsigned char buffer[] = {
-  1,   0,   0,   1,   0,   2,   1,   0,   4,   1,   0,   6,   1,   0,   8,   1,
-  0,  10,   1,   0,  12,   1,   0,  14,   1,   0,  16,   1,   0,  18,   1,   0,
- 20,   1,   0,  22,   1,   0,  24,   1,   0,  26,   1,   0,  28,   1,   0,  30,
-  1,   0,  32,   1,   0,  34,   1,   0,  36,   1,   0,  38,   1,   0,  40,   1,
-  0,  42,   1,   0,  44,   1,   0,  46,   1,   0,  48,   1,   0,  50,   1,   0,
- 52,   1,   0,  54,   1,   0,  56,   1,   0,  58,   1,   0,  60,   1,   0,  62,
-  1,   0,  64,   1,   0,  66,   1,   0,  68,   1,   0,  70,   1,   0,  72,   1,
-  0,  74,   1,   0,  76,   1,   0,  78,   1,   0,  80,   1,   0,  82,   1,   0,
- 84,   1,   0,  86,   1,   0,  88,   1,   0,  90,   1,   0,  92,   1,   0,  94,
-  1,   0,  96,   1,   0,  98,   1,   0, 100,   1,   0, 102,   1,   0, 104,   1,
-  0, 106,   1,   0, 108,   1,   0, 110,   1,   0, 112,   1,   0, 114,   1,   0,
-116,   1,   0, 118,   1,   0, 120,   1,   0, 122,   1,   0, 124,   1,   0, 126,
-  1,   0, 128,   1,   1,   0, 130,   1,   1,   0, 132,   1,   1,   0, 134,   1,
-  1,   0, 136,   1,   1,   0, 138,   1,   1,   0, 140,   1,   1,   0, 142,   1,
-  1,   0, 144,   1,   1,   0, 146,   1,   1,   0, 148,   1,   1,   0, 150,   1,
-  1,   0, 152,   1,   1,   0, 154,   1,   1,   0, 156,   1,   1,   0, 158,   1,
-  1,   0, 160,   1,   1,   0, 162,   1,   1,   0, 164,   1,   1,   0, 166,   1,
-  1,   0, 168,   1,   1,   0, 170,   1,   1,   0, 172,   1,   1,   0, 174,   1,
-  1,   0, 176,   1,   1,   0, 178,   1,   1,   0, 180,   1,   1,   0, 182,   1,
-  1,   0, 184,   1,   1,   0, 186,   1,   1,   0, 188,   1,   1,   0, 190,   1,
-  1,   0, 192,   1,   1,   0, 194,   1,   1,   0, 196,   1,   1,   0, 198,   1,
-  1,   0, 200,   1,   1,   0, 202,   1,   1,   0, 204,   1,   1,   0, 206,   1,
-  1,   0, 208,   1,   1,   0, 210,   1,   1,   0, 212,   1,   1,   0, 214,   1,
-  1,   0, 216,   1,   1,   0, 218,   1,   1,   0, 220,   1,   1,   0, 222,   1,
-  1,   0, 224,   1,   1,   0, 226,   1,   1,   0, 228,   1,   1,   0, 230,   1,
-  1,   0, 232,   1,   1,   0, 234,   1,   1,   0, 236,   1,   1,   0, 238,   1,
-  1,   0, 240,   1,   1,   0, 242,   1,   1,   0, 244,   1,   1,   0, 246,   1,
-  1,   0, 248,   1,   1,   0, 250,   1,   1,   0, 252,   1,   1,   0, 254,   1,
-  1,   0, 128,   2,   1,   0, 130,   2,   1,   0, 132,   2,   1,   0, 134,   2,
-  1,   0, 136,   2,   1,   0, 138,   2,   1,   0, 140,   2,   1,   0, 142,   2,
-  1,   0, 144,   2,   1,   0, 146,   2,   1,   0, 148,   2,   1,   0, 150,   2,
-  1,   0, 152,   2,   1,   0, 154,   2,   1,   0, 156,   2,   1,   0, 158,   2,
-  1,   0, 160,   2,   1,   0, 162,   2,   1,   0, 164,   2,   1,   0, 166,   2,
-  1,   0, 168,   2,   1,   0, 170,   2,   1,   0, 172,   2,   1,   0, 174,   2,
-  1,   0, 176,   2,   1,   0, 178,   2,   1,   0, 180,   2,   1,   0, 182,   2,
-  1,   0, 184,   2,   1,   0, 186,   2,   1,   0, 188,   2,   1,   0, 190,   2,
-  1,   0, 192,   2,   1,   0, 194,   2,   1,   0, 196,   2,   1,   0, 198,   2,
-  1,   0, 200,   2,   1,   0, 202,   2,   1,   0, 204,   2,   1,   0, 206,   2,
-  1,   0, 208,   2,   1,   0, 210,   2,   1,   0, 212,   2,   1,   0, 214,   2,
-  1,   0, 216,   2,   1,   0, 218,   2,   1,   0, 220,   2,   1,   0, 222,   2,
-  1,   0, 224,   2,   1,   0, 226,   2,   1,   0, 228,   2,   1,   0, 230,   2,
-  1,   0, 232,   2,   1,   0, 234,   2,   1,   0, 236,   2,   1,   0, 238,   2,
-  1,   0, 240,   2,   1,   0, 242,   2,   1,   0, 244,   2,   1,   0, 246,   2,
-  1,   0, 248,   2,   1,   0, 250,   2,   1,   0, 252,   2,   1,   0, 254,   2,
-  1,   0, 128,   3,   1,   0, 130,   3,   1,   0, 132,   3,   1,   0, 134,   3,
-  1,   0, 136,   3,   1,   0, 138,   3,   1,   0, 140,   3,   1,   0, 142,   3,
-  1,   0, 144,   3,   1,   0, 146,   3,   1,   0, 148,   3,   1,   0, 150,   3,
-  1,   0, 152,   3,   1,   0, 154,   3,   1,   0, 156,   3,   1,   0, 158,   3,
-  1,   0, 160,   3,   1,   0, 162,   3,   1,   0, 164,   3,   1,   0, 166,   3,
-  1,   0, 168,   3,   1,   0, 170,   3,   1,   0, 172,   3,   1,   0, 174,   3,
-  1,   0, 176,   3,   1,   0, 178,   3,   1,   0, 180,   3,   1,   0, 182,   3,
-  1,   0, 184,   3,   1,   0, 186,   3,   1,   0, 188,   3,   1,   0, 190,   3,
-  1,   0, 192,   3,   1,   0, 194,   3,   1,   0, 196,   3,   1,   0, 198,   3,
-  1,   0, 200,   3,   1,   0, 202,   3,   1,   0, 204,   3,   1,   0, 206,   3,
-  1,   0, 208,   3,   1,   0, 210,   3,   1,   0, 212,   3,   1,   0, 214,   3,
-  1,   0, 216,   3,   1,   0, 218,   3,   1,   0, 220,   3,   1,   0, 222,   3,
-  1,   0, 224,   3,   1,   0, 226,   3,   1,   0, 228,   3,   1,   0, 230,   3,
-  1,   0, 232,   3,   1,   0, 234,   3,   1,   0, 236,   3,   1,   0, 238,   3,
-  1,   0, 240,   3,   1,   0, 242,   3,   1,   0, 244,   3,   1,   0, 246,   3,
-  1,   0, 248,   3,   1,   0, 250,   3,   1,   0, 252,   3,   1,   0, 254,   3,
-127,   2, 128,  32, 127,   2, 136,  36, 127,   2, 144,  40, 127,   2, 152,  44,
-127,   2, 160,  48, 127,   2, 168,  52, 127,   2, 176,  56, 111,   2, 184,  60,
-128, 147, 150, 232, 240,   8, 168, 134, 179, 187,  12, 246, 145, 173, 142,  11,
-241, 162, 190, 162,   9, 239, 218, 128, 243,   5, 202, 175, 131, 196,  12, 151,
-253, 204, 160,   4, 229, 167, 247, 255,  12, 255, 177, 140, 184,   7, 188, 145,
-181, 229,   1, 178, 190, 158, 163,   8, 147, 179, 151, 132,   8, 150, 133, 222,
-129,  11, 193, 218, 187, 242,  14, 181, 177, 154, 155,   9, 150, 145, 194, 135,
-  8, 186, 222, 142, 242,  10, 140, 195, 254, 237,  11, 141, 189, 143, 198,  14,
-229, 146, 237, 203,   8, 251, 162, 179, 211,   3, 222, 237, 175, 145,  13, 221,
-178, 163, 162,   3, 211, 192, 165, 189,  14, 230, 228, 168, 250,   4, 141, 140,
-247, 178,   7, 143, 164, 170, 152,   2, 131, 166, 136,  26, 171, 143, 232, 134,
- 12, 158, 239, 246, 204,  11, 133, 128, 213, 223,  14, 255, 213, 190, 250,  15,
-143, 162, 252, 157,   4, 204, 181, 135, 245,   7, 206, 241, 254, 136,   4, 184,
-182, 211, 190,  15, 172, 156, 202, 135,  10, 249, 180, 139, 131,   4, 202, 128,
-204, 221,   9, 131, 247, 166, 249,   8, 141, 236, 241, 185,   3, 128, 229, 150,
-186,   2, 237, 189, 141, 218,   9, 193, 240, 241, 156,   3, 210, 142, 198, 202,
- 10, 227, 241, 194, 234,   7, 145, 180, 228, 254,   6, 171, 249, 185, 188,  11,
-215, 135, 224, 219,   4, 133, 132, 178, 165,   7, 205, 180, 133, 209,  11, 198,
-253, 246, 145,  12, 190, 194, 153, 146,   8, 139, 220, 235, 249,   1, 170, 203,
-205, 159,   6, 136, 130, 154, 166,  14, 250, 189, 153, 191,   7, 178, 163, 191,
-158,  12, 251, 138, 135, 245,  10, 175, 249, 219, 164,  14, 136, 185, 220, 188,
-  7, 170, 135, 221, 146,   7, 209, 224, 204, 171,  11, 216, 144, 236, 172,   1,
-133, 205, 202, 170,   6, 215, 250, 133, 181,   3, 181, 133, 142, 158,   5, 166,
-192, 134, 238,  13, 246, 243, 233, 218,  12, 163, 202, 238, 241,  14, 241, 214,
-224, 215,   2, 212, 192, 237, 243,  10, 163, 165, 163, 206,   6, 159, 161, 227,
-152,  14, 209, 234, 225, 249,  13, 167, 206, 188, 161,   3, 143, 209, 188, 214,
- 11, 184, 224, 210, 200,  10, 185, 171, 199, 183,   3, 177, 229, 245,  86, 255,
-183, 178, 142,   9, 232, 209, 135, 151,   8, 191, 153, 174, 175,   7, 190, 245,
-224, 174,   9, 243, 165, 145, 169,   1, 145, 161, 221, 249,  13, 195, 221, 244,
-240,   5, 157, 156, 217, 237,  15, 143, 201, 155, 207,   5, 169, 136, 192, 238,
- 12, 135, 223, 244, 200,   2, 137, 228, 167, 187,   1, 134, 212, 158, 155,  15,
-186, 224, 212, 214,   7, 193, 141, 216, 241,   2, 246, 159, 138, 117, 216, 230,
-215,  29, 204, 178, 147, 255,   8, 195, 140, 136, 164,  11, 234, 204, 155, 222,
- 10, 193, 156, 138, 187,   8, 161, 161, 184, 212,   1, 128, 141, 162, 133,  13,
-180, 211, 132, 210,   9, 239, 203, 201, 177,   5, 236, 191, 140, 207,  13, 173,
-205, 192, 186,   7, 179, 214, 222, 136,   8, 189, 142, 204, 152,   5, 221, 176,
-135, 241,   1, 223, 146, 195, 166,  11, 146, 133, 226, 137,   6, 150, 243, 247,
-  1, 153, 246, 184,  42, 234, 194, 229,  98, 237, 144, 253, 133,  11, 196, 131,
-158, 244,   6, 218, 149, 253, 221,   7, 219, 180, 234, 156,  10, 179, 255, 197,
-218,  13, 150, 137, 240, 204,   9, 240, 185, 181, 203,   2, 160, 194, 146, 246,
-  5, 131, 168, 191, 138,   4, 158, 245, 240, 150,  15, 157, 202, 136,  14, 135,
-154, 226, 240,   5, 153, 168, 212, 222,   8, 128, 218, 198, 244, 133,  13, 183,
-245, 153, 118, 139, 141, 238, 141,   1, 235, 193, 197,   5, 169, 141, 210,  62,
-231, 186, 238, 219,   6, 141, 243, 204, 242,  12, 172, 165, 150, 187,  13, 163,
-254, 250, 230,  12, 203, 166, 166, 223,   3, 177, 155, 168, 182,   4, 213, 130,
-148, 221,   3, 150, 178, 146, 235,   6, 149, 226, 237, 225,   2, 177, 149, 218,
- 10, 205, 241, 161,  21, 186, 239, 197, 189,  15, 132, 249, 249, 171,   5, 130,
-223, 220, 167,   5, 171, 235, 129,  84, 207, 145, 246, 231,   2, 183, 176, 230,
-148,  11, 180, 142, 254, 128,   1, 171, 251, 177, 177,   1, 188, 190, 157, 222,
- 11, 140, 195, 192, 141,  10, 200, 139, 160, 247,   9, 139, 247, 194, 144,   1,
-160, 160, 234, 208,  11, 174, 210, 150, 196,  15, 209, 201, 176, 208,  14, 199,
-183, 218, 132,   8, 175, 143, 188, 168,   7, 172, 234, 158, 248,  11, 192, 223,
-160, 152,   7, 178, 134, 130, 235,   3, 243, 134, 181, 181,   4, 225, 135, 251,
-236,   7, 203, 166, 149, 169,  10, 181, 213, 156, 193,  12, 239, 138, 235, 252,
-  2, 183, 243, 201, 133,  10, 137, 186, 227, 237,  13, 255, 188, 221, 148,  14,
-188, 156, 198, 143,  15, 223, 224, 252, 208,   9, 160, 241, 190, 221,  13, 195,
-241, 163, 241,   9, 199, 253, 138, 163,  12, 173, 251, 143, 133,  12, 167, 246,
-153, 247,  14, 237, 223, 140, 174,  14, 219, 229, 138, 242,   2, 200, 163, 210,
- 86, 197, 251, 199, 241,   9, 243, 211, 209, 132,   3, 178, 176, 152, 224,  13,
-195, 131, 248, 159,   5, 194, 255, 160, 171,  14, 145, 243, 143, 173,   3, 222,
-168, 246, 134,   2, 178, 145, 204, 240,   1, 176, 240, 236, 165,  14, 254, 145,
-162, 165,   8, 243, 173, 131, 238,   3, 247, 192, 235, 163,   4, 244, 239, 180,
-203,  15, 214, 167, 152, 233,  13, 176, 158, 206, 235,   9, 252, 150, 228, 160,
- 13, 148, 243, 234, 239,   2, 225, 152, 250, 167,   5, 252, 143, 229, 254,   4,
-184, 202, 161, 157,  14, 233, 190, 185, 195,   9, 159, 223, 240, 216,  11, 132,
-172, 243, 200,   6, 212, 182, 191, 194,  13, 230, 245, 240, 130,  12, 189, 146,
-233, 239,   2, 155, 190, 214, 183,  15, 159, 222, 148, 155,  13, 195, 158, 248,
-112, 224, 219, 145, 234,  12, 145, 169, 172, 135,  10, 234, 184, 245, 220,   4,
-138, 150, 232, 212,   5, 132, 195, 135, 214,   5, 181, 247, 216, 205,  12, 239,
-160, 183, 178,   9, 161, 143, 210, 206,  11, 248, 209, 207,  94, 166, 178, 165,
- 97, 133, 162, 246, 212,   9, 206, 240, 235, 156,   1, 200, 228, 176, 252,  12,
-163, 215, 219, 141,   1, 236, 133, 216, 202,   9, 220, 170, 222, 242,  10, 239,
-203, 197, 220,  11, 148, 218, 209, 161,   7, 185, 175, 210, 171,  15, 153, 213,
-208, 214,  15, 188, 239, 128, 244,  13, 141, 220, 136, 166,  12, 150, 148, 250,
-175,  13, 130, 145, 226, 216,   1, 216, 204, 215, 193,   9, 191, 211, 181, 229,
- 14, 233, 168, 165,   9, 240, 188, 146, 132,  12, 173, 220, 201, 244,   4, 140,
-147, 190, 199,  15, 190, 213, 175, 213,   1, 254, 212, 239, 171,  10, 200, 161,
-168, 144,  10, 161, 188, 230, 163,   6, 192, 198, 213, 167,   3, 240, 251, 180,
-243,   5, 202, 165, 247, 147,   7, 173, 191, 133, 228,   3, 229, 139, 154, 210,
-  7, 147, 254, 164, 236,  13, 162, 214, 180, 128,   8, 202, 176, 252, 143,  13,
-154, 179, 169, 149,   3, 169, 156, 168, 229,   1, 164, 128, 214, 138,  15, 128,
-239, 253, 160, 181,   2, 232, 203, 196, 235,  11, 181, 153, 131, 240,  12, 145,
-178, 179, 206,  12, 134, 244, 215, 141,  10, 138, 228, 171, 244,   7, 246, 160,
-221, 177,  14, 176, 231, 208, 135,   9, 194, 210, 159, 234,   2, 238, 250, 139,
-146,  10, 249, 191, 224, 241,  10, 250, 140, 140, 147,   5, 190, 185, 216, 220,
- 15, 248, 131, 153, 236,   9, 140, 219, 183, 252,  14, 254, 184, 223, 216,  14,
-253, 211, 235, 254,  14, 252, 180, 147, 152,   9, 147, 221, 188, 174,   1, 222,
-219, 180, 185,  12, 185, 175, 244, 136,   9, 214, 147, 217, 182,   4, 191, 193,
-233, 157,   2, 238, 191, 156, 211,  14, 229, 221, 129, 224,   2, 230, 212, 248,
-128,   3, 186, 165, 136,  84, 129, 216, 148, 139,  15, 150, 231, 196, 184,   8,
-160, 156, 253, 171,   2, 156, 198, 161, 183,  11, 164, 181, 155, 137,   8, 133,
-196, 192, 213,   6, 140, 174, 143, 152,  12, 142, 202, 143, 192,   9, 128, 167,
-234, 152,  13, 214, 131, 156, 246,  14, 167, 223, 250, 135,   4, 233, 185, 236,
-128,   1, 138, 131, 251, 181,   9, 184, 141, 213, 136,  15, 171, 224, 222, 192,
- 12, 244, 168, 162, 144,   1, 212, 183, 184, 200,   9, 177, 193, 168, 174,  14,
-249, 175, 129, 197,   1, 142, 181, 130, 162,  10, 214, 197, 196, 214,   4, 148,
-146, 228, 202,  13, 213, 154, 241, 127, 165, 166, 144, 164,   4, 205, 251, 139,
-128,  13, 244, 188, 143, 236,  12, 190, 247, 138, 217,   8, 185, 201, 217, 187,
-  4, 130, 142, 167, 137,   4, 139, 185, 215,  95, 136, 170, 224, 218,   9, 154,
-158, 177, 200,  15, 227, 154, 189, 136,  15, 224, 233, 220, 179,   3, 227, 203,
-160, 188,   7, 236, 228, 239, 162,  15, 214, 227, 159, 242,   4, 151, 252, 232,
- 42, 151, 166, 168, 245,   3, 135, 180, 250, 243,  15, 167, 254, 137, 160,  13,
-214, 240, 225, 152,   8, 190, 229, 204, 136,  13, 150, 219, 186,  10, 163, 249,
-225, 249,   6, 215, 233, 254, 162,   9, 171, 204, 237, 189,   5, 229, 137, 174,
-157,   6, 135, 205, 140, 164,  10, 189, 136, 130, 244,   1, 210, 222, 223, 247,
-  1, 189, 128, 142, 203,  12, 232, 241, 180, 195,  12, 237, 228, 243, 183,   7,
-218, 155, 204, 158,  14, 235, 167, 134, 183,   6, 171, 218, 141, 128,   3, 184,
-152, 251, 187,  10, 138, 217, 169, 182,   2, 210, 140, 240, 138,   7, 150, 156,
-232, 128,   9, 209, 231, 181, 174,  14, 243, 210, 173,  34, 220, 254, 188, 199,
- 14, 245, 195, 226, 124, 141, 228, 248, 228,  15, 158, 166, 194, 150,   6, 152,
-220, 238, 252,  13, 179, 132, 217, 220,  15, 213, 168, 186, 245,   4, 241, 243,
-200, 226,  10, 216, 178, 141, 137,  13, 134, 176, 169, 179,   6, 212, 242, 197,
- 75, 175, 222, 238, 237,  10, 185, 143, 171, 166,   6, 180, 198, 129, 170,   5,
-159, 129, 176, 134,  11, 130, 248, 213, 183,  12, 204, 162, 169, 238,   8, 139,
-139, 145, 227,  15, 232, 239, 206, 163,   3, 145, 157, 143, 183,  10, 250, 190,
-179, 189,   3, 185, 138, 211, 215,   3, 179, 147, 158, 165,  13, 231, 226, 199,
-245,  11, 147, 179, 178, 190,   1, 208, 217, 154, 195,  14, 226, 194, 229, 142,
-  8, 198, 175, 184, 231,   4, 199, 198, 191,  24, 184, 134, 226, 231,  10, 152,
-208, 222, 254,   1, 134, 167, 234,  69, 175, 214, 177, 218,   3, 218, 234, 128,
-162,   3, 160, 177, 187, 166,   3, 201, 210, 191, 159,  13, 240, 152, 160, 250,
-  6, 235, 130, 214, 240,  11, 128, 237, 251, 245, 225,   3, 245, 237, 174, 230,
-  9, 252, 148, 229, 201,   7, 152, 148, 165, 153,   7, 223, 238, 242,  16, 156,
-212, 237, 228,   7, 139, 153, 178,  37, 219, 217, 217, 172,  15, 178, 168, 128,
-199,   9, 236, 189, 144, 226,  12, 214, 248, 134, 230,  13, 163, 252, 247,  55,
-239, 252, 149, 196,   3, 230, 159, 214, 139,   6, 132, 200, 241, 154,   2, 129,
-231, 153, 173,  12, 235, 131, 255, 157,   2, 246, 190, 145,  55, 205, 201, 240,
-141,   9, 188, 202, 199, 189,   6, 196, 235, 245, 205,  11, 249, 253, 241, 223,
-  6, 187, 250, 137, 241,   9, 133, 135, 168, 146,   8, 132, 248, 219, 156,   8,
-132, 241, 185,   4, 198, 209, 147, 129,  11, 229, 192, 218, 178,   4, 199, 210,
-138, 166,  13, 244, 148, 172, 141,   2, 194, 215, 171, 220,   1, 192, 248, 230,
-128,   2, 238, 167, 209, 222,  11, 240, 200, 227, 150,  11, 182, 217, 170, 158,
- 14, 223, 223, 254, 201,  10, 140, 164, 245, 175,   2, 178, 140, 153, 102, 139,
-145, 181, 242,   8, 188, 154, 214, 154,  15, 149, 187, 204, 192,   2, 223, 153,
-219,  51, 245, 236, 130, 133,   5, 197, 138, 169,  80, 243, 162, 164, 167,   1,
-206, 232, 180, 137,  12, 180, 191, 164, 226,   8, 162, 180, 231, 222,  13, 184,
-143, 156,  74, 134, 230, 248, 219,  10, 203, 156, 149, 205,   1, 219, 205, 173,
-167,  10, 174, 146, 180, 141,   7, 214, 231, 229, 231,  10, 181, 246, 174, 180,
- 15, 236, 175, 222, 241,   7, 191, 150, 253, 209,   8, 233, 139, 167, 149,  13,
-142, 249, 150, 223,  10, 220, 151, 135, 222,   5, 138, 228, 133, 131,   4, 232,
-183, 160, 245,   3, 157, 219, 209, 200,   5, 159, 242, 142, 148,  13, 241, 207,
-248, 177,  11, 179, 226, 169, 150,  13, 169, 201, 212, 218,   8, 172, 214, 220,
- 31, 155, 173, 251, 231,  12, 221, 150, 137, 174,  15, 146, 137, 251, 255,  14,
-245, 216, 203, 138,   1, 163, 170, 194, 133,  12, 205, 157, 188, 131,  12, 184,
-220, 161,  97, 162, 240, 190, 243,   2, 213, 134, 147, 251,   3, 178, 160, 193,
-188,  14, 214, 153, 226, 140,  12, 191, 208, 235, 174,  13, 138, 188, 204, 236,
- 11, 214, 135, 129, 235,  10, 198, 242, 226, 128,  11, 154, 219, 163, 144,   7,
-236, 134, 217, 197,   2, 181, 248, 144, 157,   8, 150, 174, 195, 224,  12, 156,
-247, 234, 192,   7, 156, 206, 174, 246,   2, 181, 214, 138, 155,   1, 246, 242,
-141, 152,   9, 207, 157, 139, 243,   1, 153, 135, 158, 249,   6, 162, 129, 144,
-170,  13, 227, 162, 245, 246,   1, 130, 237, 192, 208,  13, 187, 165, 153, 215,
-  8, 178, 141, 203, 163,  15, 172, 179, 180, 172,  10, 206, 200, 237, 194,  12,
-129, 235, 165, 143,   7, 129, 230, 217, 244,   8, 223, 249, 152, 233,   2, 160,
-224, 204, 187,  10, 167, 211, 138, 247,   7, 207, 204, 131, 200,   1, 207, 240,
-161, 219,   9, 219, 213, 129, 183,  11, 186, 163, 243, 198,  13, 217, 197, 175,
-218,   8, 195, 228, 209, 137,   1, 149, 253, 193, 190,   8, 216, 231, 225, 190,
- 15, 244, 168, 191, 152,   6, 180, 210, 162, 198,   9, 172, 159, 195, 158,   9,
-173, 151, 226,  34, 143, 231, 162, 212,   6, 250, 171, 192, 187,  11, 229, 212,
-155, 156,   9, 234, 159, 165, 254,   8, 180, 154, 227, 197,   3, 175, 158, 214,
-235,   8, 164, 157, 160, 130,   4, 158, 223, 243, 254,  10, 178, 236, 213, 212,
- 12, 194, 173, 185, 159,   6, 184, 214, 195, 172,   5, 128, 161, 203, 183, 194,
- 10, 207, 218, 209, 222,  12, 136, 166, 226, 224,   3, 148, 153, 145, 214,   4,
-164, 178, 253, 243,   4, 173, 162, 237, 129,   4, 236, 134, 193, 169,  14, 140,
-234, 164, 190,   7, 211, 148, 252, 223,   8, 213, 149, 180, 170,  12, 194, 182,
-191, 205,  15, 206, 233, 190, 211,   2, 241, 136, 223, 152,  12, 184, 185, 231,
-176,  10, 201, 166, 182, 211,   4, 209, 201, 205, 235,   1, 141, 184, 205, 173,
- 15, 244, 222, 218, 113, 175, 190, 179, 140,   4, 234, 232, 231, 183,   8, 174,
-167, 140, 130,   9, 169, 157, 136, 196,  14, 187, 244, 242, 135,   7, 248, 183,
-178, 253,  10, 135, 216, 152, 153,  15, 226, 223, 172, 161,  11, 236, 183, 231,
-216,   3, 183, 169, 209, 137,  13, 130, 219, 233, 167,   4, 168, 132, 197, 161,
-  7, 164, 146, 152, 207,   4, 239, 229, 147, 130,   2, 172, 156, 244, 148,   6,
-171, 253, 185, 213,   4, 184, 181, 241, 207,   1, 144, 250, 219, 222,   1, 213,
-189, 209, 177,  10, 207, 252, 251, 239,   9, 181, 132, 203, 147,   6, 159, 135,
-181,  18, 215, 252, 202, 234,   7, 207, 215, 210, 222,  12, 195, 211, 185, 171,
- 14, 178, 132, 165, 140,   9, 139, 160, 171, 250,   1, 248, 176, 203, 170,  14,
-148, 184, 131, 141,   4, 158, 226, 204, 197,   3, 215, 157, 148, 219,  15, 228,
-206, 156, 132,   3, 234, 206, 202, 231,   8, 232, 177, 135, 215,  10, 173, 253,
-176, 172,   5, 144, 188, 170, 229,  14, 200, 165, 144,  50, 198, 153, 206, 184,
-  3, 150, 128, 128, 141,  14, 155, 221, 221, 199,  12, 229, 199, 160, 156,   3,
-176, 172, 200,  97, 222, 255, 134, 158,   9, 233, 155, 199, 193,  14, 146, 216,
-186, 250,  13, 156, 152, 194, 212,   8, 254, 190, 240, 232,   2, 178, 210, 194,
-160,   3, 142, 216, 141, 184,  10, 173, 210, 214, 187,   2, 161, 211, 201, 143,
-  5, 213, 149, 210, 222,  15, 134, 165, 184, 171,   9, 211, 175, 153, 241,   9,
-227, 201, 184, 213,   1, 173, 225, 213, 176,  13, 143, 228, 200, 151,  12, 224,
-224, 224, 186,   8, 188, 153, 234, 254,   7, 137, 188, 238, 186,   8, 166, 236,
-135, 180,  13, 202, 174, 133, 194,  13, 179, 243, 158, 193,  13, 210, 173, 128,
-149,   2, 208, 216, 158, 168,  13, 205, 251, 152, 230,   3, 245, 245, 254, 163,
-  9, 211, 243, 234, 164,   9, 173, 221, 221, 215,   4, 146, 220, 209, 198,   1,
-235, 237, 170, 130,   7, 181, 227, 149, 141,   2, 170, 245, 149, 217,   5, 153,
-179, 215, 195,  14, 249, 206, 140, 148,   1, 247, 200, 219, 152,  15, 165, 228,
-197, 152,  11, 234, 192, 242, 244,   6, 217, 229, 173, 147,   3, 216, 209, 206,
-189,   7, 165, 171, 221, 214,   2, 151, 250, 211, 138,   2, 144, 169, 182, 176,
- 13, 179, 254, 191, 225,   3, 244, 147, 218, 212,   3, 129, 187, 183, 253,  10,
-218, 149, 188, 168,  10, 223, 241, 149, 129,   8, 209, 128, 150, 126, 153, 139,
-195, 131,   6, 201, 208, 246, 221,   1, 194, 165, 175, 173,   5, 197, 133, 207,
-196,   2, 192, 211, 129, 210,   7, 211, 147, 163, 220,   9, 173, 191, 188, 152,
-  1, 169, 242, 205,  20, 167, 133, 213, 211,   2, 213, 226, 129, 166,  12, 186,
-202, 155, 203,   5, 180, 251, 220, 174,  12, 145, 228, 247, 146,  12, 196, 151,
-247, 184,  10, 217, 233, 238, 147,   6, 149, 174, 181, 128,  13, 128, 246, 173,
-207,  15, 200, 162, 139, 103, 237, 199, 220, 252,   7, 208, 201, 133, 231,   3,
-140, 148, 223, 137,   5, 128, 242, 251, 140, 228,  11, 214, 205, 158, 228,   2,
-147, 190, 212, 138,   4, 228, 228, 253, 154,   9, 146, 191, 248, 187,   8, 168,
-200, 246, 160,   4, 224, 168, 147, 211,  11, 153, 197, 133, 229,   5, 176, 131,
-167, 203,   6, 213, 183, 189, 178,  10, 185, 222, 229, 183,   5, 171, 185, 208,
-162,  15, 203, 130, 137, 201,   6, 236, 152, 138, 176,   1, 221, 200, 169, 183,
- 11, 237, 230, 219, 108, 152, 247, 239, 145,  14, 242, 220, 245, 148,   6, 183,
-147, 218, 144,  11, 236, 190, 230, 197,   1, 253, 147, 205, 165,  10, 181, 130,
-138, 249,  10, 193, 135, 148, 142,  10, 232, 132, 254, 163,   4, 244, 153, 241,
-197,  13, 251, 150, 230, 242,  10, 211, 255, 182, 243,   3, 247, 137, 150, 236,
-  5, 137, 168, 208, 161,  10, 192, 178, 137, 210,  13, 192, 158, 177, 203,   7,
-237, 221, 208, 153,   4, 180, 129, 195, 139,   4, 195, 220, 254, 129,   8, 235,
-249, 252, 142,   2, 171, 195, 208, 162,  12, 205, 185, 192, 166,   9, 208, 205,
-169, 160,  10, 156, 148, 150, 185,   2, 246, 165, 207, 129,  12, 145, 207, 129,
-130,  15, 253, 209, 184, 133,  11, 247, 226, 200, 185,   9, 193, 147, 150, 128,
-  8, 251, 208, 155,  45, 251, 142, 248, 144,  15, 174, 199, 157, 236,  12, 206,
-215, 156, 131,  14, 224, 242, 193, 145,   9, 194, 231, 136, 243,   7, 135, 188,
-221, 220,  10, 252, 138, 172, 180,  15, 222, 245, 235, 161,   2, 147, 195, 191,
-195,   7, 191, 205, 163, 247,   3, 237, 172, 239, 187,   6, 137, 141, 231, 233,
- 10, 246, 253, 140, 184,   5, 191, 252, 199, 190,  13, 235, 212, 206, 220,   8,
-163, 219, 233, 232,  13, 166, 129, 242, 168,  12, 131, 217, 184, 209,   7, 138,
-139, 223, 216,   8, 186, 152, 149, 207,   6, 229, 191, 144, 149,   8, 223, 167,
-204, 251,   1, 181, 240, 166, 200,   9, 194, 230, 150, 122, 210, 176, 221, 179,
-  5, 137, 169, 225, 196,   2, 190, 138, 243, 173,  10, 155, 224, 148, 154,  15,
-180, 176, 218, 153,   2, 194, 220, 179, 239,   3, 209, 243, 151, 171,   1, 135,
-192, 192, 129,   3, 154, 145, 158, 166,   8, 174, 159, 201, 207,   1, 134, 247,
-247, 152,   5, 169, 139, 159, 171,   3, 173, 170, 159, 244,  15, 201, 205, 215,
-223,   9, 227, 214, 226, 134,  14, 237, 245, 216, 153,   1, 207, 208, 244,  63,
-136, 146, 237, 215,   2, 131, 173, 129, 187,   4, 150, 204, 222, 185,   6, 243,
-177, 246, 252,   5, 246, 173, 234, 215,  14, 207, 252, 211, 199,   3, 177, 211,
-230, 228,   5, 208, 143, 209, 191,  13, 173, 192, 232, 246,  12, 132, 255, 207,
-139,  14, 171, 129, 141, 173,   7, 255, 222, 227, 255,  12, 155, 193, 184, 244,
- 14, 171, 144, 214, 163,   1, 241, 232, 221, 228,  15, 188, 160, 210, 226,  13,
-189, 190, 189,   5, 204, 252, 250, 234,  10, 228, 161, 153, 190,   9, 210, 208,
-187, 214,   7, 198, 154, 214, 242,   9, 197, 163, 254,  27, 220, 251, 130, 172,
-  2, 193, 147, 157, 255,  14, 242, 131, 138, 180,  14, 200, 239, 175, 239,   5,
-181, 157, 238, 152,   1, 203, 211, 156, 220,  10, 210, 166, 223, 241,   2, 214,
-243, 250, 244,  10, 238, 200, 226, 216,   9, 168, 140, 235, 228,  14, 149, 176,
-161, 188,   9, 180, 224, 247, 138,  11, 168, 159, 157, 226,   7, 216, 226, 212,
-131,   5, 158, 162, 174, 190,   2, 147, 131, 155, 194,   4, 227, 156, 248, 169,
- 14, 210, 216, 130, 142,  14, 233, 234, 248, 230,  13, 146, 190, 216, 248,   9,
-128, 173, 190, 149, 182,  11, 254, 210, 132, 152,   8, 211, 239, 231, 248,   9,
-132, 255, 247, 168,   7, 149, 224, 145, 136,  14, 162, 220, 148, 134,   6, 204,
-244, 192, 159,   8, 178, 160, 245, 237,  15, 193, 167, 249, 251,   5, 238, 159,
-153, 199,   9, 228, 225, 136, 225,   9, 147, 221, 134, 220,   7, 249, 129, 250,
-131,   5, 255, 249, 227, 129,  15, 183, 246, 177, 190,  10, 217, 182, 196, 128,
-  6, 136, 242, 159, 173,   1, 244, 128, 137, 210,  10, 154, 223, 230, 173,   7,
-193, 171, 203, 220,   9, 193, 222, 146, 129,   2, 159, 229, 247, 153,   1, 205,
-139, 189, 204,  13, 181, 152, 211, 186,   3, 252, 181, 234, 182,   4, 230, 212,
-233, 169,  13, 134, 211, 157, 165,   1, 218, 165, 218, 239,   4, 148, 140, 245,
-130,  11, 197, 152, 165, 199,   2, 235, 219, 158, 232,   9, 187, 231, 171, 149,
- 12, 134, 191, 248, 157,   3, 219, 140, 128, 208,   1, 181, 140, 225, 226,  15,
-234, 239, 208, 170,  10, 166, 152, 192, 138,  15, 237, 204, 242, 197,  12, 230,
-224, 210,  68, 128, 170, 249, 251,  10, 193, 202, 171, 142,   7, 235, 192, 224,
-175,  14, 147, 243, 214,  94, 165, 202, 243, 157,   6, 192, 178, 204, 211,   8,
-242, 240, 207, 231,   4, 251, 234, 238, 218,   1, 207, 227, 224, 149,   4, 155,
-215, 210, 203,   2, 164, 248, 235, 166,   6, 226, 234, 165, 222,  13, 228, 197,
-249, 231,  14, 169, 172, 201, 163,  14, 149, 206, 208, 159,  15, 178, 216, 205,
-227,  15, 210, 228, 223, 220,   5, 161, 214, 153, 136,  11, 181, 178, 246, 212,
-  7, 128, 131, 238, 218,  13, 138, 156, 141, 139,  15, 134, 187, 137, 234,   4,
-152, 215, 181, 142,   6, 160, 185, 166, 193,  13, 213, 145, 204, 240,  13, 190,
-164, 216, 231,  13, 251, 208, 176, 231,   4, 243, 160, 187, 150,   5, 235, 251,
-246, 205,   3, 142, 232, 229, 222,   5, 227, 251, 238, 161,  12, 224, 198, 250,
-176,   3, 187, 162, 200, 223,   5, 199, 133, 234, 181,   3, 167, 160, 247, 232,
-  4, 174, 198, 216, 180,  15, 144, 251, 131, 187,  10, 161, 171, 169, 190,   9,
-223, 175, 171, 171,   4, 141, 165, 211, 128,   5, 139, 239, 131, 173,   3, 211,
-163, 253,  45, 212, 199, 216, 226,  11, 137, 216, 228, 198,   3, 216, 209, 199,
-233,   3, 249, 144, 225, 146,   1, 216, 184, 225, 218,   9, 197, 219, 219, 247,
- 12, 214, 227, 243, 240,  14, 221, 155, 244, 141,   4, 239, 249, 179, 130,   4,
-161, 187, 191, 135,   3, 245, 241, 237, 241,  12, 194, 211, 209, 238,   5, 252,
-210, 135, 149,   1, 134, 241, 220, 170,  12, 175, 208, 242, 229,   9, 181, 144,
-172, 202,   7, 170, 195, 174, 180,   5, 198, 153, 178, 158,   6, 146, 142, 204,
-119, 137, 185, 250, 204,  10, 208, 190, 240, 166,   1, 138, 183, 212, 226,   3,
-241, 240, 245, 140,  15, 250, 184, 161, 117, 198, 194, 173, 133,  15, 135, 247,
-179, 180,  11, 158, 233, 195, 162,   2, 209, 143, 142, 203,  13, 156, 215, 224,
-192,   5, 228, 223, 167, 163,   6, 253, 160, 223, 182,   5, 178, 178, 223, 147,
-  5, 150, 180, 221, 189,  10, 168, 197, 173, 169,   6, 166, 146, 252, 254,  15,
-154, 211, 198, 238,   6, 182, 166, 227, 223,   3, 152, 209, 173, 192,   3, 147,
-255, 130, 153,   9, 152, 159, 128, 195,   7, 204, 199, 174, 227,   8, 149, 133,
-142,  33, 236, 185, 160, 136,  14, 154, 137, 143, 236,   7, 246, 149, 237, 166,
-  3, 150, 184, 224, 232,   3, 204, 220, 171, 245,  15, 128, 131, 146, 236, 219,
- 10, 168, 253, 226, 198,   3, 196, 185, 159, 245,  14, 246, 239, 172, 207,   7,
-172, 188, 238, 233,  13, 193, 158, 247, 192,  10, 178, 146, 230, 233,   8, 143,
-221, 252, 145,   5, 169, 173, 160, 149,   7, 141, 199, 235,  35, 225, 224, 227,
-213,   7, 233, 249, 164, 132,  11, 255, 158, 248, 254,   2, 248, 200, 154, 176,
-  3, 168, 248, 134, 165,   8, 145, 177, 231, 188,  10, 189, 223, 182, 129,   7,
-246, 146, 219,  62, 185, 190, 133, 217,   3, 228, 177, 227, 170,   1, 230, 175,
-223, 120, 150, 130, 206, 166,   5, 223, 216, 157, 168,   1, 225, 151, 175, 248,
-  5, 140, 228, 227, 235,   7, 243, 148, 219, 250,   3, 250, 215, 234, 130,   1,
-191, 146, 221, 133,   8, 220, 223, 135, 100, 233, 148, 197, 224,  11, 164, 203,
-178, 134,   9, 170, 133, 159, 133,   8, 162, 189, 239,  68, 144, 186, 204, 211,
-  6, 167, 218, 219, 144,   2, 208, 155, 181, 237,   2, 253, 223, 151, 180,  15,
-137, 132, 173, 135,   7, 172, 137, 239, 146,  13, 250, 140, 255, 211,  11, 231,
-134, 228, 145,   3, 149, 220, 253, 168,  10, 236, 163, 149, 221,  10, 247, 151,
-236, 190,   6, 166, 210, 238,  52, 192, 248, 168, 229,   9, 237, 182, 227, 199,
- 12, 189, 199, 195, 216,  12, 178, 236, 220, 158,   2, 247, 182, 235, 221,  14,
-219, 148, 216, 159,  15, 158, 234, 200, 167,   2, 184, 132, 251, 232,   2, 138,
-227, 158, 204,  14, 225, 192, 227, 165,   8, 130, 214, 149, 173,  13, 210, 140,
-161, 181,   9, 222, 217, 168, 158,  10, 220, 222, 238, 137,  10, 237, 248, 184,
- 57, 167, 213, 169, 132,   5, 236, 173, 141,  25, 131, 201, 181, 180,   4, 133,
-182, 179, 134,  14, 243, 180, 195, 169,  11, 145, 153, 139, 242,  14, 210, 148,
-136, 230,   2, 174, 147, 246, 185,   7, 185, 230, 252, 230,  10, 247, 210, 139,
-242,  13, 187, 227, 199, 158,  14, 186, 209, 178, 166,   8, 148, 174, 212, 154,
-  6, 193, 139, 246, 160,   4, 180, 129, 135, 190,   7, 253, 202, 252, 194,   1,
-145, 192, 198, 192,   2, 136, 201, 194, 165,   5, 238, 198, 216, 222,   8, 148,
-132, 194, 231,   2, 179, 212, 226, 152,  13, 216, 203, 190,  81, 241, 158, 205,
-205,   3, 153, 250, 248, 251,  11, 157, 223, 163, 229,  11, 160, 240, 198, 156,
- 13, 155, 254, 151, 138,  14, 219, 233, 172, 254,   4, 186, 194, 189, 227,   4,
-169, 243, 181, 201,  14, 161, 158, 146, 201,   3, 135, 139, 242, 206,   4, 222,
-141, 186, 201,  11, 247, 182, 166, 198,  12, 141, 168, 155, 172,   4, 206, 218,
-254, 175,   4, 140, 213, 159, 204,   7, 214, 128, 160, 215,   9, 253, 242, 237,
-147,   8, 162, 233, 151, 181,   5, 183, 223, 151,  21, 132, 164, 206, 242,   1,
-179, 227, 155, 165,  11, 189, 251, 195, 212,   3, 154, 195, 137, 190,   6, 129,
-212, 227, 177,   4, 185, 141, 235, 183,   7, 233, 220, 229, 174,   4, 215, 138,
-248,  25, 161, 210, 193, 241,  14, 239, 201, 231, 152,  12, 240, 169, 204, 169,
- 14, 228, 195, 196, 225,   6, 250, 159, 144, 234,   1, 167, 238, 191, 142,  11,
-202, 222, 151, 207,   9, 205, 219, 185, 142,   3, 230, 224, 187, 235,   5, 194,
-167, 210, 173,   7, 235, 250, 253, 178,  12, 239, 128, 215, 198,  13, 130, 141,
-191, 238,   3, 173, 252, 172, 217,  14, 129, 203, 164,  16, 191, 131, 153, 141,
-  8, 133, 200, 131, 240,  15, 173, 165, 172,  11, 182, 247, 244, 165,   9, 128,
-238, 232, 219,  37, 214, 148, 220, 206,  10, 199, 154, 167, 130,   1, 188, 191,
-233, 235,   9, 167, 131, 215, 154,   5, 133, 224, 241, 202,   1, 237, 213, 192,
-223,   4, 160, 202, 178, 132,  10, 248, 217, 142, 133,  12, 199, 164, 231, 189,
-  5, 240, 129, 134, 189,   6, 173, 135, 204, 176,  15, 164, 142, 214, 137,   8,
-208, 169, 163, 251,  15, 196, 171, 247, 187,  14, 230, 177, 251, 130,  13, 200,
-234, 146, 173,   4, 252, 218, 210, 212,  10, 206, 187, 236, 129,   5, 165, 161,
-220, 171,  11, 135, 129, 179, 205,   2, 240, 251, 134, 254,   3, 136, 185, 186,
-220,  10, 230, 142, 156, 211,   1, 215, 243, 241, 179,  12, 141, 140, 140, 166,
-  5, 136, 183, 213, 220,  14, 182, 213, 134, 202,  10, 177, 197, 170, 230,   6,
-210, 133, 203, 128,  14, 145, 196, 176, 139,   5, 191, 143, 140, 133,  11, 247,
-155, 221, 233,  10, 131, 192, 238, 143,   3, 194, 196, 146, 129,   9, 245, 183,
-142, 133,   6, 200, 197, 143, 185,   2, 133, 144, 194, 144,   4, 149, 202, 240,
- 36, 230, 214, 182, 211,   5, 254, 227, 217, 246,   2, 128, 164, 220, 255,   5,
-132, 138, 149, 153,   6, 200, 139, 167,  97, 203, 137, 179, 195,   2, 141, 176,
-199, 134,   9, 165, 244, 225, 254,   3, 136, 180, 252, 193,   3, 200, 165, 159,
-207,  12, 147, 222, 142, 148,   5, 191, 146, 228, 191,   9, 213, 255, 236, 152,
- 13, 132, 240, 164, 174,   2, 204, 152, 214,   3, 251, 240, 222, 248,  10, 219,
-208, 211, 189,  15, 175, 252, 221,  88, 182, 234, 154, 107, 208, 190, 199, 159,
-  2, 209, 139, 150, 182,  13, 212, 219, 146, 154,  15, 221, 178, 221, 188,  11,
-148, 200, 197,  17, 129, 218, 170, 253,  11, 164, 244, 228, 252,   2, 220, 175,
-146, 195,  15, 141, 223, 154, 232,   9, 227, 186, 130, 220,   8, 153, 157, 145,
-139,  12, 233, 140, 173, 183,  12, 223, 255, 155, 139,  13, 162, 238, 129, 242,
- 11, 252, 162, 211, 191,   2, 228, 182, 210, 101, 171, 202, 191, 167,  11, 247,
-189, 170, 255,   3, 217, 150, 238, 215,  10, 173, 188, 234, 177,   5, 166, 139,
-147, 132,  12, 230, 216, 153, 200,   3, 182, 202, 167, 210,  12, 222, 169, 137,
-180,   7, 253, 249, 181, 197,   2, 198, 205, 156, 192,  12, 168, 135, 243, 185,
-  2, 138, 158, 139, 159,  11, 138, 210, 248, 255,  14, 157, 141, 161, 207,   9,
-218, 206, 244, 191,   4, 222, 169, 188, 238,   5, 133, 211, 152, 218,  14, 248,
-191, 242, 250,  13, 217, 188, 239, 231,  14, 137, 198, 135, 144,   1, 231, 227,
-214, 168,   7, 128, 136, 152, 103, 150, 151, 161, 171,  12, 251, 222, 212, 229,
-  4, 154, 193, 182,  62, 251, 246, 205, 142,   3, 132, 140, 242, 166,  14, 165,
-231, 192, 250,   6, 136, 154, 230, 163,   1, 230, 228, 246, 182,   3, 187, 215,
-217, 177,   8, 137, 171, 251,  15, 211, 128, 230, 244,  15, 160, 146, 188, 255,
-  4, 204, 242, 150, 194,   1, 128, 184, 177, 139,  14, 139, 209, 245, 134,  11,
-241, 167, 181, 139,   5, 159, 129, 160,  74, 159, 200, 133, 222,   5, 157, 204,
-165, 199,  10, 193, 159, 169, 151,  11, 205, 219, 226, 134,   9, 197, 252, 179,
-128,  14, 230, 250, 244, 215,   5, 207, 138, 239, 212,  14, 237, 216, 191, 199,
- 15, 250, 250, 198, 148,   9, 212, 228, 174, 146,  15, 221, 137, 207, 196,   3,
-146, 165, 245, 220,  13, 157, 249, 149, 228,   5, 185, 219, 188, 185,   8, 212,
-150, 240, 218,  15, 128, 211, 229, 202, 129,  14, 132, 225, 178, 226,   1, 251,
-195, 132,  66, 210, 245, 154, 234,   5, 145, 183, 146, 177,   9, 218, 223, 128,
-170,  13, 238, 227, 168, 197,  11, 189, 225, 206, 179,   6, 221, 169, 239, 193,
-  4, 194, 207, 170, 203,   7, 163, 206, 232, 197,   1, 160, 130, 131, 160,   4,
-139, 146, 149, 173,  10, 140, 240, 243, 180,   4, 231, 180, 202, 245,   9, 146,
-250, 195, 157,   1, 233, 199, 188, 210,  15, 253, 222, 137, 142,  10, 174, 245,
-231,  20, 219, 156, 185, 201,   5, 139, 137, 230, 135,   1, 236, 207, 146, 138,
-  4, 149, 174, 164, 221,   4, 158, 227, 224, 210,   7, 206, 150, 186, 244,   9,
-156, 183, 159, 142,  13, 176, 152, 163, 193,   8, 190, 229, 232, 155,   7, 234,
-132, 236, 132,   9, 242, 254, 204, 134,  14, 143, 226, 253, 180,   2, 138, 226,
-214, 218,   2, 199, 228, 210, 186,  12, 147, 179, 230, 254,   5, 249, 135, 247,
-147,  10, 148, 253, 186, 214,  12, 250, 240, 173, 159,  14, 162, 215, 177,  42,
-162, 142, 248, 135,   3, 196, 143, 150, 150,  10, 236, 221, 178, 147,   7, 165,
-248, 197, 136,   7, 199, 152, 158, 228,  13, 229, 215, 242, 194,   7, 145, 249,
-246, 181,  13, 134, 191, 196, 245,   3, 161, 251, 235, 200,  14, 255, 232, 248,
-228,  10, 170, 188, 227, 177,  14, 212, 202, 144, 143,  13, 199, 230, 234, 155,
- 10, 247, 239, 142, 167,   6, 197, 129, 192, 235,   2, 207, 229, 194, 237,  12,
-228, 239, 211, 136,   3, 199, 135, 194, 244,   4, 167, 137, 158, 132,  15, 208,
-199, 176, 183,   2, 161, 181, 218, 155,  11, 218, 235, 160, 207,   5, 250, 181,
-244, 252,   9, 197, 130, 193, 168,   1, 153, 235, 181, 253,   2, 203, 245, 229,
-255,  11, 134, 136, 148, 249,   8, 179, 174, 133, 187,   8, 145, 212, 156, 196,
-  7, 163, 222, 227, 236,  11, 242, 171, 200, 143,  12, 185, 225, 231, 211,  15,
-135, 230, 213, 153,   6, 254, 187, 227, 167,   2, 147, 191, 160, 185,  12, 177,
-145, 137, 133,   1, 241, 244, 217, 231,   3, 225, 213, 246, 253,  11, 138, 185,
-169, 229,   8, 129, 248, 228, 155,   4, 150, 208, 194, 129,  13, 149, 233, 140,
-159,   7, 149, 223, 199,  33, 153, 214, 176, 117, 175, 193, 163, 144,   9, 135,
-207, 150,  12, 216, 138, 151,  55, 233, 245, 225, 219,   8, 215, 194, 201, 214,
-  6, 235, 254, 134,  70, 251, 142, 174, 209,  12, 215, 218, 132, 174,   4, 209,
-177, 189, 144,   3, 247, 136, 205, 212,   8, 152, 220, 178, 208,   5, 183, 146,
-202, 149,   6, 248, 229, 196, 211,  12, 226, 191, 237, 227,   7, 234, 157, 195,
-196,   4, 203, 147, 213, 156,   1, 245, 161, 241,  97, 186, 245, 223, 246,   8,
-170, 241, 234, 188,   8, 171, 155, 201, 168,   8, 193, 168, 145, 142,  10, 254,
-183, 192, 202,  14, 137, 175, 147, 223,   9, 176, 133, 131, 166,  12, 211, 168,
-155, 225,   4, 197, 193, 255, 204,   8, 154, 208, 144, 165,   1, 134, 190, 143,
-217,  11, 148, 242, 203, 237,  11, 161, 142, 172, 215,   3, 166, 203, 240, 162,
-  6, 200, 195, 186, 162,   7, 198, 211, 223, 252,  15, 132, 160, 226, 204,  15,
-158, 187, 167, 222,   6, 174, 214, 139, 220,   9, 130, 243, 221, 206,   6, 190,
-217, 211, 145,   4, 160, 255, 142, 201,   5, 201, 166, 217, 174,   7, 240, 197,
-130, 214,   7, 216, 133, 220, 184,   3, 241, 148, 192, 185,   6, 213, 181, 240,
-210,   2, 137, 194, 206, 172,   5, 221, 189, 134, 241,  10, 128, 180, 234, 178,
-219,  13, 203, 213, 182, 247,  10, 172, 229, 222, 178,  15, 188, 154, 206, 196,
- 12, 240, 136, 172, 156,  11, 165, 151, 164, 200,   7, 189, 152, 225, 146,   7,
-214, 167, 205, 147,   4, 216, 175, 130, 230,  10, 243, 162, 145, 154,   7, 155,
-169, 190, 182,   6, 255, 212, 152, 251,   6, 147, 152, 160, 237,   2, 170, 228,
-233, 210,  13, 166, 255, 247, 207,  14, 238, 175, 242, 171,   9, 174, 241, 193,
-193,   4, 245, 210, 147, 167,  14, 151, 233, 199, 154,   1, 193, 184, 194, 249,
-  9, 216, 255, 201, 246,  10, 138, 198, 240, 208,   5, 187, 230, 137, 145,   1,
-200, 237, 144, 115, 131, 149, 167, 201,  15, 249, 130, 240, 202,   5, 141, 220,
-198, 233,   3, 216, 165, 204, 210,  12, 176, 166, 249, 207,   4, 244, 158, 162,
-140,   8, 174, 153, 181, 253,  14, 249, 157, 148, 130,  10, 178, 203, 201, 162,
-  4, 161, 215, 176, 137,   3, 164, 232, 198, 200,  15, 141, 189, 153, 206,   6,
-148, 138, 219, 252,  12, 147, 134, 206, 210,   9, 214, 186, 141, 183,  10, 235,
-192, 204, 245,  10, 155, 177, 148, 174,   7, 246, 150, 200, 167,  15, 134, 228,
-212, 210,   7, 128, 198, 173, 133,  10, 173, 148, 155, 170,   4, 131, 242, 205,
-148,  14, 154, 220, 156, 236,  11, 213, 150, 219, 145,   4, 171, 231, 199, 224,
- 12, 190, 139, 161, 155,   3, 136, 151, 199, 129,   9, 182, 161, 156, 237,   1,
-218, 151, 248, 132,  13, 201, 207, 164, 115, 190, 137, 205, 255,  11, 191, 198,
-251, 165,  10, 234, 205, 249, 181,   3, 172, 185, 218, 244,  14, 134, 171, 214,
-151,   9, 152, 245, 182, 215,  10, 204, 161, 209, 196,  14, 180, 134, 204, 240,
-  4, 242, 196, 170, 185,  13, 156, 255, 134, 178,  14, 203, 145, 211, 216,   3,
-190, 148, 160, 180,  14, 189, 162, 214, 209,  10, 238, 176, 239, 248,  15, 151,
-163, 176, 168,   5, 152, 247, 207, 238,  14, 181, 238, 168, 251,   8, 181, 189,
-202,  33, 232, 239, 229, 226,   5, 133, 156, 212, 180,   4, 224, 169, 249, 216,
-  4, 198, 245, 205, 147,   8, 231, 232, 149, 230,   8, 243, 161, 191, 162,   3,
-194, 189, 237, 227,  15, 223, 185, 161, 232,   4, 153, 233, 249, 155,   8, 240,
-147, 199, 249,   5, 135, 205, 250, 160,  11, 252, 183, 238, 210,  10, 244, 146,
-156, 160,   5, 196, 252, 142,  22, 191, 148, 222, 231,  11, 182, 201, 163, 219,
- 13, 199, 238, 233, 179,   4, 180, 199, 255, 249,   4, 254, 237, 180, 213,   2,
-211, 221, 157, 151,   7, 178, 192, 158, 241,  13, 133, 212, 252,  51, 146, 221,
-241, 177,   7, 137, 246, 204, 171,   5, 138, 209, 144, 231,   2, 153, 213, 230,
-179,   4, 171, 244, 213, 172,   3, 191, 201, 249, 129,   3, 184, 184, 186, 243,
- 11, 145, 176, 183, 103, 145, 131, 206, 147,  14, 136, 134, 191, 173,   9, 180,
-164, 241, 245,   5, 172, 133, 212, 167,   8, 198, 162, 158, 244,   4, 232, 175,
-222, 231,  12, 146, 246, 134, 196,   8, 147, 248, 177, 230,   8, 145, 216, 180,
-139,  12, 224, 254, 191, 222,   1, 182, 145, 213, 232,  10, 178, 139, 143, 237,
-  9, 253, 230, 172, 181,  13, 225, 218, 252, 132,   6, 141, 175, 159, 197,  14,
-185, 222, 237, 246,   3, 154, 184, 245, 228,  11, 193, 198, 235, 204,  10, 182,
-239, 253, 136,  15, 205, 143, 161, 211,   7, 164, 207, 235, 220,   4, 158, 235,
-183, 187,   9, 203, 201, 147, 139,   3, 169, 181, 153, 201,  11, 222, 206, 192,
-251,  12, 221, 253, 242, 152,  11, 128, 249, 190, 248, 152,   5, 151, 199, 221,
-227,  14, 209, 246, 133, 200,   7, 246, 181, 176, 131,   9, 165, 219, 139, 171,
-  4, 254, 130, 187, 208,   8, 144, 221, 189, 192,  10, 163, 146, 139, 166,  12,
-231, 177, 223, 205,   9, 229, 179, 214, 227,   2, 132, 153, 150, 154,   5, 242,
-250, 159, 171,   9, 144, 228, 238, 120, 168, 206, 130, 107, 145, 144, 235, 248,
-  1, 254, 218, 166, 129,   4, 237, 129, 235,   7, 150, 199, 251, 175,   9, 252,
-199, 200, 168,   9, 172, 147, 153, 151,   5, 168, 129, 129, 188,  13, 200, 166,
-192, 192,   8, 154, 184, 218, 232,   2, 155, 202, 193, 156,  12, 182, 241, 250,
-153,  13, 180, 141, 206, 141,   6, 206, 129, 157, 153,  12, 132, 158, 212, 247,
- 14, 160, 135, 203, 238,  12, 216, 173, 204, 156,   9, 166, 214, 242, 138,  14,
-178, 248, 246, 135,   1, 244, 219, 210, 155,   3, 208, 155, 189, 180,   3, 156,
-189, 171, 174,  13, 162, 161, 233, 108, 231, 134, 177, 255,  11, 216, 159, 226,
-244,   2, 130, 227, 211, 185,   6, 169, 146, 187, 143,   2, 153, 225, 150, 187,
-  9, 210, 153, 211, 181,  13, 147, 216, 152, 173,   3, 246, 236, 142,  33, 222,
-140, 194, 241,  10, 171, 251, 248, 210,  11, 239, 197, 137, 242,   8, 132, 179,
-189, 209,   6, 181, 221, 179, 161,   2, 168, 172, 241, 163,   2, 203, 241, 250,
-226,  12, 184, 188, 237, 210,   8, 228, 163, 153, 234,   5, 222, 162, 216, 142,
- 13, 235, 251, 186, 239,   1, 199, 133, 166, 158,   9, 134, 241, 161, 174,   3,
-183, 248, 214, 158,   7, 181, 248, 184, 143,  11, 234, 152, 151, 169,   9, 198,
-134, 159, 251,   7, 144, 176, 211, 121, 199, 255, 166, 132,   5, 201, 243, 215,
-189,  14, 213, 240, 205, 223,  10, 205, 191, 234, 185,   9, 240, 221, 255, 234,
-  5, 210, 250, 179, 148,   7, 185, 162, 155, 243,   3, 140, 197, 165, 222,  12,
-150, 143, 215, 241,  13, 138, 138, 246,  30, 236, 151, 243, 235,  12, 232, 222,
-197, 223,   5, 177, 198, 228, 194,   4, 130, 172, 242, 221,  11, 208, 235, 221,
-161,   1, 254, 141, 148, 144,  14, 168, 251, 185, 179,   9, 247, 144, 244, 178,
- 12, 209, 235, 151, 183,   9, 131

<TRUNCATED>

[15/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/test/TestByteRle.cc
----------------------------------------------------------------------
diff --git a/c++/test/TestByteRle.cc b/c++/test/TestByteRle.cc
new file mode 100644
index 0000000..ecaa504
--- /dev/null
+++ b/c++/test/TestByteRle.cc
@@ -0,0 +1,1385 @@
+/**
+ * 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 "ByteRLE.hh"
+#include "OrcTest.hh"
+#include "wrap/gtest-wrapper.h"
+
+#include <iostream>
+#include <vector>
+
+namespace orc {
+
+TEST(ByteRle, simpleTest) {
+  const unsigned char buffer[] = {0x61, 0x00, 0xfd, 0x44, 0x45, 0x46};
+  std::unique_ptr<ByteRleDecoder> rle =
+      createByteRleDecoder(
+        std::unique_ptr<SeekableInputStream>
+        (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer))));
+  std::vector<char> data(103);
+  rle->next(data.data(), data.size(), nullptr);
+
+  for(size_t i = 0; i < 100; ++i) {
+    EXPECT_EQ(0, data[i]) << "Output wrong at " << i;
+  }
+  EXPECT_EQ(0x44, data[100]);
+  EXPECT_EQ(0x45, data[101]);
+  EXPECT_EQ(0x46, data[102]);
+}
+
+TEST(ByteRle, nullTest) {
+  char buffer[258];
+  char notNull[266];
+  char result[266];
+  buffer[0] = -128;
+  buffer[129] = -128;
+  for(int i=0; i < 128; ++i) {
+    buffer[1 + i] = static_cast<char>(i);
+    buffer[130 + i] = static_cast<char>(128 + i);
+  }
+  for(int i=0; i < 266; ++i) {
+    notNull[i] = static_cast<char>(i >= 10);
+  }
+  std::unique_ptr<ByteRleDecoder> rle =
+    createByteRleDecoder(std::unique_ptr<SeekableInputStream>
+                         (new SeekableArrayInputStream(buffer,
+                                                       sizeof(buffer))));
+  rle->next(result, sizeof(result), notNull);
+  for(size_t i = 0; i < sizeof(result); ++i) {
+    if (i >= 10) {
+      EXPECT_EQ((i - 10) & 0xff,
+                static_cast<int>(result[i]) & 0xff) << "Output wrong at " << i;
+    }
+  }
+}
+
+TEST(ByteRle, literalCrossBuffer) {
+  const unsigned char buffer[] = {0xf6, 0x00, 0x01, 0x02, 0x03, 0x04,
+				  0x05, 0x06, 0x07, 0x08, 0x09, 0x07, 0x10};
+  std::unique_ptr<ByteRleDecoder> rle =
+      createByteRleDecoder(
+        std::unique_ptr<SeekableInputStream>
+        (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer), 6)));
+  std::vector<char> data(20);
+  rle->next(data.data(), data.size(), nullptr);
+
+  for(size_t i = 0; i < 10; ++i) {
+    EXPECT_EQ(i, data[i]) << "Output wrong at " << i;
+  }
+  for(size_t i = 10; i < 20; ++i) {
+    EXPECT_EQ(16, data[i]) << "Output wrong at " << i;
+  }
+}
+
+TEST(ByteRle, skipLiteralBufferUnderflowTest) {
+  const unsigned char buffer[] =
+    {0xf8, 0x0, 0x1, 0x2, 0x3, 0x4, 0x5, 0x6, 0x7};
+  std::unique_ptr<ByteRleDecoder> rle =
+      createByteRleDecoder(
+        std::unique_ptr<SeekableInputStream>(
+          new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer), 4)));
+  std::vector<char> data(8);
+  rle->next(data.data(), 3, nullptr);
+  EXPECT_EQ(0x0, data[0]);
+  EXPECT_EQ(0x1, data[1]);
+  EXPECT_EQ(0x2, data[2]);
+
+  rle->skip(2);
+  rle->next(data.data(), 3, nullptr);
+  EXPECT_EQ(0x5, data[0]);
+  EXPECT_EQ(0x6, data[1]);
+  EXPECT_EQ(0x7, data[2]);
+}
+
+TEST(ByteRle, simpleRuns) {
+  const unsigned char buffer[] = {0x0d, 0xff, 0x0d, 0xfe, 0x0d, 0xfd};
+  std::unique_ptr<ByteRleDecoder> rle =
+      createByteRleDecoder(
+        std::unique_ptr<SeekableInputStream>
+	(new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer))));
+  std::vector<char> data(16);
+  for (size_t i = 0; i < 3; ++i) {
+    rle->next(data.data(), data.size(), nullptr);
+    for (size_t j = 0; j < data.size(); ++j) {
+      EXPECT_EQ(static_cast<char>(-1 - static_cast<int>(i)), data[j])
+        << "Output wrong at " << (16 * i + j);
+    }
+  }
+}
+
+TEST(ByteRle, splitHeader) {
+  const unsigned char buffer[] = {0x00, 0x01, 0xe0, 0x01, 0x02, 0x03, 0x04,
+				  0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b,
+				  0x0c, 0x0d, 0x0e, 0x0f, 0x10, 0x11, 0x12,
+				  0x13, 0x14, 0x15, 0x16, 0x17, 0x18, 0x19,
+				  0x1a, 0x1b, 0x1c, 0x1d, 0x1e, 0x1f, 0x20};
+  std::unique_ptr<ByteRleDecoder> rle =
+      createByteRleDecoder(
+        std::unique_ptr<orc::SeekableInputStream>
+	(new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer), 1)));
+  std::vector<char> data(35);
+  rle->next(data.data(), data.size(), nullptr);
+  for (size_t i = 0; i < 3; ++i) {
+    EXPECT_EQ(1, data[i]) << "Output wrong at " << i;
+  }
+  for (size_t i = 3; i < data.size(); ++i) {
+    EXPECT_EQ(i-2, data[i]) << "Output wrong at " << i;
+  }
+}
+
+TEST(ByteRle, splitRuns) {
+  const unsigned char buffer[] =
+    {0x0d, 0x02, 0xf0, 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08,
+     0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10};
+  std::unique_ptr<ByteRleDecoder> rle =
+      createByteRleDecoder(
+        std::unique_ptr<orc::SeekableInputStream>
+	(new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer))));
+  std::vector<char> data(5);
+  for (size_t i = 0; i < 3; ++i) {
+    rle->next(data.data(), data.size(), nullptr);
+    for (size_t j = 0; j < data.size(); ++j) {
+      EXPECT_EQ(2, data[j]) << "Output wrong at " << (i * data.size() + j);
+    }
+  }
+  rle->next(data.data(), data.size(), nullptr);
+  EXPECT_EQ(2, data[0]) << "Output wrong at 15";
+  for (size_t i = 1; i < data.size(); ++i) {
+    EXPECT_EQ(i, data[i]) << "Output wrong at " << (15 + i);
+  }
+  for(size_t i = 0; i < 2; ++i) {
+    rle->next(data.data(), data.size(), nullptr);
+    for(size_t j = 0; j < data.size(); ++j) {
+      EXPECT_EQ(5 * i + j + data.size(), data[j])
+        << "Output wrong at " << (20 + data.size() * i + j);
+    }
+  }
+  rle->next(data.data(), 2, nullptr);
+  EXPECT_EQ(15, data[0]) << "Output wrong at 30" ;
+  EXPECT_EQ(16, data[1]) << "Output wrong at 31" ;
+}
+
+TEST(ByteRle, testNulls) {
+  const unsigned char buffer[] = 
+    {0xf0, 0x00, 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07,
+     0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x3d, 0xdc};
+  std::unique_ptr<ByteRleDecoder> rle =
+      createByteRleDecoder(
+        std::unique_ptr<orc::SeekableInputStream>
+	(new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer), 3)));
+  std::vector<char> data(16, -1);
+  std::vector<char> notNull(data.size());
+  for (size_t i = 0; i < data.size(); ++i) {
+    notNull[i] = (i + 1) % 2;
+  }
+  for (size_t i = 0; i < 2; ++i) {
+    rle->next(data.data(), data.size(), notNull.data());
+    for (size_t j = 0; j < data.size(); ++j) {
+      if (j % 2 == 0) {
+        EXPECT_EQ((i*data.size() + j)/2, data[j]) << "Output wrong at "
+                                                  << (i * data.size() + j);
+      } else {
+        EXPECT_EQ(-1, data[j]) << "Output wrong at "
+                               << (i * data.size() + j);
+      }
+    }
+  }
+  for (size_t i = 0; i < 8; ++i) {
+    rle->next(data.data(), data.size(), notNull.data());
+    for (size_t j = 0; j < data.size(); ++j) {
+      EXPECT_EQ(j % 2 == 0 ? -36 : -1,
+                data[j])
+          << "Output wrong at " << (i * data.size() + j + 32);
+    }
+  }
+}
+
+TEST(ByteRle, testAllNulls) {
+  const unsigned char buffer[] =
+    {0xf0, 0x00, 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07,
+     0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x3d, 0xdc};
+  std::unique_ptr<ByteRleDecoder> rle =
+      createByteRleDecoder(
+        std::unique_ptr<orc::SeekableInputStream>
+	(new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer))));
+  std::vector<char> data(16, -1);
+  std::vector<char> allNull(data.size(), 0);
+  std::vector<char> noNull(data.size(), 1);
+  rle->next(data.data(), data.size(), allNull.data());
+  for (size_t i = 0; i < data.size(); ++i) {
+    EXPECT_EQ(-1, data[i]) << "Output wrong at " << i;
+  }
+  rle->next(data.data(), data.size(), noNull.data());
+  for (size_t i = 0; i < data.size(); ++i) {
+    EXPECT_EQ(i, data[i]) << "Output wrong at " << i;
+    data[i] = -1;
+  }
+  rle->next(data.data(), data.size(), allNull.data());
+  for (size_t i = 0; i < data.size(); ++i) {
+    EXPECT_EQ(-1, data[i]) << "Output wrong at " << i;
+  }
+  for (size_t i = 0; i < 4; ++i) {
+    rle->next(data.data(), data.size(), noNull.data());
+    for (size_t j = 0; j < data.size(); ++j) {
+      EXPECT_EQ(-36, data[j]) << "Output wrong at " << i;
+    }
+  }
+  rle->next(data.data(), data.size(), allNull.data());
+}
+
+TEST(ByteRle, testSkip) {
+  // the stream generated by Java's TestRunLengthByteReader.testSkips
+  // for (int i = 0; i < 2048; ++i) {
+  //   if (i < 1024) {
+  //     out.write(i / 16);
+  //   } else {
+  //     out.write(i % 256);
+  //   }
+  // }
+  const unsigned char buffer[] = {
+ 0xd,  0x0,  0xd,  0x1,  0xd,  0x2,  0xd,  0x3,  0xd,  0x4,  0xd,  0x5,  0xd,
+ 0x6,  0xd,  0x7,  0xd,  0x8,  0xd,  0x9,  0xd,  0xa,  0xd,  0xb,  0xd,  0xc,
+ 0xd,  0xd,  0xd,  0xe,  0xd,  0xf,  0xd, 0x10,  0xd, 0x11,  0xd, 0x12,  0xd,
+0x13,  0xd, 0x14,  0xd, 0x15,  0xd, 0x16,  0xd, 0x17,  0xd, 0x18,  0xd, 0x19,
+ 0xd, 0x1a,  0xd, 0x1b,  0xd, 0x1c,  0xd, 0x1d,  0xd, 0x1e,  0xd, 0x1f,  0xd,
+0x20,  0xd, 0x21,  0xd, 0x22,  0xd, 0x23,  0xd, 0x24,  0xd, 0x25,  0xd, 0x26,
+ 0xd, 0x27,  0xd, 0x28,  0xd, 0x29,  0xd, 0x2a,  0xd, 0x2b,  0xd, 0x2c,  0xd,
+0x2d,  0xd, 0x2e,  0xd, 0x2f,  0xd, 0x30,  0xd, 0x31,  0xd, 0x32,  0xd, 0x33,
+ 0xd, 0x34,  0xd, 0x35,  0xd, 0x36,  0xd, 0x37,  0xd, 0x38,  0xd, 0x39,  0xd,
+0x3a,  0xd, 0x3b,  0xd, 0x3c,  0xd, 0x3d,  0xd, 0x3e,  0xd, 0x3f, 0x80,  0x0,
+ 0x1,  0x2,  0x3,  0x4,  0x5,  0x6,  0x7,  0x8,  0x9,  0xa,  0xb,  0xc,  0xd,
+ 0xe,  0xf, 0x10, 0x11, 0x12, 0x13, 0x14, 0x15, 0x16, 0x17, 0x18, 0x19, 0x1a,
+0x1b, 0x1c, 0x1d, 0x1e, 0x1f, 0x20, 0x21, 0x22, 0x23, 0x24, 0x25, 0x26, 0x27,
+0x28, 0x29, 0x2a, 0x2b, 0x2c, 0x2d, 0x2e, 0x2f, 0x30, 0x31, 0x32, 0x33, 0x34,
+0x35, 0x36, 0x37, 0x38, 0x39, 0x3a, 0x3b, 0x3c, 0x3d, 0x3e, 0x3f, 0x40, 0x41,
+0x42, 0x43, 0x44, 0x45, 0x46, 0x47, 0x48, 0x49, 0x4a, 0x4b, 0x4c, 0x4d, 0x4e,
+0x4f, 0x50, 0x51, 0x52, 0x53, 0x54, 0x55, 0x56, 0x57, 0x58, 0x59, 0x5a, 0x5b,
+0x5c, 0x5d, 0x5e, 0x5f, 0x60, 0x61, 0x62, 0x63, 0x64, 0x65, 0x66, 0x67, 0x68,
+0x69, 0x6a, 0x6b, 0x6c, 0x6d, 0x6e, 0x6f, 0x70, 0x71, 0x72, 0x73, 0x74, 0x75,
+0x76, 0x77, 0x78, 0x79, 0x7a, 0x7b, 0x7c, 0x7d, 0x7e, 0x7f, 0x80, 0x80, 0x81,
+0x82, 0x83, 0x84, 0x85, 0x86, 0x87, 0x88, 0x89, 0x8a, 0x8b, 0x8c, 0x8d, 0x8e,
+0x8f, 0x90, 0x91, 0x92, 0x93, 0x94, 0x95, 0x96, 0x97, 0x98, 0x99, 0x9a, 0x9b,
+0x9c, 0x9d, 0x9e, 0x9f, 0xa0, 0xa1, 0xa2, 0xa3, 0xa4, 0xa5, 0xa6, 0xa7, 0xa8,
+0xa9, 0xaa, 0xab, 0xac, 0xad, 0xae, 0xaf, 0xb0, 0xb1, 0xb2, 0xb3, 0xb4, 0xb5,
+0xb6, 0xb7, 0xb8, 0xb9, 0xba, 0xbb, 0xbc, 0xbd, 0xbe, 0xbf, 0xc0, 0xc1, 0xc2,
+0xc3, 0xc4, 0xc5, 0xc6, 0xc7, 0xc8, 0xc9, 0xca, 0xcb, 0xcc, 0xcd, 0xce, 0xcf,
+0xd0, 0xd1, 0xd2, 0xd3, 0xd4, 0xd5, 0xd6, 0xd7, 0xd8, 0xd9, 0xda, 0xdb, 0xdc,
+0xdd, 0xde, 0xdf, 0xe0, 0xe1, 0xe2, 0xe3, 0xe4, 0xe5, 0xe6, 0xe7, 0xe8, 0xe9,
+0xea, 0xeb, 0xec, 0xed, 0xee, 0xef, 0xf0, 0xf1, 0xf2, 0xf3, 0xf4, 0xf5, 0xf6,
+0xf7, 0xf8, 0xf9, 0xfa, 0xfb, 0xfc, 0xfd, 0xfe, 0xff, 0x80,  0x0,  0x1,  0x2,
+ 0x3,  0x4,  0x5,  0x6,  0x7,  0x8,  0x9,  0xa,  0xb,  0xc,  0xd,  0xe,  0xf,
+0x10, 0x11, 0x12, 0x13, 0x14, 0x15, 0x16, 0x17, 0x18, 0x19, 0x1a, 0x1b, 0x1c,
+0x1d, 0x1e, 0x1f, 0x20, 0x21, 0x22, 0x23, 0x24, 0x25, 0x26, 0x27, 0x28, 0x29,
+0x2a, 0x2b, 0x2c, 0x2d, 0x2e, 0x2f, 0x30, 0x31, 0x32, 0x33, 0x34, 0x35, 0x36,
+0x37, 0x38, 0x39, 0x3a, 0x3b, 0x3c, 0x3d, 0x3e, 0x3f, 0x40, 0x41, 0x42, 0x43,
+0x44, 0x45, 0x46, 0x47, 0x48, 0x49, 0x4a, 0x4b, 0x4c, 0x4d, 0x4e, 0x4f, 0x50,
+0x51, 0x52, 0x53, 0x54, 0x55, 0x56, 0x57, 0x58, 0x59, 0x5a, 0x5b, 0x5c, 0x5d,
+0x5e, 0x5f, 0x60, 0x61, 0x62, 0x63, 0x64, 0x65, 0x66, 0x67, 0x68, 0x69, 0x6a,
+0x6b, 0x6c, 0x6d, 0x6e, 0x6f, 0x70, 0x71, 0x72, 0x73, 0x74, 0x75, 0x76, 0x77,
+0x78, 0x79, 0x7a, 0x7b, 0x7c, 0x7d, 0x7e, 0x7f, 0x80, 0x80, 0x81, 0x82, 0x83,
+0x84, 0x85, 0x86, 0x87, 0x88, 0x89, 0x8a, 0x8b, 0x8c, 0x8d, 0x8e, 0x8f, 0x90,
+0x91, 0x92, 0x93, 0x94, 0x95, 0x96, 0x97, 0x98, 0x99, 0x9a, 0x9b, 0x9c, 0x9d,
+0x9e, 0x9f, 0xa0, 0xa1, 0xa2, 0xa3, 0xa4, 0xa5, 0xa6, 0xa7, 0xa8, 0xa9, 0xaa,
+0xab, 0xac, 0xad, 0xae, 0xaf, 0xb0, 0xb1, 0xb2, 0xb3, 0xb4, 0xb5, 0xb6, 0xb7,
+0xb8, 0xb9, 0xba, 0xbb, 0xbc, 0xbd, 0xbe, 0xbf, 0xc0, 0xc1, 0xc2, 0xc3, 0xc4,
+0xc5, 0xc6, 0xc7, 0xc8, 0xc9, 0xca, 0xcb, 0xcc, 0xcd, 0xce, 0xcf, 0xd0, 0xd1,
+0xd2, 0xd3, 0xd4, 0xd5, 0xd6, 0xd7, 0xd8, 0xd9, 0xda, 0xdb, 0xdc, 0xdd, 0xde,
+0xdf, 0xe0, 0xe1, 0xe2, 0xe3, 0xe4, 0xe5, 0xe6, 0xe7, 0xe8, 0xe9, 0xea, 0xeb,
+0xec, 0xed, 0xee, 0xef, 0xf0, 0xf1, 0xf2, 0xf3, 0xf4, 0xf5, 0xf6, 0xf7, 0xf8,
+0xf9, 0xfa, 0xfb, 0xfc, 0xfd, 0xfe, 0xff, 0x80,  0x0,  0x1,  0x2,  0x3,  0x4,
+ 0x5,  0x6,  0x7,  0x8,  0x9,  0xa,  0xb,  0xc,  0xd,  0xe,  0xf, 0x10, 0x11,
+0x12, 0x13, 0x14, 0x15, 0x16, 0x17, 0x18, 0x19, 0x1a, 0x1b, 0x1c, 0x1d, 0x1e,
+0x1f, 0x20, 0x21, 0x22, 0x23, 0x24, 0x25, 0x26, 0x27, 0x28, 0x29, 0x2a, 0x2b,
+0x2c, 0x2d, 0x2e, 0x2f, 0x30, 0x31, 0x32, 0x33, 0x34, 0x35, 0x36, 0x37, 0x38,
+0x39, 0x3a, 0x3b, 0x3c, 0x3d, 0x3e, 0x3f, 0x40, 0x41, 0x42, 0x43, 0x44, 0x45,
+0x46, 0x47, 0x48, 0x49, 0x4a, 0x4b, 0x4c, 0x4d, 0x4e, 0x4f, 0x50, 0x51, 0x52,
+0x53, 0x54, 0x55, 0x56, 0x57, 0x58, 0x59, 0x5a, 0x5b, 0x5c, 0x5d, 0x5e, 0x5f,
+0x60, 0x61, 0x62, 0x63, 0x64, 0x65, 0x66, 0x67, 0x68, 0x69, 0x6a, 0x6b, 0x6c,
+0x6d, 0x6e, 0x6f, 0x70, 0x71, 0x72, 0x73, 0x74, 0x75, 0x76, 0x77, 0x78, 0x79,
+0x7a, 0x7b, 0x7c, 0x7d, 0x7e, 0x7f, 0x80, 0x80, 0x81, 0x82, 0x83, 0x84, 0x85,
+0x86, 0x87, 0x88, 0x89, 0x8a, 0x8b, 0x8c, 0x8d, 0x8e, 0x8f, 0x90, 0x91, 0x92,
+0x93, 0x94, 0x95, 0x96, 0x97, 0x98, 0x99, 0x9a, 0x9b, 0x9c, 0x9d, 0x9e, 0x9f,
+0xa0, 0xa1, 0xa2, 0xa3, 0xa4, 0xa5, 0xa6, 0xa7, 0xa8, 0xa9, 0xaa, 0xab, 0xac,
+0xad, 0xae, 0xaf, 0xb0, 0xb1, 0xb2, 0xb3, 0xb4, 0xb5, 0xb6, 0xb7, 0xb8, 0xb9,
+0xba, 0xbb, 0xbc, 0xbd, 0xbe, 0xbf, 0xc0, 0xc1, 0xc2, 0xc3, 0xc4, 0xc5, 0xc6,
+0xc7, 0xc8, 0xc9, 0xca, 0xcb, 0xcc, 0xcd, 0xce, 0xcf, 0xd0, 0xd1, 0xd2, 0xd3,
+0xd4, 0xd5, 0xd6, 0xd7, 0xd8, 0xd9, 0xda, 0xdb, 0xdc, 0xdd, 0xde, 0xdf, 0xe0,
+0xe1, 0xe2, 0xe3, 0xe4, 0xe5, 0xe6, 0xe7, 0xe8, 0xe9, 0xea, 0xeb, 0xec, 0xed,
+0xee, 0xef, 0xf0, 0xf1, 0xf2, 0xf3, 0xf4, 0xf5, 0xf6, 0xf7, 0xf8, 0xf9, 0xfa,
+0xfb, 0xfc, 0xfd, 0xfe, 0xff, 0x80,  0x0,  0x1,  0x2,  0x3,  0x4,  0x5,  0x6,
+ 0x7,  0x8,  0x9,  0xa,  0xb,  0xc,  0xd,  0xe,  0xf, 0x10, 0x11, 0x12, 0x13,
+0x14, 0x15, 0x16, 0x17, 0x18, 0x19, 0x1a, 0x1b, 0x1c, 0x1d, 0x1e, 0x1f, 0x20,
+0x21, 0x22, 0x23, 0x24, 0x25, 0x26, 0x27, 0x28, 0x29, 0x2a, 0x2b, 0x2c, 0x2d,
+0x2e, 0x2f, 0x30, 0x31, 0x32, 0x33, 0x34, 0x35, 0x36, 0x37, 0x38, 0x39, 0x3a,
+0x3b, 0x3c, 0x3d, 0x3e, 0x3f, 0x40, 0x41, 0x42, 0x43, 0x44, 0x45, 0x46, 0x47,
+0x48, 0x49, 0x4a, 0x4b, 0x4c, 0x4d, 0x4e, 0x4f, 0x50, 0x51, 0x52, 0x53, 0x54,
+0x55, 0x56, 0x57, 0x58, 0x59, 0x5a, 0x5b, 0x5c, 0x5d, 0x5e, 0x5f, 0x60, 0x61,
+0x62, 0x63, 0x64, 0x65, 0x66, 0x67, 0x68, 0x69, 0x6a, 0x6b, 0x6c, 0x6d, 0x6e,
+0x6f, 0x70, 0x71, 0x72, 0x73, 0x74, 0x75, 0x76, 0x77, 0x78, 0x79, 0x7a, 0x7b,
+0x7c, 0x7d, 0x7e, 0x7f, 0x80, 0x80, 0x81, 0x82, 0x83, 0x84, 0x85, 0x86, 0x87,
+0x88, 0x89, 0x8a, 0x8b, 0x8c, 0x8d, 0x8e, 0x8f, 0x90, 0x91, 0x92, 0x93, 0x94,
+0x95, 0x96, 0x97, 0x98, 0x99, 0x9a, 0x9b, 0x9c, 0x9d, 0x9e, 0x9f, 0xa0, 0xa1,
+0xa2, 0xa3, 0xa4, 0xa5, 0xa6, 0xa7, 0xa8, 0xa9, 0xaa, 0xab, 0xac, 0xad, 0xae,
+0xaf, 0xb0, 0xb1, 0xb2, 0xb3, 0xb4, 0xb5, 0xb6, 0xb7, 0xb8, 0xb9, 0xba, 0xbb,
+0xbc, 0xbd, 0xbe, 0xbf, 0xc0, 0xc1, 0xc2, 0xc3, 0xc4, 0xc5, 0xc6, 0xc7, 0xc8,
+0xc9, 0xca, 0xcb, 0xcc, 0xcd, 0xce, 0xcf, 0xd0, 0xd1, 0xd2, 0xd3, 0xd4, 0xd5,
+0xd6, 0xd7, 0xd8, 0xd9, 0xda, 0xdb, 0xdc, 0xdd, 0xde, 0xdf, 0xe0, 0xe1, 0xe2,
+0xe3, 0xe4, 0xe5, 0xe6, 0xe7, 0xe8, 0xe9, 0xea, 0xeb, 0xec, 0xed, 0xee, 0xef,
+0xf0, 0xf1, 0xf2, 0xf3, 0xf4, 0xf5, 0xf6, 0xf7, 0xf8, 0xf9, 0xfa, 0xfb, 0xfc,
+0xfd, 0xfe, 0xff,
+};
+  SeekableInputStream* const stream =
+    new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer));
+  std::unique_ptr<ByteRleDecoder> rle =
+      createByteRleDecoder(std::unique_ptr<orc::SeekableInputStream>(stream));
+  std::vector<char> data(1);
+  for (size_t i = 0; i < 2048; i += 10) {
+    rle->next(data.data(), data.size(), nullptr);
+    EXPECT_EQ(static_cast<char>(i < 1024 ? i/16 : i & 0xff),
+              data[0])
+        << "Output wrong at " << i;
+    if (i < 2038) {
+      rle->skip(9);
+    }
+    rle->skip(0);
+  }
+}
+
+TEST(ByteRle, testSeek) {
+  // the stream generated by Java's
+  // TestRunLengthByteReader.testUncompressedSeek
+  // for (int i = 0; i < 2048; ++i) {
+  //   if (i < 1024) {
+  //     out.write(i / 4);
+  //   } else {
+  //     out.write(i % 256);
+  //   }
+  // }
+  const unsigned char buffer[] = {
+ 0x1,  0x0,  0x1,  0x1,  0x1,  0x2,  0x1,  0x3,  0x1,  0x4,  0x1,  0x5,  0x1,
+ 0x6,  0x1,  0x7,  0x1,  0x8,  0x1,  0x9,  0x1,  0xa,  0x1,  0xb,  0x1,  0xc,
+ 0x1,  0xd,  0x1,  0xe,  0x1,  0xf,  0x1, 0x10,  0x1, 0x11,  0x1, 0x12,  0x1,
+0x13,  0x1, 0x14,  0x1, 0x15,  0x1, 0x16,  0x1, 0x17,  0x1, 0x18,  0x1, 0x19,
+ 0x1, 0x1a,  0x1, 0x1b,  0x1, 0x1c,  0x1, 0x1d,  0x1, 0x1e,  0x1, 0x1f,  0x1,
+0x20,  0x1, 0x21,  0x1, 0x22,  0x1, 0x23,  0x1, 0x24,  0x1, 0x25,  0x1, 0x26,
+ 0x1, 0x27,  0x1, 0x28,  0x1, 0x29,  0x1, 0x2a,  0x1, 0x2b,  0x1, 0x2c,  0x1,
+0x2d,  0x1, 0x2e,  0x1, 0x2f,  0x1, 0x30,  0x1, 0x31,  0x1, 0x32,  0x1, 0x33,
+ 0x1, 0x34,  0x1, 0x35,  0x1, 0x36,  0x1, 0x37,  0x1, 0x38,  0x1, 0x39,  0x1,
+0x3a,  0x1, 0x3b,  0x1, 0x3c,  0x1, 0x3d,  0x1, 0x3e,  0x1, 0x3f,  0x1, 0x40,
+ 0x1, 0x41,  0x1, 0x42,  0x1, 0x43,  0x1, 0x44,  0x1, 0x45,  0x1, 0x46,  0x1,
+0x47,  0x1, 0x48,  0x1, 0x49,  0x1, 0x4a,  0x1, 0x4b,  0x1, 0x4c,  0x1, 0x4d,
+ 0x1, 0x4e,  0x1, 0x4f,  0x1, 0x50,  0x1, 0x51,  0x1, 0x52,  0x1, 0x53,  0x1,
+0x54,  0x1, 0x55,  0x1, 0x56,  0x1, 0x57,  0x1, 0x58,  0x1, 0x59,  0x1, 0x5a,
+ 0x1, 0x5b,  0x1, 0x5c,  0x1, 0x5d,  0x1, 0x5e,  0x1, 0x5f,  0x1, 0x60,  0x1,
+0x61,  0x1, 0x62,  0x1, 0x63,  0x1, 0x64,  0x1, 0x65,  0x1, 0x66,  0x1, 0x67,
+ 0x1, 0x68,  0x1, 0x69,  0x1, 0x6a,  0x1, 0x6b,  0x1, 0x6c,  0x1, 0x6d,  0x1,
+0x6e,  0x1, 0x6f,  0x1, 0x70,  0x1, 0x71,  0x1, 0x72,  0x1, 0x73,  0x1, 0x74,
+ 0x1, 0x75,  0x1, 0x76,  0x1, 0x77,  0x1, 0x78,  0x1, 0x79,  0x1, 0x7a,  0x1,
+0x7b,  0x1, 0x7c,  0x1, 0x7d,  0x1, 0x7e,  0x1, 0x7f,  0x1, 0x80,  0x1, 0x81,
+ 0x1, 0x82,  0x1, 0x83,  0x1, 0x84,  0x1, 0x85,  0x1, 0x86,  0x1, 0x87,  0x1,
+0x88,  0x1, 0x89,  0x1, 0x8a,  0x1, 0x8b,  0x1, 0x8c,  0x1, 0x8d,  0x1, 0x8e,
+ 0x1, 0x8f,  0x1, 0x90,  0x1, 0x91,  0x1, 0x92,  0x1, 0x93,  0x1, 0x94,  0x1,
+0x95,  0x1, 0x96,  0x1, 0x97,  0x1, 0x98,  0x1, 0x99,  0x1, 0x9a,  0x1, 0x9b,
+ 0x1, 0x9c,  0x1, 0x9d,  0x1, 0x9e,  0x1, 0x9f,  0x1, 0xa0,  0x1, 0xa1,  0x1,
+0xa2,  0x1, 0xa3,  0x1, 0xa4,  0x1, 0xa5,  0x1, 0xa6,  0x1, 0xa7,  0x1, 0xa8,
+ 0x1, 0xa9,  0x1, 0xaa,  0x1, 0xab,  0x1, 0xac,  0x1, 0xad,  0x1, 0xae,  0x1,
+0xaf,  0x1, 0xb0,  0x1, 0xb1,  0x1, 0xb2,  0x1, 0xb3,  0x1, 0xb4,  0x1, 0xb5,
+ 0x1, 0xb6,  0x1, 0xb7,  0x1, 0xb8,  0x1, 0xb9,  0x1, 0xba,  0x1, 0xbb,  0x1,
+0xbc,  0x1, 0xbd,  0x1, 0xbe,  0x1, 0xbf,  0x1, 0xc0,  0x1, 0xc1,  0x1, 0xc2,
+ 0x1, 0xc3,  0x1, 0xc4,  0x1, 0xc5,  0x1, 0xc6,  0x1, 0xc7,  0x1, 0xc8,  0x1,
+0xc9,  0x1, 0xca,  0x1, 0xcb,  0x1, 0xcc,  0x1, 0xcd,  0x1, 0xce,  0x1, 0xcf,
+ 0x1, 0xd0,  0x1, 0xd1,  0x1, 0xd2,  0x1, 0xd3,  0x1, 0xd4,  0x1, 0xd5,  0x1,
+0xd6,  0x1, 0xd7,  0x1, 0xd8,  0x1, 0xd9,  0x1, 0xda,  0x1, 0xdb,  0x1, 0xdc,
+ 0x1, 0xdd,  0x1, 0xde,  0x1, 0xdf,  0x1, 0xe0,  0x1, 0xe1,  0x1, 0xe2,  0x1,
+0xe3,  0x1, 0xe4,  0x1, 0xe5,  0x1, 0xe6,  0x1, 0xe7,  0x1, 0xe8,  0x1, 0xe9,
+ 0x1, 0xea,  0x1, 0xeb,  0x1, 0xec,  0x1, 0xed,  0x1, 0xee,  0x1, 0xef,  0x1,
+0xf0,  0x1, 0xf1,  0x1, 0xf2,  0x1, 0xf3,  0x1, 0xf4,  0x1, 0xf5,  0x1, 0xf6,
+ 0x1, 0xf7,  0x1, 0xf8,  0x1, 0xf9,  0x1, 0xfa,  0x1, 0xfb,  0x1, 0xfc,  0x1,
+0xfd,  0x1, 0xfe,  0x1, 0xff, 0x80,  0x0,  0x1,  0x2,  0x3,  0x4,  0x5,  0x6,
+ 0x7,  0x8,  0x9,  0xa,  0xb,  0xc,  0xd,  0xe,  0xf, 0x10, 0x11, 0x12, 0x13,
+0x14, 0x15, 0x16, 0x17, 0x18, 0x19, 0x1a, 0x1b, 0x1c, 0x1d, 0x1e, 0x1f, 0x20,
+0x21, 0x22, 0x23, 0x24, 0x25, 0x26, 0x27, 0x28, 0x29, 0x2a, 0x2b, 0x2c, 0x2d,
+0x2e, 0x2f, 0x30, 0x31, 0x32, 0x33, 0x34, 0x35, 0x36, 0x37, 0x38, 0x39, 0x3a,
+0x3b, 0x3c, 0x3d, 0x3e, 0x3f, 0x40, 0x41, 0x42, 0x43, 0x44, 0x45, 0x46, 0x47,
+0x48, 0x49, 0x4a, 0x4b, 0x4c, 0x4d, 0x4e, 0x4f, 0x50, 0x51, 0x52, 0x53, 0x54,
+0x55, 0x56, 0x57, 0x58, 0x59, 0x5a, 0x5b, 0x5c, 0x5d, 0x5e, 0x5f, 0x60, 0x61,
+0x62, 0x63, 0x64, 0x65, 0x66, 0x67, 0x68, 0x69, 0x6a, 0x6b, 0x6c, 0x6d, 0x6e,
+0x6f, 0x70, 0x71, 0x72, 0x73, 0x74, 0x75, 0x76, 0x77, 0x78, 0x79, 0x7a, 0x7b,
+0x7c, 0x7d, 0x7e, 0x7f, 0x80, 0x80, 0x81, 0x82, 0x83, 0x84, 0x85, 0x86, 0x87,
+0x88, 0x89, 0x8a, 0x8b, 0x8c, 0x8d, 0x8e, 0x8f, 0x90, 0x91, 0x92, 0x93, 0x94,
+0x95, 0x96, 0x97, 0x98, 0x99, 0x9a, 0x9b, 0x9c, 0x9d, 0x9e, 0x9f, 0xa0, 0xa1,
+0xa2, 0xa3, 0xa4, 0xa5, 0xa6, 0xa7, 0xa8, 0xa9, 0xaa, 0xab, 0xac, 0xad, 0xae,
+0xaf, 0xb0, 0xb1, 0xb2, 0xb3, 0xb4, 0xb5, 0xb6, 0xb7, 0xb8, 0xb9, 0xba, 0xbb,
+0xbc, 0xbd, 0xbe, 0xbf, 0xc0, 0xc1, 0xc2, 0xc3, 0xc4, 0xc5, 0xc6, 0xc7, 0xc8,
+0xc9, 0xca, 0xcb, 0xcc, 0xcd, 0xce, 0xcf, 0xd0, 0xd1, 0xd2, 0xd3, 0xd4, 0xd5,
+0xd6, 0xd7, 0xd8, 0xd9, 0xda, 0xdb, 0xdc, 0xdd, 0xde, 0xdf, 0xe0, 0xe1, 0xe2,
+0xe3, 0xe4, 0xe5, 0xe6, 0xe7, 0xe8, 0xe9, 0xea, 0xeb, 0xec, 0xed, 0xee, 0xef,
+0xf0, 0xf1, 0xf2, 0xf3, 0xf4, 0xf5, 0xf6, 0xf7, 0xf8, 0xf9, 0xfa, 0xfb, 0xfc,
+0xfd, 0xfe, 0xff, 0x80,  0x0,  0x1,  0x2,  0x3,  0x4,  0x5,  0x6,  0x7,  0x8,
+ 0x9,  0xa,  0xb,  0xc,  0xd,  0xe,  0xf, 0x10, 0x11, 0x12, 0x13, 0x14, 0x15,
+0x16, 0x17, 0x18, 0x19, 0x1a, 0x1b, 0x1c, 0x1d, 0x1e, 0x1f, 0x20, 0x21, 0x22,
+0x23, 0x24, 0x25, 0x26, 0x27, 0x28, 0x29, 0x2a, 0x2b, 0x2c, 0x2d, 0x2e, 0x2f,
+0x30, 0x31, 0x32, 0x33, 0x34, 0x35, 0x36, 0x37, 0x38, 0x39, 0x3a, 0x3b, 0x3c,
+0x3d, 0x3e, 0x3f, 0x40, 0x41, 0x42, 0x43, 0x44, 0x45, 0x46, 0x47, 0x48, 0x49,
+0x4a, 0x4b, 0x4c, 0x4d, 0x4e, 0x4f, 0x50, 0x51, 0x52, 0x53, 0x54, 0x55, 0x56,
+0x57, 0x58, 0x59, 0x5a, 0x5b, 0x5c, 0x5d, 0x5e, 0x5f, 0x60, 0x61, 0x62, 0x63,
+0x64, 0x65, 0x66, 0x67, 0x68, 0x69, 0x6a, 0x6b, 0x6c, 0x6d, 0x6e, 0x6f, 0x70,
+0x71, 0x72, 0x73, 0x74, 0x75, 0x76, 0x77, 0x78, 0x79, 0x7a, 0x7b, 0x7c, 0x7d,
+0x7e, 0x7f, 0x80, 0x80, 0x81, 0x82, 0x83, 0x84, 0x85, 0x86, 0x87, 0x88, 0x89,
+0x8a, 0x8b, 0x8c, 0x8d, 0x8e, 0x8f, 0x90, 0x91, 0x92, 0x93, 0x94, 0x95, 0x96,
+0x97, 0x98, 0x99, 0x9a, 0x9b, 0x9c, 0x9d, 0x9e, 0x9f, 0xa0, 0xa1, 0xa2, 0xa3,
+0xa4, 0xa5, 0xa6, 0xa7, 0xa8, 0xa9, 0xaa, 0xab, 0xac, 0xad, 0xae, 0xaf, 0xb0,
+0xb1, 0xb2, 0xb3, 0xb4, 0xb5, 0xb6, 0xb7, 0xb8, 0xb9, 0xba, 0xbb, 0xbc, 0xbd,
+0xbe, 0xbf, 0xc0, 0xc1, 0xc2, 0xc3, 0xc4, 0xc5, 0xc6, 0xc7, 0xc8, 0xc9, 0xca,
+0xcb, 0xcc, 0xcd, 0xce, 0xcf, 0xd0, 0xd1, 0xd2, 0xd3, 0xd4, 0xd5, 0xd6, 0xd7,
+0xd8, 0xd9, 0xda, 0xdb, 0xdc, 0xdd, 0xde, 0xdf, 0xe0, 0xe1, 0xe2, 0xe3, 0xe4,
+0xe5, 0xe6, 0xe7, 0xe8, 0xe9, 0xea, 0xeb, 0xec, 0xed, 0xee, 0xef, 0xf0, 0xf1,
+0xf2, 0xf3, 0xf4, 0xf5, 0xf6, 0xf7, 0xf8, 0xf9, 0xfa, 0xfb, 0xfc, 0xfd, 0xfe,
+0xff, 0x80,  0x0,  0x1,  0x2,  0x3,  0x4,  0x5,  0x6,  0x7,  0x8,  0x9,  0xa,
+ 0xb,  0xc,  0xd,  0xe,  0xf, 0x10, 0x11, 0x12, 0x13, 0x14, 0x15, 0x16, 0x17,
+0x18, 0x19, 0x1a, 0x1b, 0x1c, 0x1d, 0x1e, 0x1f, 0x20, 0x21, 0x22, 0x23, 0x24,
+0x25, 0x26, 0x27, 0x28, 0x29, 0x2a, 0x2b, 0x2c, 0x2d, 0x2e, 0x2f, 0x30, 0x31,
+0x32, 0x33, 0x34, 0x35, 0x36, 0x37, 0x38, 0x39, 0x3a, 0x3b, 0x3c, 0x3d, 0x3e,
+0x3f, 0x40, 0x41, 0x42, 0x43, 0x44, 0x45, 0x46, 0x47, 0x48, 0x49, 0x4a, 0x4b,
+0x4c, 0x4d, 0x4e, 0x4f, 0x50, 0x51, 0x52, 0x53, 0x54, 0x55, 0x56, 0x57, 0x58,
+0x59, 0x5a, 0x5b, 0x5c, 0x5d, 0x5e, 0x5f, 0x60, 0x61, 0x62, 0x63, 0x64, 0x65,
+0x66, 0x67, 0x68, 0x69, 0x6a, 0x6b, 0x6c, 0x6d, 0x6e, 0x6f, 0x70, 0x71, 0x72,
+0x73, 0x74, 0x75, 0x76, 0x77, 0x78, 0x79, 0x7a, 0x7b, 0x7c, 0x7d, 0x7e, 0x7f,
+0x80, 0x80, 0x81, 0x82, 0x83, 0x84, 0x85, 0x86, 0x87, 0x88, 0x89, 0x8a, 0x8b,
+0x8c, 0x8d, 0x8e, 0x8f, 0x90, 0x91, 0x92, 0x93, 0x94, 0x95, 0x96, 0x97, 0x98,
+0x99, 0x9a, 0x9b, 0x9c, 0x9d, 0x9e, 0x9f, 0xa0, 0xa1, 0xa2, 0xa3, 0xa4, 0xa5,
+0xa6, 0xa7, 0xa8, 0xa9, 0xaa, 0xab, 0xac, 0xad, 0xae, 0xaf, 0xb0, 0xb1, 0xb2,
+0xb3, 0xb4, 0xb5, 0xb6, 0xb7, 0xb8, 0xb9, 0xba, 0xbb, 0xbc, 0xbd, 0xbe, 0xbf,
+0xc0, 0xc1, 0xc2, 0xc3, 0xc4, 0xc5, 0xc6, 0xc7, 0xc8, 0xc9, 0xca, 0xcb, 0xcc,
+0xcd, 0xce, 0xcf, 0xd0, 0xd1, 0xd2, 0xd3, 0xd4, 0xd5, 0xd6, 0xd7, 0xd8, 0xd9,
+0xda, 0xdb, 0xdc, 0xdd, 0xde, 0xdf, 0xe0, 0xe1, 0xe2, 0xe3, 0xe4, 0xe5, 0xe6,
+0xe7, 0xe8, 0xe9, 0xea, 0xeb, 0xec, 0xed, 0xee, 0xef, 0xf0, 0xf1, 0xf2, 0xf3,
+0xf4, 0xf5, 0xf6, 0xf7, 0xf8, 0xf9, 0xfa, 0xfb, 0xfc, 0xfd, 0xfe, 0xff, 0x80,
+ 0x0,  0x1,  0x2,  0x3,  0x4,  0x5,  0x6,  0x7,  0x8,  0x9,  0xa,  0xb,  0xc,
+ 0xd,  0xe,  0xf, 0x10, 0x11, 0x12, 0x13, 0x14, 0x15, 0x16, 0x17, 0x18, 0x19,
+0x1a, 0x1b, 0x1c, 0x1d, 0x1e, 0x1f, 0x20, 0x21, 0x22, 0x23, 0x24, 0x25, 0x26,
+0x27, 0x28, 0x29, 0x2a, 0x2b, 0x2c, 0x2d, 0x2e, 0x2f, 0x30, 0x31, 0x32, 0x33,
+0x34, 0x35, 0x36, 0x37, 0x38, 0x39, 0x3a, 0x3b, 0x3c, 0x3d, 0x3e, 0x3f, 0x40,
+0x41, 0x42, 0x43, 0x44, 0x45, 0x46, 0x47, 0x48, 0x49, 0x4a, 0x4b, 0x4c, 0x4d,
+0x4e, 0x4f, 0x50, 0x51, 0x52, 0x53, 0x54, 0x55, 0x56, 0x57, 0x58, 0x59, 0x5a,
+0x5b, 0x5c, 0x5d, 0x5e, 0x5f, 0x60, 0x61, 0x62, 0x63, 0x64, 0x65, 0x66, 0x67,
+0x68, 0x69, 0x6a, 0x6b, 0x6c, 0x6d, 0x6e, 0x6f, 0x70, 0x71, 0x72, 0x73, 0x74,
+0x75, 0x76, 0x77, 0x78, 0x79, 0x7a, 0x7b, 0x7c, 0x7d, 0x7e, 0x7f, 0x80, 0x80,
+0x81, 0x82, 0x83, 0x84, 0x85, 0x86, 0x87, 0x88, 0x89, 0x8a, 0x8b, 0x8c, 0x8d,
+0x8e, 0x8f, 0x90, 0x91, 0x92, 0x93, 0x94, 0x95, 0x96, 0x97, 0x98, 0x99, 0x9a,
+0x9b, 0x9c, 0x9d, 0x9e, 0x9f, 0xa0, 0xa1, 0xa2, 0xa3, 0xa4, 0xa5, 0xa6, 0xa7,
+0xa8, 0xa9, 0xaa, 0xab, 0xac, 0xad, 0xae, 0xaf, 0xb0, 0xb1, 0xb2, 0xb3, 0xb4,
+0xb5, 0xb6, 0xb7, 0xb8, 0xb9, 0xba, 0xbb, 0xbc, 0xbd, 0xbe, 0xbf, 0xc0, 0xc1,
+0xc2, 0xc3, 0xc4, 0xc5, 0xc6, 0xc7, 0xc8, 0xc9, 0xca, 0xcb, 0xcc, 0xcd, 0xce,
+0xcf, 0xd0, 0xd1, 0xd2, 0xd3, 0xd4, 0xd5, 0xd6, 0xd7, 0xd8, 0xd9, 0xda, 0xdb,
+0xdc, 0xdd, 0xde, 0xdf, 0xe0, 0xe1, 0xe2, 0xe3, 0xe4, 0xe5, 0xe6, 0xe7, 0xe8,
+0xe9, 0xea, 0xeb, 0xec, 0xed, 0xee, 0xef, 0xf0, 0xf1, 0xf2, 0xf3, 0xf4, 0xf5,
+0xf6, 0xf7, 0xf8, 0xf9, 0xfa, 0xfb, 0xfc, 0xfd, 0xfe, 0xff,
+  };
+  std::unique_ptr<SeekableInputStream> stream
+    (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer)));
+  const uint64_t fileLocs[] = {
+   0,    0,    0,    0,    0,    2,    2,    2,    2,    4,    4,    4,    4,
+   6,    6,    6,    6,    8,    8,    8,    8,   10,   10,   10,   10,   12,
+  12,   12,   12,   14,   14,   14,   14,   16,   16,   16,   16,   18,   18,
+  18,   18,   20,   20,   20,   20,   22,   22,   22,   22,   24,   24,   24,
+  24,   26,   26,   26,   26,   28,   28,   28,   28,   30,   30,   30,   30,
+  32,   32,   32,   32,   34,   34,   34,   34,   36,   36,   36,   36,   38,
+  38,   38,   38,   40,   40,   40,   40,   42,   42,   42,   42,   44,   44,
+  44,   44,   46,   46,   46,   46,   48,   48,   48,   48,   50,   50,   50,
+  50,   52,   52,   52,   52,   54,   54,   54,   54,   56,   56,   56,   56,
+  58,   58,   58,   58,   60,   60,   60,   60,   62,   62,   62,   62,   64,
+  64,   64,   64,   66,   66,   66,   66,   68,   68,   68,   68,   70,   70,
+  70,   70,   72,   72,   72,   72,   74,   74,   74,   74,   76,   76,   76,
+  76,   78,   78,   78,   78,   80,   80,   80,   80,   82,   82,   82,   82,
+  84,   84,   84,   84,   86,   86,   86,   86,   88,   88,   88,   88,   90,
+  90,   90,   90,   92,   92,   92,   92,   94,   94,   94,   94,   96,   96,
+  96,   96,   98,   98,   98,   98,  100,  100,  100,  100,  102,  102,  102,
+ 102,  104,  104,  104,  104,  106,  106,  106,  106,  108,  108,  108,  108,
+ 110,  110,  110,  110,  112,  112,  112,  112,  114,  114,  114,  114,  116,
+ 116,  116,  116,  118,  118,  118,  118,  120,  120,  120,  120,  122,  122,
+ 122,  122,  124,  124,  124,  124,  126,  126,  126,  126,  128,  128,  128,
+ 128,  130,  130,  130,  130,  132,  132,  132,  132,  134,  134,  134,  134,
+ 136,  136,  136,  136,  138,  138,  138,  138,  140,  140,  140,  140,  142,
+ 142,  142,  142,  144,  144,  144,  144,  146,  146,  146,  146,  148,  148,
+ 148,  148,  150,  150,  150,  150,  152,  152,  152,  152,  154,  154,  154,
+ 154,  156,  156,  156,  156,  158,  158,  158,  158,  160,  160,  160,  160,
+ 162,  162,  162,  162,  164,  164,  164,  164,  166,  166,  166,  166,  168,
+ 168,  168,  168,  170,  170,  170,  170,  172,  172,  172,  172,  174,  174,
+ 174,  174,  176,  176,  176,  176,  178,  178,  178,  178,  180,  180,  180,
+ 180,  182,  182,  182,  182,  184,  184,  184,  184,  186,  186,  186,  186,
+ 188,  188,  188,  188,  190,  190,  190,  190,  192,  192,  192,  192,  194,
+ 194,  194,  194,  196,  196,  196,  196,  198,  198,  198,  198,  200,  200,
+ 200,  200,  202,  202,  202,  202,  204,  204,  204,  204,  206,  206,  206,
+ 206,  208,  208,  208,  208,  210,  210,  210,  210,  212,  212,  212,  212,
+ 214,  214,  214,  214,  216,  216,  216,  216,  218,  218,  218,  218,  220,
+ 220,  220,  220,  222,  222,  222,  222,  224,  224,  224,  224,  226,  226,
+ 226,  226,  228,  228,  228,  228,  230,  230,  230,  230,  232,  232,  232,
+ 232,  234,  234,  234,  234,  236,  236,  236,  236,  238,  238,  238,  238,
+ 240,  240,  240,  240,  242,  242,  242,  242,  244,  244,  244,  244,  246,
+ 246,  246,  246,  248,  248,  248,  248,  250,  250,  250,  250,  252,  252,
+ 252,  252,  254,  254,  254,  254,  256,  256,  256,  256,  258,  258,  258,
+ 258,  260,  260,  260,  260,  262,  262,  262,  262,  264,  264,  264,  264,
+ 266,  266,  266,  266,  268,  268,  268,  268,  270,  270,  270,  270,  272,
+ 272,  272,  272,  274,  274,  274,  274,  276,  276,  276,  276,  278,  278,
+ 278,  278,  280,  280,  280,  280,  282,  282,  282,  282,  284,  284,  284,
+ 284,  286,  286,  286,  286,  288,  288,  288,  288,  290,  290,  290,  290,
+ 292,  292,  292,  292,  294,  294,  294,  294,  296,  296,  296,  296,  298,
+ 298,  298,  298,  300,  300,  300,  300,  302,  302,  302,  302,  304,  304,
+ 304,  304,  306,  306,  306,  306,  308,  308,  308,  308,  310,  310,  310,
+ 310,  312,  312,  312,  312,  314,  314,  314,  314,  316,  316,  316,  316,
+ 318,  318,  318,  318,  320,  320,  320,  320,  322,  322,  322,  322,  324,
+ 324,  324,  324,  326,  326,  326,  326,  328,  328,  328,  328,  330,  330,
+ 330,  330,  332,  332,  332,  332,  334,  334,  334,  334,  336,  336,  336,
+ 336,  338,  338,  338,  338,  340,  340,  340,  340,  342,  342,  342,  342,
+ 344,  344,  344,  344,  346,  346,  346,  346,  348,  348,  348,  348,  350,
+ 350,  350,  350,  352,  352,  352,  352,  354,  354,  354,  354,  356,  356,
+ 356,  356,  358,  358,  358,  358,  360,  360,  360,  360,  362,  362,  362,
+ 362,  364,  364,  364,  364,  366,  366,  366,  366,  368,  368,  368,  368,
+ 370,  370,  370,  370,  372,  372,  372,  372,  374,  374,  374,  374,  376,
+ 376,  376,  376,  378,  378,  378,  378,  380,  380,  380,  380,  382,  382,
+ 382,  382,  384,  384,  384,  384,  386,  386,  386,  386,  388,  388,  388,
+ 388,  390,  390,  390,  390,  392,  392,  392,  392,  394,  394,  394,  394,
+ 396,  396,  396,  396,  398,  398,  398,  398,  400,  400,  400,  400,  402,
+ 402,  402,  402,  404,  404,  404,  404,  406,  406,  406,  406,  408,  408,
+ 408,  408,  410,  410,  410,  410,  412,  412,  412,  412,  414,  414,  414,
+ 414,  416,  416,  416,  416,  418,  418,  418,  418,  420,  420,  420,  420,
+ 422,  422,  422,  422,  424,  424,  424,  424,  426,  426,  426,  426,  428,
+ 428,  428,  428,  430,  430,  430,  430,  432,  432,  432,  432,  434,  434,
+ 434,  434,  436,  436,  436,  436,  438,  438,  438,  438,  440,  440,  440,
+ 440,  442,  442,  442,  442,  444,  444,  444,  444,  446,  446,  446,  446,
+ 448,  448,  448,  448,  450,  450,  450,  450,  452,  452,  452,  452,  454,
+ 454,  454,  454,  456,  456,  456,  456,  458,  458,  458,  458,  460,  460,
+ 460,  460,  462,  462,  462,  462,  464,  464,  464,  464,  466,  466,  466,
+ 466,  468,  468,  468,  468,  470,  470,  470,  470,  472,  472,  472,  472,
+ 474,  474,  474,  474,  476,  476,  476,  476,  478,  478,  478,  478,  480,
+ 480,  480,  480,  482,  482,  482,  482,  484,  484,  484,  484,  486,  486,
+ 486,  486,  488,  488,  488,  488,  490,  490,  490,  490,  492,  492,  492,
+ 492,  494,  494,  494,  494,  496,  496,  496,  496,  498,  498,  498,  498,
+ 500,  500,  500,  500,  502,  502,  502,  502,  504,  504,  504,  504,  506,
+ 506,  506,  506,  508,  508,  508,  508,  510,  510,  510,  510,  512,  512,
+ 512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,
+ 512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,
+ 512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,
+ 512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,
+ 512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,
+ 512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,
+ 512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,
+ 512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,
+ 512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,
+ 512,  512,  512,  512,  512,  512,  512,  512,  641,  641,  641,  641,  641,
+ 641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,
+ 641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,
+ 641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,
+ 641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,
+ 641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,
+ 641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,
+ 641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,
+ 641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,
+ 641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,
+ 641,  641,  641,  641,  641,  641,  770,  770,  770,  770,  770,  770,  770,
+ 770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,
+ 770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,
+ 770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,
+ 770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,
+ 770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,
+ 770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,
+ 770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,
+ 770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,
+ 770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,
+ 770,  770,  770,  770,  899,  899,  899,  899,  899,  899,  899,  899,  899,
+ 899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,
+ 899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,
+ 899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,
+ 899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,
+ 899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,
+ 899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,
+ 899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,
+ 899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,
+ 899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,
+ 899,  899, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
+1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
+1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
+1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
+1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
+1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
+1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
+1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
+1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
+1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
+1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157,
+1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157,
+1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157,
+1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157,
+1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157,
+1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157,
+1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157,
+1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157,
+1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157,
+1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1286, 1286,
+1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286,
+1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286,
+1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286,
+1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286,
+1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286,
+1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286,
+1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286,
+1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286,
+1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286,
+1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1415, 1415, 1415, 1415,
+1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415,
+1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415,
+1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415,
+1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415,
+1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415,
+1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415,
+1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415,
+1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415,
+1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415,
+1415, 1415, 1415, 1415, 1415, 1415, 1415,
+    };
+  const uint64_t rleLocs[] = {
+  0,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
+  4,   1,   2,   3,   4,   5,   6,   7,   8,   9,  10,  11,  12,  13,  14,  15,
+ 16,  17,  18,  19,  20,  21,  22,  23,  24,  25,  26,  27,  28,  29,  30,  31,
+ 32,  33,  34,  35,  36,  37,  38,  39,  40,  41,  42,  43,  44,  45,  46,  47,
+ 48,  49,  50,  51,  52,  53,  54,  55,  56,  57,  58,  59,  60,  61,  62,  63,
+ 64,  65,  66,  67,  68,  69,  70,  71,  72,  73,  74,  75,  76,  77,  78,  79,
+ 80,  81,  82,  83,  84,  85,  86,  87,  88,  89,  90,  91,  92,  93,  94,  95,
+ 96,  97,  98,  99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111,
+112, 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126, 127,
+  0,   1,   2,   3,   4,   5,   6,   7,   8,   9,  10,  11,  12,  13,  14,  15,
+ 16,  17,  18,  19,  20,  21,  22,  23,  24,  25,  26,  27,  28,  29,  30,  31,
+ 32,  33,  34,  35,  36,  37,  38,  39,  40,  41,  42,  43,  44,  45,  46,  47,
+ 48,  49,  50,  51,  52,  53,  54,  55,  56,  57,  58,  59,  60,  61,  62,  63,
+ 64,  65,  66,  67,  68,  69,  70,  71,  72,  73,  74,  75,  76,  77,  78,  79,
+ 80,  81,  82,  83,  84,  85,  86,  87,  88,  89,  90,  91,  92,  93,  94,  95,
+ 96,  97,  98,  99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111,
+112, 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126, 127,
+  0,   1,   2,   3,   4,   5,   6,   7,   8,   9,  10,  11,  12,  13,  14,  15,
+ 16,  17,  18,  19,  20,  21,  22,  23,  24,  25,  26,  27,  28,  29,  30,  31,
+ 32,  33,  34,  35,  36,  37,  38,  39,  40,  41,  42,  43,  44,  45,  46,  47,
+ 48,  49,  50,  51,  52,  53,  54,  55,  56,  57,  58,  59,  60,  61,  62,  63,
+ 64,  65,  66,  67,  68,  69,  70,  71,  72,  73,  74,  75,  76,  77,  78,  79,
+ 80,  81,  82,  83,  84,  85,  86,  87,  88,  89,  90,  91,  92,  93,  94,  95,
+ 96,  97,  98,  99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111,
+112, 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126, 127,
+  0,   1,   2,   3,   4,   5,   6,   7,   8,   9,  10,  11,  12,  13,  14,  15,
+ 16,  17,  18,  19,  20,  21,  22,  23,  24,  25,  26,  27,  28,  29,  30,  31,
+ 32,  33,  34,  35,  36,  37,  38,  39,  40,  41,  42,  43,  44,  45,  46,  47,
+ 48,  49,  50,  51,  52,  53,  54,  55,  56,  57,  58,  59,  60,  61,  62,  63,
+ 64,  65,  66,  67,  68,  69,  70,  71,  72,  73,  74,  75,  76,  77,  78,  79,
+ 80,  81,  82,  83,  84,  85,  86,  87,  88,  89,  90,  91,  92,  93,  94,  95,
+ 96,  97,  98,  99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111,
+112, 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126, 127,
+  0,   1,   2,   3,   4,   5,   6,   7,   8,   9,  10,  11,  12,  13,  14,  15,
+ 16,  17,  18,  19,  20,  21,  22,  23,  24,  25,  26,  27,  28,  29,  30,  31,
+ 32,  33,  34,  35,  36,  37,  38,  39,  40,  41,  42,  43,  44,  45,  46,  47,
+ 48,  49,  50,  51,  52,  53,  54,  55,  56,  57,  58,  59,  60,  61,  62,  63,
+ 64,  65,  66,  67,  68,  69,  70,  71,  72,  73,  74,  75,  76,  77,  78,  79,
+ 80,  81,  82,  83,  84,  85,  86,  87,  88,  89,  90,  91,  92,  93,  94,  95,
+ 96,  97,  98,  99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111,
+112, 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126, 127,
+  0,   1,   2,   3,   4,   5,   6,   7,   8,   9,  10,  11,  12,  13,  14,  15,
+ 16,  17,  18,  19,  20,  21,  22,  23,  24,  25,  26,  27,  28,  29,  30,  31,
+ 32,  33,  34,  35,  36,  37,  38,  39,  40,  41,  42,  43,  44,  45,  46,  47,
+ 48,  49,  50,  51,  52,  53,  54,  55,  56,  57,  58,  59,  60,  61,  62,  63,
+ 64,  65,  66,  67,  68,  69,  70,  71,  72,  73,  74,  75,  76,  77,  78,  79,
+ 80,  81,  82,  83,  84,  85,  86,  87,  88,  89,  90,  91,  92,  93,  94,  95,
+ 96,  97,  98,  99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111,
+112, 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126, 127,
+  0,   1,   2,   3,   4,   5,   6,   7,   8,   9,  10,  11,  12,  13,  14,  15,
+ 16,  17,  18,  19,  20,  21,  22,  23,  24,  25,  26,  27,  28,  29,  30,  31,
+ 32,  33,  34,  35,  36,  37,  38,  39,  40,  41,  42,  43,  44,  45,  46,  47,
+ 48,  49,  50,  51,  52,  53,  54,  55,  56,  57,  58,  59,  60,  61,  62,  63,
+ 64,  65,  66,  67,  68,  69,  70,  71,  72,  73,  74,  75,  76,  77,  78,  79,
+ 80,  81,  82,  83,  84,  85,  86,  87,  88,  89,  90,  91,  92,  93,  94,  95,
+ 96,  97,  98,  99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111,
+112, 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126, 127,
+  0,   1,   2,   3,   4,   5,   6,   7,   8,   9,  10,  11,  12,  13,  14,  15,
+ 16,  17,  18,  19,  20,  21,  22,  23,  24,  25,  26,  27,  28,  29,  30,  31,
+ 32,  33,  34,  35,  36,  37,  38,  39,  40,  41,  42,  43,  44,  45,  46,  47,
+ 48,  49,  50,  51,  52,  53,  54,  55,  56,  57,  58,  59,  60,  61,  62,  63,
+ 64,  65,  66,  67,  68,  69,  70,  71,  72,  73,  74,  75,  76,  77,  78,  79,
+ 80,  81,  82,  83,  84,  85,  86,  87,  88,  89,  90,  91,  92,  93,  94,  95,
+ 96,  97,  98,  99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111,
+112, 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126, 127,
+};
+  std::list<uint64_t> positions[2048];
+  for (size_t i = 0; i < 2048; ++i) {
+    positions[i].push_back(fileLocs[i]);
+    positions[i].push_back(rleLocs[i]);
+  }
+  std::unique_ptr<ByteRleDecoder> rle =
+      createByteRleDecoder(std::move(stream));
+  std::vector<char> data(1);
+  for (size_t i = 0; i < 2048; ++i) {
+    rle->next(data.data(), 1, nullptr);
+    EXPECT_EQ(static_cast<char>(i < 1024 ? i / 4 : i & 0xff),
+              data[0])
+        << "Output wrong at " << i;
+  }
+  size_t i = 2048;
+  do {
+    --i;
+    PositionProvider location(positions[i]);
+    rle->seek(location);
+    rle->next(data.data(), 1, nullptr);
+    EXPECT_EQ(static_cast<char>(i < 1024 ? i / 4 : i & 0xff),
+              data[0])
+        << "Output wrong at " << i;
+  } while (i != 0);
+}
+
+TEST(BooleanRle, simpleTest) {
+  const unsigned char buffer[] = {0x61, 0xf0, 0xfd, 0x55, 0xAA, 0x55};
+  std::unique_ptr<SeekableInputStream> stream
+    (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer)));
+  std::unique_ptr<ByteRleDecoder> rle =
+      createBooleanRleDecoder(std::move(stream));
+  std::vector<char> data(50);
+  for (size_t i = 0; i < 16; ++i) {
+    rle->next(data.data(), data.size(), nullptr);
+    for (size_t j = 0; j < data.size(); ++j) {
+      const int bitPosn = static_cast<int>(50 * i + j);
+      EXPECT_EQ((bitPosn & 0x4) == 0 ? 1 : 0,
+                data[j])
+          << "Output wrong at " << i << ", " << j;
+    }
+  }
+  rle->next(data.data(), 24, nullptr);
+  for (size_t i = 0; i < 3; ++i) {
+    for (size_t j = 0; j < 8; ++j) {
+      EXPECT_EQ((i % 2) == (j % 2) ? 0 : 1,
+                data[i * 8 + j])
+          << "Output wrong at " << i << "," << j;
+    }
+  }
+}
+
+TEST(BooleanRle, runsTest) {
+  const unsigned char buffer[] = {0xf7, 0xff, 0x80, 0x3f, 0xe0, 0x0f,
+				  0xf8, 0x03, 0xfe, 0x00};
+  std::unique_ptr<SeekableInputStream> stream
+    (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer)));
+  std::unique_ptr<ByteRleDecoder> rle =
+      createBooleanRleDecoder(std::move(stream));
+  std::vector<char> data(72);
+  rle->next(data.data(), data.size(), nullptr);
+  for (size_t i = 0; i < data.size(); ++i) {
+    EXPECT_EQ(i % 18 < 9 ? 1 : 0,
+              data[i])
+        << "Output wrong at " << i;
+  }
+  std::list<uint64_t> position(3, 0);
+  PositionProvider location(position);
+  rle->seek(location);
+  for (size_t i = 0; i < data.size(); ++i) {
+    rle->next(data.data(), 1, nullptr);
+    EXPECT_EQ(i % 18 < 9 ? 1 : 0,
+              data[0])
+        << "Output wrong at " << i;
+  }
+}
+
+TEST(BooleanRle, runsTestWithNull) {
+  const unsigned char buffer[] = {0xf7, 0xff, 0x80, 0x3f, 0xe0, 0x0f,
+				  0xf8, 0x03, 0xfe, 0x00};
+  std::unique_ptr<SeekableInputStream> stream
+    (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer)));
+  std::unique_ptr<ByteRleDecoder> rle =
+      createBooleanRleDecoder(std::move(stream));
+  std::vector<char> data(72);
+  std::vector<char> notNull(data.size(), 1);
+  rle->next(data.data(), data.size(), notNull.data());
+  for (size_t i = 0; i < data.size(); ++i) {
+    EXPECT_EQ(i % 18 < 9 ? 1 : 0,
+              data[i])
+        << "Output wrong at " << i;
+  }
+  std::list<uint64_t> position(3, 0);
+  PositionProvider location(position);
+  rle->seek(location);
+  for (size_t i = 0; i < data.size(); ++i) {
+    rle->next(data.data(), 1, notNull.data());
+    EXPECT_EQ(i % 18 < 9 ? 1 : 0,
+              data[0])
+        << "Output wrong at " << i;
+  }
+}
+
+TEST(BooleanRle, skipTest) {
+  // stream copied from Java's TestBitFieldReader.testSkips
+  // for i in 0..16383
+  //   if i < 8192
+  //     out.write(i & 1)
+  //   else
+  //     out.write((i / 3) & 1)
+  const unsigned char buffer[] = {
+0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f,
+0x55, 0x6f, 0x55, 0x80, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x80, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0x80, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x80, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x80,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0x80, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x80, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x80, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71
+  };
+  std::unique_ptr<SeekableInputStream> stream
+    (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer)));
+  std::unique_ptr<ByteRleDecoder> rle =
+      createBooleanRleDecoder(std::move(stream));
+  std::vector<char> data(1);
+  for (size_t i = 0; i < 16384; i += 5) {
+    rle->next(data.data(), data.size(), nullptr);
+    EXPECT_EQ(i < 8192 ? i & 1 : (i / 3) & 1,
+              data[0])
+        << "Output wrong at " << i;
+    if (i < 16379) {
+      rle->skip(4);
+    }
+    rle->skip(0);
+  }
+}
+
+TEST(BooleanRle, skipTestWithNulls) {
+  // stream copied from Java's TestBitFieldReader.testSkips
+  // for i in 0..16383
+  //   if i < 8192
+  //     out.write(i & 1)
+  //   else
+  //     out.write((i / 3) & 1)
+  const unsigned char buffer[] = {
+0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f,
+0x55, 0x6f, 0x55, 0x80, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x80, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0x80, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x80, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x80,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0x80, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x80, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x80, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71
+  };
+  std::unique_ptr<SeekableInputStream> stream
+    (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer)));
+  std::unique_ptr<ByteRleDecoder> rle =
+      createBooleanRleDecoder(std::move(stream));
+  std::vector<char> data(3);
+  std::vector<char> someNull(data.size(), 0);
+  someNull[1] = 1;
+  std::vector<char> allNull(data.size(), 0);
+  for (size_t i = 0; i < 16384; i += 5) {
+    data.assign(data.size(), -1);
+    rle->next(data.data(), data.size(), someNull.data());
+    EXPECT_EQ(0, data[0]) << "Output wrong at " << i;
+    EXPECT_EQ(0, data[2]) << "Output wrong at " << i;
+    EXPECT_EQ(i < 8192 ? i & 1 : (i / 3) & 1,
+              data[1])
+        << "Output wrong at " << i;
+    if (i < 16379) {
+      rle->skip(4);
+    }
+    rle->skip(0);
+    data.assign(data.size(), -1);;
+    rle->next(data.data(), data.size(), allNull.data());
+    for (size_t j = 0; j < data.size(); ++j) {
+      EXPECT_EQ(0, data[j]) << "Output wrong at " << i << ", " << j;
+    }
+  }
+}
+
+TEST(BooleanRle, seekTest) {
+  // stream copied from Java's TestBitFieldReader.testUncompressedSeek
+  // for i in 0..16383
+  //   if i < 8192
+  //     out.write(i & 1)
+  //   else
+  //     out.write((i / 3) & 1)
+  const unsigned char buffer[] = {
+0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f,
+0x55, 0x6f, 0x55, 0x80, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x80, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0x80, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x80, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x80,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0x80, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x80, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x80, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71
+  };
+  std::unique_ptr<SeekableInputStream> stream
+    (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer)));
+  std::unique_ptr<ByteRleDecoder> rle =
+      createBooleanRleDecoder(std::move(stream));
+  std::vector<char> data(16384);
+  rle->next(data.data(), data.size(), nullptr);
+  for (size_t i = 0; i < data.size(); ++i) {
+    EXPECT_EQ(i < 8192 ? i & 1 : (i / 3) & 1,
+              data[i])
+        << "Output wrong at " << i;
+  }
+  // set up all of the positions
+  std::list<uint64_t> positions[16384];
+  for (uint64_t i = 0; i < 16384; ++i) {
+    const uint64_t bytePosn = i / 8;
+    // add the stream position
+    positions[i].push_back(bytePosn < 1025
+                           ? 2 * (bytePosn / 130)
+                           : (bytePosn < 1152
+                              ? 16
+                              : 145 + 129 * ((bytePosn - 1152)/ 128)));
+    // add the byte RLE position
+    positions[i].push_back(bytePosn < 1025
+                           ? bytePosn % 130
+                           : (bytePosn - 1024) % 128);
+    // add the bit position
+    positions[i].push_back(i % 8);
+  }
+  size_t i = 16384;
+  do {
+    --i;
+    PositionProvider location(positions[i]);
+    rle->seek(location);
+    rle->next(data.data(), 1, nullptr);
+    EXPECT_EQ(i < 8192 ? i & 1 : (i / 3) & 1,
+              data[i])
+        << "Output wrong at " << i;
+  } while (i != 0);
+}
+
+TEST(BooleanRle, seekTestWithNulls) {
+  // stream copied from Java's TestBitFieldReader.testUncompressedSeek
+  // for i in 0..16383
+  //   if i < 8192
+  //     out.write(i & 1)
+  //   else
+  //     out.write((i / 3) & 1)
+  const unsigned char buffer[] = {
+0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f,
+0x55, 0x6f, 0x55, 0x80, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x80, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0x80, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x80, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x80,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0x80, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x80, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x80, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
+0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
+0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
+0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71
+};
+  std::unique_ptr<SeekableInputStream> stream
+    (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer)));
+  std::unique_ptr<ByteRleDecoder> rle =
+      createBooleanRleDecoder(std::move(stream));
+  std::vector<char> data(16384);
+  std::vector<char> allNull(data.size(), 0);
+  std::vector<char> noNull(data.size(), 1);
+  rle->next(data.data(), data.size(), allNull.data());
+  for (size_t i = 0; i < data.size(); ++i) {
+    EXPECT_EQ(0, data[i]) << "Output wrong at " << i;
+  }
+  rle->next(data.data(), data.size(), noNull.data());
+  for (size_t i = 0; i < data.size(); ++i) {
+    EXPECT_EQ(i < 8192 ? i & 1 : (i / 3) & 1,
+              data[i])
+        << "Output wrong at " << i;
+  }
+  // set up all of the positions
+  std::list<uint64_t> positions[16384];
+  for (uint64_t i = 0; i < 16384; ++i) {
+    const uint64_t bytePosn = i / 8;
+    // add the stream position
+    // add the stream position
+    positions[i].push_back(bytePosn < 1025
+                           ? 2 * (bytePosn / 130)
+                           : (bytePosn < 1152
+                              ? 16
+                              : 145 + 129 * ((bytePosn - 1152)/ 128)));
+    // add the byte RLE position
+    positions[i].push_back(bytePosn < 1025
+                           ? bytePosn % 130
+                           : (bytePosn - 1024) % 128);
+    // add the bit position
+    positions[i].push_back(i % 8);
+  }
+  size_t i = 16384;
+  do {
+    --i;
+    PositionProvider location(positions[i]);
+    rle->seek(location);
+    rle->next(data.data(), 1, noNull.data());
+    EXPECT_EQ(i < 8192 ? i & 1 : (i / 3) & 1,
+              data[i])
+        << "Output wrong at " << i;
+    data[0] = -1;
+    rle->next(data.data(), 1, allNull.data());
+    EXPECT_EQ(0, data[0]) << "Output wrong at " << i;
+  } while (i != 0);
+}
+
+}  // namespace orc


[07/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/test/orc/TestCompression.cc
----------------------------------------------------------------------
diff --git a/c++/test/orc/TestCompression.cc b/c++/test/orc/TestCompression.cc
deleted file mode 100644
index 4b4f13f..0000000
--- a/c++/test/orc/TestCompression.cc
+++ /dev/null
@@ -1,647 +0,0 @@
-/**
- * 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 "orc/Compression.hh"
-#include "orc/Exceptions.hh"
-#include "wrap/gtest-wrapper.h"
-#include "OrcTest.hh"
-
-#include <cstdio>
-#include <fstream>
-#include <iostream>
-#include <sstream>
-
-#include "zlib.h"
-#include "wrap/snappy-wrapper.h"
-
-namespace orc {
-
-  class TestCompression : public ::testing::Test {
-  public:
-    ~TestCompression();
-  protected:
-    // Per-test-case set-up.
-    static void SetUpTestCase() {
-      simpleFile = "simple-file.binary";
-      remove(simpleFile);
-      std::ofstream file;
-      file.exceptions(std::ofstream::failbit | std::ofstream::badbit);
-      file.open(simpleFile,
-                std::ios::out | std::ios::binary | std::ios::trunc);
-      for(unsigned int i = 0; i < 200; ++i) {
-        file.put(static_cast<char>(i));
-      }
-      file.close();
-    }
-
-    // Per-test-case tear-down.
-    static void TearDownTestCase() {
-      simpleFile = 0;
-    }
-
-    static const char *simpleFile;
-  };
-
-  const char *TestCompression::simpleFile;
-
-  TestCompression::~TestCompression() {
-    // PASS
-  }
-
-  TEST_F(TestCompression, testPrintBufferEmpty) {
-    std::ostringstream str;
-    printBuffer(str, 0, 0);
-    EXPECT_EQ("", str.str());
-  }
-
-  TEST_F(TestCompression, testPrintBufferSmall) {
-    std::vector<char> buffer(10);
-    std::ostringstream str;
-    for(size_t i=0; i < 10; ++i) {
-      buffer[i] = static_cast<char>(i);
-    }
-    printBuffer(str, buffer.data(), 10);
-    EXPECT_EQ("0000000 00 01 02 03 04 05 06 07 08 09\n", str.str());
-  }
-
-  TEST_F(TestCompression, testPrintBufferLong) {
-    std::vector<char> buffer(300);
-    std::ostringstream str;
-    for(size_t i=0; i < 300; ++i) {
-      buffer[i] = static_cast<char>(i);
-    }
-    printBuffer(str, buffer.data(), 300);
-    std::ostringstream expected;
-    expected << "0000000 00 01 02 03 04 05 06 07 08 09 0a 0b 0c 0d 0e 0f 10"
-             << " 11 12 13 14 15 16 17\n"
-             << "0000018 18 19 1a 1b 1c 1d 1e 1f 20 21 22 23 24 25 26 27 28"
-             << " 29 2a 2b 2c 2d 2e 2f\n"
-             << "0000030 30 31 32 33 34 35 36 37 38 39 3a 3b 3c 3d 3e 3f 40"
-             << " 41 42 43 44 45 46 47\n"
-             << "0000048 48 49 4a 4b 4c 4d 4e 4f 50 51 52 53 54 55 56 57 58"
-             << " 59 5a 5b 5c 5d 5e 5f\n"
-             << "0000060 60 61 62 63 64 65 66 67 68 69 6a 6b 6c 6d 6e 6f 70"
-             << " 71 72 73 74 75 76 77\n"
-             << "0000078 78 79 7a 7b 7c 7d 7e 7f 80 81 82 83 84 85 86 87 88"
-             << " 89 8a 8b 8c 8d 8e 8f\n"
-             << "0000090 90 91 92 93 94 95 96 97 98 99 9a 9b 9c 9d 9e 9f a0"
-             << " a1 a2 a3 a4 a5 a6 a7\n"
-             << "00000a8 a8 a9 aa ab ac ad ae af b0 b1 b2 b3 b4 b5 b6 b7 b8"
-             << " b9 ba bb bc bd be bf\n"
-             << "00000c0 c0 c1 c2 c3 c4 c5 c6 c7 c8 c9 ca cb cc cd ce cf d0"
-             << " d1 d2 d3 d4 d5 d6 d7\n"
-             << "00000d8 d8 d9 da db dc dd de df e0 e1 e2 e3 e4 e5 e6 e7 e8"
-             << " e9 ea eb ec ed ee ef\n"
-             << "00000f0 f0 f1 f2 f3 f4 f5 f6 f7 f8 f9 fa fb fc fd fe ff 00"
-             << " 01 02 03 04 05 06 07\n"
-             << "0000108 08 09 0a 0b 0c 0d 0e 0f 10 11 12 13 14 15 16 17 18"
-             << " 19 1a 1b 1c 1d 1e 1f\n"
-             << "0000120 20 21 22 23 24 25 26 27 28 29 2a 2b\n";
-    EXPECT_EQ(expected.str(), str.str());
-  }
-
-  TEST_F(TestCompression, testArrayBackup) {
-    std::vector<char> bytes(200);
-    for(size_t i=0; i < bytes.size(); ++i) {
-      bytes[i] = static_cast<char>(i);
-    }
-    SeekableArrayInputStream stream(bytes.data(), bytes.size(), 20);
-    const void *ptr;
-    int len;
-    ASSERT_THROW(stream.BackUp(10), std::logic_error);
-    EXPECT_EQ(true, stream.Next(&ptr, &len));
-    EXPECT_EQ(bytes.data(), static_cast<const char *>(ptr));
-    EXPECT_EQ(20, len);
-    stream.BackUp(0);
-    EXPECT_EQ(true, stream.Next(&ptr, &len));
-    EXPECT_EQ(bytes.data() + 20, static_cast<const char *>(ptr));
-    EXPECT_EQ(20, len);
-    stream.BackUp(10);
-    for(unsigned int i=0; i < 8; ++i) {
-      EXPECT_EQ(true, stream.Next(&ptr, &len));
-      unsigned int consumedBytes = 30 + 20 * i;
-      EXPECT_EQ(bytes.data() + consumedBytes, static_cast<const char *>(ptr));
-      EXPECT_EQ(consumedBytes + 20, stream.ByteCount());
-      EXPECT_EQ(20, len);
-    }
-    EXPECT_EQ(true, stream.Next(&ptr, &len));
-    EXPECT_EQ(bytes.data() + 190, static_cast<const char *>(ptr));
-    EXPECT_EQ(10, len);
-    EXPECT_EQ(true, !stream.Next(&ptr, &len));
-    EXPECT_EQ(0, len);
-    ASSERT_THROW(stream.BackUp(30), std::logic_error);
-    EXPECT_EQ(200, stream.ByteCount());
-  }
-
-  TEST_F(TestCompression, testArraySkip) {
-    std::vector<char> bytes(200);
-    for(size_t i=0; i < bytes.size(); ++i) {
-      bytes[i] = static_cast<char>(i);
-    }
-    SeekableArrayInputStream stream(bytes.data(), bytes.size(), 20);
-    const void *ptr;
-    int len;
-    ASSERT_EQ(true, stream.Next(&ptr, &len));
-    EXPECT_EQ(bytes.data(), static_cast<const char *>(ptr));
-    EXPECT_EQ(20, len);
-    ASSERT_EQ(true, !stream.Skip(-10));
-    ASSERT_EQ(true, stream.Skip(80));
-    ASSERT_EQ(true, stream.Next(&ptr, &len));
-    EXPECT_EQ(bytes.data() + 100, static_cast<const char *>(ptr));
-    EXPECT_EQ(20, len);
-    ASSERT_EQ(true, stream.Skip(80));
-    ASSERT_EQ(true, !stream.Next(&ptr, &len));
-    ASSERT_EQ(true, !stream.Skip(181));
-    EXPECT_EQ("SeekableArrayInputStream 200 of 200", stream.getName());
-  }
-
-  TEST_F(TestCompression, testArrayCombo) {
-    std::vector<char> bytes(200);
-    for(size_t i=0; i < bytes.size(); ++i) {
-      bytes[i] = static_cast<char>(i);
-    }
-    SeekableArrayInputStream stream(bytes.data(), bytes.size(), 20);
-    const void *ptr;
-    int len;
-    ASSERT_EQ(true, stream.Next(&ptr, &len));
-    EXPECT_EQ(bytes.data(), static_cast<const char *>(ptr));
-    EXPECT_EQ(20, len);
-    stream.BackUp(10);
-    EXPECT_EQ(10, stream.ByteCount());
-    stream.Skip(4);
-    EXPECT_EQ(14, stream.ByteCount());
-    ASSERT_EQ(true, stream.Next(&ptr, &len));
-    EXPECT_EQ(bytes.data() + 14, static_cast<const char *>(ptr));
-    EXPECT_EQ(true, !stream.Skip(320));
-    EXPECT_EQ(200, stream.ByteCount());
-    EXPECT_EQ(true, !stream.Next(&ptr, &len));
-  }
-
-  // this checks to make sure that a given set of bytes are ascending
-  void checkBytes(const char*data, int length,
-                  unsigned int startValue) {
-    for(unsigned int i=0; static_cast<int>(i) < length; ++i) {
-      EXPECT_EQ(startValue + i, static_cast<unsigned char>(data[i]))
-        << "Output wrong at " << startValue << " + " << i;
-    }
-  }
-
-  TEST_F(TestCompression, testFileBackup) {
-    SCOPED_TRACE("testFileBackup");
-    std::unique_ptr<InputStream> file = readLocalFile(simpleFile);
-    SeekableFileInputStream stream(file.get(), 0, 200, *getDefaultPool(), 20);
-    const void *ptr;
-    int len;
-    ASSERT_THROW(stream.BackUp(10), std::logic_error);
-    EXPECT_EQ(true, stream.Next(&ptr, &len));
-    EXPECT_EQ(20, len);
-    checkBytes(static_cast<const char*>(ptr), len, 0);
-    stream.BackUp(0);
-    EXPECT_EQ(true, stream.Next(&ptr, &len));
-    EXPECT_EQ(20, len);
-    checkBytes(static_cast<const char*>(ptr), len, 20);
-    stream.BackUp(10);
-    EXPECT_EQ(30, stream.ByteCount());
-    EXPECT_EQ(true, stream.Next(&ptr, &len));
-    EXPECT_EQ(10, len);
-    checkBytes(static_cast<const char*>(ptr), len, 30);
-    for(unsigned int i=0; i < 8; ++i) {
-      EXPECT_EQ(20 * i + 40, stream.ByteCount());
-      EXPECT_EQ(true, stream.Next(&ptr, &len));
-      EXPECT_EQ(20, len);
-      checkBytes(static_cast<const char*>(ptr), len, 20 * i + 40);
-    }
-    EXPECT_EQ(true, !stream.Next(&ptr, &len));
-    EXPECT_EQ(0, len);
-    ASSERT_THROW(stream.BackUp(30), std::logic_error);
-    EXPECT_EQ(200, stream.ByteCount());
-  }
-
-  TEST_F(TestCompression, testFileSkip) {
-    SCOPED_TRACE("testFileSkip");
-    std::unique_ptr<InputStream> file = readLocalFile(simpleFile);
-    SeekableFileInputStream stream(file.get(), 0, 200, *getDefaultPool(), 20);
-    const void *ptr;
-    int len;
-    ASSERT_EQ(true, stream.Next(&ptr, &len));
-    checkBytes(static_cast<const char*>(ptr), len, 0);
-    EXPECT_EQ(20, len);
-    ASSERT_EQ(true, !stream.Skip(-10));
-    ASSERT_EQ(true, stream.Skip(80));
-    ASSERT_EQ(true, stream.Next(&ptr, &len));
-    checkBytes(static_cast<const char*>(ptr), len, 100);
-    EXPECT_EQ(20, len);
-    ASSERT_EQ(true, !stream.Skip(80));
-    ASSERT_EQ(true, !stream.Next(&ptr, &len));
-    ASSERT_EQ(true, !stream.Skip(181));
-    EXPECT_EQ("simple-file.binary from 0 for 200", stream.getName());
-  }
-
-  TEST_F(TestCompression, testFileCombo) {
-    SCOPED_TRACE("testFileCombo");
-    std::unique_ptr<InputStream> file = readLocalFile(simpleFile);
-    SeekableFileInputStream stream(file.get(), 0, 200, *getDefaultPool(), 20);
-    const void *ptr;
-    int len;
-    ASSERT_EQ(true, stream.Next(&ptr, &len));
-    checkBytes(static_cast<const char*>(ptr), len, 0);
-    EXPECT_EQ(20, len);
-    stream.BackUp(10);
-    EXPECT_EQ(10, stream.ByteCount());
-    stream.Skip(4);
-    EXPECT_EQ(14, stream.ByteCount());
-    ASSERT_EQ(true, stream.Next(&ptr, &len));
-    checkBytes(static_cast<const char*>(ptr), len, 14);
-    EXPECT_EQ(true, !stream.Skip(320));
-    EXPECT_EQ(200, stream.ByteCount());
-    EXPECT_EQ(true, !stream.Next(&ptr, &len));
-  }
-
-  TEST_F(TestCompression, testFileSeek) {
-    SCOPED_TRACE("testFileSeek");
-    std::unique_ptr<InputStream> file = readLocalFile(simpleFile);
-    SeekableFileInputStream stream(file.get(), 0, 200, *getDefaultPool(), 20);
-    const void *ptr;
-    int len;
-    EXPECT_EQ(0, stream.ByteCount());
-    ASSERT_EQ(true, stream.Next(&ptr, &len));
-    checkBytes(static_cast<const char*>(ptr), len, 0);
-    EXPECT_EQ(20, len);
-    EXPECT_EQ(20, stream.ByteCount());
-    {
-      std::list<uint64_t> offsets(1, 100);
-      PositionProvider posn(offsets);
-      stream.seek(posn);
-    }
-    EXPECT_EQ(100, stream.ByteCount());
-    {
-      std::list<uint64_t> offsets(1, 5);
-      PositionProvider posn(offsets);
-      stream.seek(posn);
-    }
-    EXPECT_EQ(5, stream.ByteCount());
-    ASSERT_EQ(true, stream.Next(&ptr, &len));
-    checkBytes(static_cast<const char*>(ptr), len, 5);
-    EXPECT_EQ(20, len);
-    {
-      std::list<uint64_t> offsets(1, 201);
-      PositionProvider posn(offsets);
-      EXPECT_THROW(stream.seek(posn), std::logic_error);
-      EXPECT_EQ(200, stream.ByteCount());
-    }
-  }
-
-  TEST_F(TestCompression, testCreateNone) {
-    std::vector<char> bytes(10);
-    for(unsigned int i=0; i < bytes.size(); ++i) {
-      bytes[i] = static_cast<char>(i);
-    }
-    std::unique_ptr<SeekableInputStream> result =
-      createDecompressor(CompressionKind_NONE,
-                         std::unique_ptr<SeekableInputStream>
-                         (new SeekableArrayInputStream(bytes.data(),
-                                                       bytes.size())),
-                         32768,
-                         *getDefaultPool());
-    const void *ptr;
-    int length;
-    result->Next(&ptr, &length);
-    for(unsigned int i=0; i < bytes.size(); ++i) {
-      EXPECT_EQ(static_cast<char>(i), static_cast<const char*>(ptr)[i]);
-    }
-  }
-
-  TEST_F(TestCompression, testCreateLzo) {
-    const unsigned char buffer[] = {0};
-    EXPECT_THROW(createDecompressor(CompressionKind_LZO,
-                             std::unique_ptr<SeekableInputStream>
-                                    (new SeekableArrayInputStream(buffer, 0)),
-                                    32768, *getDefaultPool()),
-                 NotImplementedYet);
-  }
-
-  TEST(Zlib, testCreateZlib) {
-    const unsigned char buffer[] = {0x0b, 0x0, 0x0, 0x0, 0x1, 0x2, 0x3, 0x4};
-    std::unique_ptr<SeekableInputStream> result =
-      createDecompressor(CompressionKind_ZLIB,
-                         std::unique_ptr<SeekableInputStream>
-                         (new SeekableArrayInputStream
-                          (buffer, ARRAY_SIZE(buffer))),
-                         32768, *getDefaultPool());
-    EXPECT_EQ("zlib(SeekableArrayInputStream 0 of 8)", result->getName());
-    const void *ptr;
-    int length;
-    ASSERT_EQ(true, result->Next(&ptr, &length));
-    ASSERT_EQ(5, length);
-    for(unsigned int i=0; i < 5; ++i) {
-      EXPECT_EQ(static_cast<char>(i), static_cast<const char*>(ptr)[i]);
-    }
-    EXPECT_EQ("zlib(SeekableArrayInputStream 8 of 8)", result->getName());
-    EXPECT_EQ(5, result->ByteCount());
-    result->BackUp(3);
-    EXPECT_EQ(2, result->ByteCount());
-    ASSERT_EQ(true, result->Next(&ptr, &length));
-    ASSERT_EQ(3, length);
-    for(unsigned int i=0; i < 3; ++i) {
-      EXPECT_EQ(static_cast<char>(i+2), static_cast<const char*>(ptr)[i]);
-    }
-  }
-
-  TEST(Zlib, testLiteralBlocks) {
-    const unsigned char buffer[] = {0x19, 0x0, 0x0, 0x0, 0x1,
-				    0x2, 0x3, 0x4, 0x5, 0x6,
-				    0x7, 0x8, 0x9, 0xa, 0xb,
-				    0xb, 0x0, 0x0, 0xc, 0xd,
-				    0xe, 0xf, 0x10};
-    std::unique_ptr<SeekableInputStream> result =
-      createDecompressor(CompressionKind_ZLIB,
-                         std::unique_ptr<SeekableInputStream>
-                         (new SeekableArrayInputStream
-                          (buffer, ARRAY_SIZE(buffer), 5)),
-                         5, *getDefaultPool());
-    EXPECT_EQ("zlib(SeekableArrayInputStream 0 of 23)", result->getName());
-    const void *ptr;
-    int length;
-    ASSERT_EQ(true, result->Next(&ptr, &length));
-    ASSERT_EQ(2, length);
-    EXPECT_EQ(0, static_cast<const char*>(ptr)[0]);
-    EXPECT_EQ(1, static_cast<const char*>(ptr)[1]);
-    ASSERT_EQ(true, result->Next(&ptr, &length));
-    ASSERT_EQ(5, length);
-    EXPECT_EQ(2, static_cast<const char*>(ptr)[0]);
-    EXPECT_EQ(3, static_cast<const char*>(ptr)[1]);
-    EXPECT_EQ(4, static_cast<const char*>(ptr)[2]);
-    EXPECT_EQ(5, static_cast<const char*>(ptr)[3]);
-    EXPECT_EQ(6, static_cast<const char*>(ptr)[4]);
-    ASSERT_EQ(true, result->Next(&ptr, &length));
-    ASSERT_EQ(5, length);
-    EXPECT_EQ(7, static_cast<const char*>(ptr)[0]);
-    EXPECT_EQ(8, static_cast<const char*>(ptr)[1]);
-    EXPECT_EQ(9, static_cast<const char*>(ptr)[2]);
-    EXPECT_EQ(10, static_cast<const char*>(ptr)[3]);
-    EXPECT_EQ(11, static_cast<const char*>(ptr)[4]);
-    ASSERT_EQ(true, result->Next(&ptr, &length));
-    ASSERT_EQ(2, length);
-    EXPECT_EQ(12, static_cast<const char*>(ptr)[0]);
-    EXPECT_EQ(13, static_cast<const char*>(ptr)[1]);
-    ASSERT_EQ(true, result->Next(&ptr, &length));
-    ASSERT_EQ(3, length);
-    EXPECT_EQ(14, static_cast<const char*>(ptr)[0]);
-    EXPECT_EQ(15, static_cast<const char*>(ptr)[1]);
-    EXPECT_EQ(16, static_cast<const char*>(ptr)[2]);
-  }
-
-  TEST(Zlib, testInflate) {
-    const unsigned char buffer [] = {0xe, 0x0, 0x0, 0x63, 0x60, 0x64, 0x62,
-				     0xc0, 0x8d, 0x0};
-    std::unique_ptr<SeekableInputStream> result =
-      createDecompressor(CompressionKind_ZLIB,
-                         std::unique_ptr<SeekableInputStream>
-                         (new SeekableArrayInputStream
-                          (buffer, ARRAY_SIZE(buffer))), 1000,
-			 *getDefaultPool());
-    const void *ptr;
-    int length;
-    ASSERT_EQ(true, result->Next(&ptr, &length));
-    ASSERT_EQ(30, length);
-    for(int i=0; i < 10; ++i) {
-      for(int j=0; j < 3; ++j) {
-        EXPECT_EQ(j, static_cast<const char*>(ptr)[i * 3 + j]);
-      }
-    }
-  }
-
-  TEST(Zlib, testInflateSequence) {
-    const unsigned char buffer[] = {0xe, 0x0, 0x0, 0x63, 0x60,
-				    0x64, 0x62, 0xc0, 0x8d, 0x0,
-				    0xe, 0x0, 0x0, 0x63, 0x60,
-				    0x64, 0x62, 0xc0, 0x8d, 0x0};
-    std::unique_ptr<SeekableInputStream> result =
-      createDecompressor(CompressionKind_ZLIB,
-                         std::unique_ptr<SeekableInputStream>
-                         (new SeekableArrayInputStream
-                          (buffer, ARRAY_SIZE(buffer), 3)),
-                         1000,
-                         *getDefaultPool());
-    const void *ptr;
-    int length;
-    ASSERT_THROW(result->BackUp(20), std::logic_error);
-    ASSERT_EQ(true, result->Next(&ptr, &length));
-    ASSERT_EQ(30, length);
-    for(int i=0; i < 10; ++i) {
-      for(int j=0; j < 3; ++j) {
-        EXPECT_EQ(j, static_cast<const char*>(ptr)[i * 3 + j]);
-      }
-    }
-    result->BackUp(10);
-    ASSERT_THROW(result->BackUp(2), std::logic_error);
-    ASSERT_EQ(true, result->Next(&ptr, &length));
-    ASSERT_EQ(10, length);
-    for(int i=0; i < 10; ++i) {
-      EXPECT_EQ((i + 2) % 3, static_cast<const char*>(ptr)[i]);
-    }
-    ASSERT_EQ(true, result->Next(&ptr, &length));
-    ASSERT_EQ(30, length);
-    for(int i=0; i < 10; ++i) {
-      for(int j=0; j < 3; ++j) {
-        EXPECT_EQ(j, static_cast<const char*>(ptr)[i * 3 + j]);
-      }
-    }
-  }
-
-  TEST(Zlib, testSkip) {
-    const unsigned char buffer[] = {0x19, 0x0, 0x0, 0x0, 0x1,
-				    0x2, 0x3, 0x4, 0x5, 0x6,
-				    0x7, 0x8, 0x9, 0xa, 0xb,
-				    0xb, 0x0, 0x0, 0xc, 0xd,
-				    0xe, 0xf, 0x10};
-    std::unique_ptr<SeekableInputStream> result =
-      createDecompressor(CompressionKind_ZLIB,
-                         std::unique_ptr<SeekableInputStream>
-                         (new SeekableArrayInputStream
-                          (buffer, ARRAY_SIZE(buffer), 5)),
-                         5, *getDefaultPool());
-    const void *ptr;
-    int length;
-    ASSERT_EQ(true, result->Next(&ptr, &length));
-    ASSERT_EQ(2, length);
-    result->Skip(2);
-    ASSERT_EQ(true, result->Next(&ptr, &length));
-    ASSERT_EQ(3, length);
-    EXPECT_EQ(4, static_cast<const char*>(ptr)[0]);
-    EXPECT_EQ(5, static_cast<const char*>(ptr)[1]);
-    EXPECT_EQ(6, static_cast<const char*>(ptr)[2]);
-    result->BackUp(2);
-    ASSERT_EQ(true, result->Next(&ptr, &length));
-    ASSERT_EQ(2, length);
-    EXPECT_EQ(5, static_cast<const char*>(ptr)[0]);
-    EXPECT_EQ(6, static_cast<const char*>(ptr)[1]);
-    result->Skip(8);
-    ASSERT_EQ(true, result->Next(&ptr, &length));
-    ASSERT_EQ(2, length);
-    EXPECT_EQ(15, static_cast<const char*>(ptr)[0]);
-    EXPECT_EQ(16, static_cast<const char*>(ptr)[1]);
-  }
-
-#define HEADER_SIZE 3
-
-  class CompressBuffer {
-      std::vector<char> buf;
-
-  public:
-    CompressBuffer(size_t capacity) :
-      buf(capacity + HEADER_SIZE)
-    {}
-
-    char *getCompressed() {
-      return buf.data() + HEADER_SIZE;
-    }
-    char *getBuffer() {
-      return buf.data();
-    }
-
-    void writeHeader(size_t compressedSize) {
-      buf[0] = static_cast<char>(compressedSize << 1);
-      buf[1] = static_cast<char>(compressedSize >> 7);
-      buf[2] = static_cast<char>(compressedSize >> 15);
-    }
-
-    size_t getCompressedSize() const {
-      size_t header = static_cast<unsigned char>(buf[0]);
-      header |= static_cast<size_t>(static_cast<unsigned char>(buf[1])) << 8;
-      header |= static_cast<size_t>(static_cast<unsigned char>(buf[2])) << 16;
-      return header >> 1;
-    }
-
-    size_t getBufferSize() const {
-      return getCompressedSize() + HEADER_SIZE;
-    }
-  };
-
-  TEST(Snappy, testBasic) {
-    const int N = 1024;
-    std::vector<char> buf(N * sizeof(int));
-    for (int i=0; i < N; ++i) {
-      (reinterpret_cast<int *>(buf.data()))[i] = i % 8;
-    }
-
-    CompressBuffer compressBuffer(snappy::MaxCompressedLength(buf.size()));
-    size_t compressedSize;
-    snappy::RawCompress(buf.data(), buf.size(), compressBuffer.getCompressed(),
-                        &compressedSize);
-    // compressed size must be < original
-    ASSERT_LT(compressedSize, buf.size());
-    compressBuffer.writeHeader(compressedSize);
-
-    const long blockSize = 3;
-    std::unique_ptr<SeekableInputStream> result = createDecompressor
-        (CompressionKind_SNAPPY,
-         std::unique_ptr<SeekableInputStream>
-           (new SeekableArrayInputStream(compressBuffer.getBuffer(),
-                                         compressBuffer.getBufferSize(),
-                                         blockSize)),
-         buf.size(),
-         *getDefaultPool());
-    const void *data;
-    int length;
-    ASSERT_TRUE(result->Next(&data, &length));
-    ASSERT_EQ(N * sizeof(int), length);
-    for (int i=0; i < N; ++i) {
-      EXPECT_EQ(i % 8, (reinterpret_cast<const int *>(data))[i]);
-    }
-  }
-
-  TEST(Snappy, testMultiBuffer) {
-    const int N = 1024;
-    std::vector<char> buf(N * sizeof(int));
-    for (int i=0; i < N; ++i) {
-      (reinterpret_cast<int *>(buf.data()))[i] = i % 8;
-    }
-
-    CompressBuffer compressBuffer(snappy::MaxCompressedLength(buf.size()));
-    size_t compressedSize;
-    snappy::RawCompress(buf.data(), buf.size(), compressBuffer.getCompressed(),
-                        &compressedSize);
-    // compressed size must be < original
-    ASSERT_LT(compressedSize, buf.size());
-    compressBuffer.writeHeader(compressedSize);
-
-    std::vector<char> input(compressBuffer.getBufferSize() * 4);
-    ::memcpy(input.data(), compressBuffer.getBuffer(),
-             compressBuffer.getBufferSize());
-    ::memcpy(input.data() + compressBuffer.getBufferSize(),
-             compressBuffer.getBuffer(), compressBuffer.getBufferSize());
-    ::memcpy(input.data() + 2 * compressBuffer.getBufferSize(),
-             compressBuffer.getBuffer(), compressBuffer.getBufferSize());
-    ::memcpy(input.data() + 3 * compressBuffer.getBufferSize(),
-             compressBuffer.getBuffer(), compressBuffer.getBufferSize());
-
-    const long blockSize = 3;
-    std::unique_ptr<SeekableInputStream> result = createDecompressor
-        (CompressionKind_SNAPPY,
-         std::unique_ptr<SeekableInputStream>
-         (new SeekableArrayInputStream(input.data(), input.size(), blockSize)),
-         buf.size(),
-         *getDefaultPool());
-    for (int i=0; i < 4; ++i) {
-      const void *data;
-      int length;
-      ASSERT_TRUE(result->Next(&data, &length));
-      for (int j=0; j < N; ++j) {
-          EXPECT_EQ(j % 8, (reinterpret_cast<const int *>(data))[j]);
-      }
-    }
-  }
-
-  TEST(Snappy, testSkip) {
-    const int N = 1024;
-    std::vector<char> buf(N * sizeof(int));
-    for (int i=0; i < N; ++i) {
-      (reinterpret_cast<int *>(buf.data()))[i] = i % 8;
-    }
-
-    CompressBuffer compressBuffer(snappy::MaxCompressedLength(buf.size()));
-    size_t compressedSize;
-    snappy::RawCompress(buf.data(), buf.size(), compressBuffer.getCompressed(),
-                        &compressedSize);
-    // compressed size must be < original
-    ASSERT_LT(compressedSize, buf.size());
-    compressBuffer.writeHeader(compressedSize);
-
-    const long blockSize = 3;
-    std::unique_ptr<SeekableInputStream> result = createDecompressor
-        (CompressionKind_SNAPPY,
-         std::unique_ptr<SeekableInputStream>
-           (new SeekableArrayInputStream(compressBuffer.getBuffer(),
-                                         compressBuffer.getBufferSize(),
-                                         blockSize)),
-         buf.size(),
-         *getDefaultPool());
-    const void *data;
-    int length;
-    // skip 1/2; in 2 jumps
-    ASSERT_TRUE(result->Skip(static_cast<int>(((N / 2) - 2) * sizeof(int))));
-    ASSERT_TRUE(result->Skip(static_cast<int>(2 * sizeof(int))));
-    ASSERT_TRUE(result->Next(&data, &length));
-    ASSERT_EQ((N / 2) * sizeof(int), length);
-    for (int i=N/2; i < N; ++i) {
-      EXPECT_EQ(i % 8, (reinterpret_cast<const int *>(data))[i - N/2]);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/test/orc/TestDriver.cc
----------------------------------------------------------------------
diff --git a/c++/test/orc/TestDriver.cc b/c++/test/orc/TestDriver.cc
deleted file mode 100644
index a34ff07..0000000
--- a/c++/test/orc/TestDriver.cc
+++ /dev/null
@@ -1,33 +0,0 @@
-/**
- * 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 "orc/orc-config.hh"
-
-#include "wrap/orc-proto-wrapper.hh"
-#include "wrap/gtest-wrapper.h"
-
-#include <iostream>
-
-GTEST_API_ int main(int argc, char **argv) {
-  GOOGLE_PROTOBUF_VERIFY_VERSION;
-  std::cout << "ORC version: " << ORC_VERSION << "\n";
-  testing::InitGoogleTest(&argc, argv);
-  int result = RUN_ALL_TESTS();
-  return result;
-}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/test/orc/TestInt128.cc
----------------------------------------------------------------------
diff --git a/c++/test/orc/TestInt128.cc b/c++/test/orc/TestInt128.cc
deleted file mode 100644
index cc3b30f..0000000
--- a/c++/test/orc/TestInt128.cc
+++ /dev/null
@@ -1,587 +0,0 @@
-/**
- * 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 "orc/Int128.hh"
-
-#include "wrap/gtest-wrapper.h"
-#include "OrcTest.hh"
-
-#include <iostream>
-
-namespace orc {
-
-  TEST(Int128, simpleTest) {
-    Int128 x = 12;
-    Int128 y = 13;
-    x += y;
-    EXPECT_EQ(25, x.toLong());
-    EXPECT_EQ("0x00000000000000000000000000000019", x.toHexString());
-    y -= 1;
-    EXPECT_EQ("0x0000000000000000000000000000000c", y.toHexString());
-    EXPECT_EQ(12, y.toLong());
-    EXPECT_EQ(0, y.getHighBits());
-    EXPECT_EQ(12, y.getLowBits());
-    y -= 20;
-    EXPECT_EQ("0xfffffffffffffffffffffffffffffff8", y.toHexString());
-    EXPECT_EQ(-8, y.toLong());
-    EXPECT_EQ(-1, y.getHighBits());
-    EXPECT_EQ(static_cast<uint64_t>(-8), y.getLowBits());
-    Int128 z;
-    EXPECT_EQ(0, z.toLong());
-  }
-
-  TEST(Int128, testNegate) {
-    Int128 n = -1000000000000;
-    EXPECT_EQ("0xffffffffffffffffffffff172b5af000", n.toHexString());
-    n.negate();
-    EXPECT_EQ(1000000000000, n.toLong());
-    n.abs();
-    EXPECT_EQ(1000000000000, n.toLong());
-    n.negate();
-    EXPECT_EQ(-1000000000000, n.toLong());
-    n.abs();
-    EXPECT_EQ(1000000000000, n.toLong());
-
-    Int128 big(0x12345678, 0x9abcdef0);
-    EXPECT_EQ("0x0000000012345678000000009abcdef0", big.toHexString());
-    EXPECT_EQ(305419896, big.getHighBits());
-    EXPECT_EQ(2596069104, big.getLowBits());
-    big.negate();
-    EXPECT_EQ("0xffffffffedcba987ffffffff65432110", big.toHexString());
-    EXPECT_EQ(0xffffffffedcba987, big.getHighBits());
-    EXPECT_EQ(0xffffffff65432110, big.getLowBits());
-    big.negate();
-    EXPECT_EQ("0x0000000012345678000000009abcdef0", big.toHexString());
-    big.invert();
-    EXPECT_EQ("0xffffffffedcba987ffffffff6543210f", big.toHexString());
-    big.invert();
-    EXPECT_EQ("0x0000000012345678000000009abcdef0", big.toHexString());
-  }
-
-  TEST(Int128, testPlus) {
-    Int128 n(0x1000, 0xfffffffffffffff0);
-    EXPECT_EQ("0x0000000000001000fffffffffffffff0", n.toHexString());
-    n += 0x20;
-    EXPECT_EQ("0x00000000000010010000000000000010", n.toHexString());
-    n -= 0x20;
-    EXPECT_EQ("0x0000000000001000fffffffffffffff0", n.toHexString());
-    n += Int128(2,3);
-    EXPECT_EQ("0x0000000000001002fffffffffffffff3", n.toHexString());
-
-    Int128 x(static_cast<int64_t>(0xffffffffffffff00), 0x200);
-    EXPECT_EQ("0xffffffffffffff000000000000000200", x.toHexString());
-    x -= 0x300;
-    EXPECT_EQ("0xfffffffffffffeffffffffffffffff00", x.toHexString());
-    x -= 0x100;
-    EXPECT_EQ("0xfffffffffffffefffffffffffffffe00", x.toHexString());
-    x += 0x400;
-    EXPECT_EQ("0xffffffffffffff000000000000000200", x.toHexString());
-    x -= Int128(1,2);
-    EXPECT_EQ("0xfffffffffffffeff00000000000001fe", x.toHexString());
-  }
-
-  TEST(Int128, testLogic) {
-    Int128 n = Int128(0x00000000100000002, 0x0000000400000008);
-    n |= Int128(0x0000001000000020, 0x0000004000000080);
-    EXPECT_EQ("0x00000011000000220000004400000088", n.toHexString());
-    n =  Int128(0x0000111100002222, 0x0000333300004444);
-    n &= Int128(0x0000f00000000f00, 0x000000f00000000f);
-    EXPECT_EQ( "0x00001000000002000000003000000004", n.toHexString());
-  }
-
-  TEST(Int128, testShift) {
-    Int128 n(0x123456789abcdef0,0xfedcba9876543210);
-    EXPECT_EQ("0x123456789abcdef0fedcba9876543210", n.toHexString());
-    n <<= 0;
-    EXPECT_EQ("0x123456789abcdef0fedcba9876543210", n.toHexString());
-    n <<= 4;
-    EXPECT_EQ("0x23456789abcdef0fedcba98765432100", n.toHexString());
-    n <<= 8;
-    EXPECT_EQ("0x456789abcdef0fedcba9876543210000", n.toHexString());
-    n += 0x99;
-    EXPECT_EQ("0x456789abcdef0fedcba9876543210099", n.toHexString());
-    n <<= 64;
-    EXPECT_EQ("0xcba98765432100990000000000000000", n.toHexString());
-    n += 0x312;
-    EXPECT_EQ("0xcba98765432100990000000000000312", n.toHexString());
-    n <<= 120;
-    EXPECT_EQ("0x12000000000000000000000000000000", n.toHexString());
-    n += 0x411;
-    EXPECT_EQ("0x12000000000000000000000000000411", n.toHexString());
-    n <<= 128;
-    EXPECT_EQ(0, n.toLong());
-
-    n = Int128(0x123456789abcdef0,0xfedcba9876543210);
-    EXPECT_EQ("0x123456789abcdef0fedcba9876543210", n.toHexString());
-    n >>= 0;
-    EXPECT_EQ("0x123456789abcdef0fedcba9876543210", n.toHexString());
-    n >>= 4;
-    EXPECT_EQ("0x0123456789abcdef0fedcba987654321", n.toHexString());
-    n >>= 8;
-    EXPECT_EQ("0x000123456789abcdef0fedcba9876543", n.toHexString());
-    n += Int128(0x2400000000000000, 0x0);
-    EXPECT_EQ("0x240123456789abcdef0fedcba9876543", n.toHexString());
-    n >>= 64;
-    EXPECT_EQ("0x0000000000000000240123456789abcd", n.toHexString());
-    n += Int128(0x2400000000000000, 0x0);
-    EXPECT_EQ("0x2400000000000000240123456789abcd", n.toHexString());
-    n >>= 129;
-    EXPECT_EQ(0, n.toLong());
-    n = Int128(static_cast<int64_t>(0xfedcba0987654321),0x1234567890abcdef);
-    EXPECT_EQ("0xfedcba09876543211234567890abcdef", n.toHexString());
-    n >>= 64;
-    EXPECT_EQ("0xfffffffffffffffffedcba0987654321", n.toHexString());
-    n = Int128(static_cast<int64_t>(0xfedcba0987654321),0x1234567890abcdef);
-    n >>= 129;
-    EXPECT_EQ("0xffffffffffffffffffffffffffffffff", n.toHexString());
-    n = Int128(-1, 0xffffffffffffffff);
-    n >>= 4;
-    EXPECT_EQ("0x0fffffffffffffffffffffffffffffff", n.toHexString());
-    n = Int128(-0x100, 0xffffffffffffffff);
-    n >>= 68;
-    EXPECT_EQ("0xfffffffffffffffffffffffffffffff0", n.toHexString());
-  }
-
-  TEST(Int128, testCompare) {
-    Int128 x = 123;
-    EXPECT_EQ(Int128(123), x);
-    EXPECT_EQ(true, x == 123);
-    EXPECT_EQ(true, !(x == 124));
-    EXPECT_EQ(true, !(x == -124));
-    EXPECT_EQ(true, !(x == Int128(2, 123)));
-    EXPECT_EQ(true, !(x != 123));
-    EXPECT_EQ(true, x != -123);
-    EXPECT_EQ(true, x != 124);
-    EXPECT_EQ(true, x != Int128(-1, 123));
-    x = Int128(0x123, 0x456);
-    EXPECT_EQ(true, !(x < Int128(0x123, 0x455)));
-    EXPECT_EQ(true, !(x < Int128(0x123, 0x456)));
-    EXPECT_EQ(true, x < Int128(0x123, 0x457));
-    EXPECT_EQ(true, !(x < Int128(0x122, 0x456)));
-    EXPECT_EQ(true, x < Int128(0x124, 0x456));
-
-    EXPECT_EQ(true, !(x <= Int128(0x123, 0x455)));
-    EXPECT_EQ(true, x <= Int128(0x123, 0x456));
-    EXPECT_EQ(true, x <= Int128(0x123, 0x457));
-    EXPECT_EQ(true, !(x <= Int128(0x122, 0x456)));
-    EXPECT_EQ(true, x <= Int128(0x124, 0x456));
-
-    EXPECT_EQ(true, x > Int128(0x123, 0x455));
-    EXPECT_EQ(true, !(x > Int128(0x123, 0x456)));
-    EXPECT_EQ(true, !(x > Int128(0x123, 0x457)));
-    EXPECT_EQ(true, x > Int128(0x122, 0x456));
-    EXPECT_EQ(true, !(x > Int128(0x124, 0x456)));
-
-    EXPECT_EQ(true, x >= Int128(0x123, 0x455));
-    EXPECT_EQ(true, x >= Int128(0x123, 0x456));
-    EXPECT_EQ(true, !(x >= Int128(0x123, 0x457)));
-    EXPECT_EQ(true, x >= Int128(0x122, 0x456));
-    EXPECT_EQ(true, !(x >= Int128(0x124, 0x456)));
-
-    EXPECT_EQ(true, Int128(-3) < Int128(-2));
-    EXPECT_EQ(true, Int128(-3) < Int128(0));
-    EXPECT_EQ(true, Int128(-3) < Int128(3));
-    EXPECT_EQ(true, Int128(0) < Int128(5));
-    EXPECT_EQ(true, Int128::minimumValue() < 0);
-    EXPECT_EQ(true, Int128(0) < Int128::maximumValue());
-    EXPECT_EQ(true, Int128::minimumValue() < Int128::maximumValue());
-  }
-
-  TEST(Int128, testHash) {
-    EXPECT_EQ(0, Int128().hash());
-    EXPECT_EQ(0x123, Int128(0x123).hash());
-    EXPECT_EQ(0xc3c3c3c3,
-              Int128(0x0101010102020202, 0x4040404080808080).hash());
-    EXPECT_EQ(0x122, Int128(-0x123).hash());
-    EXPECT_EQ(0x12345678, Int128(0x1234567800000000, 0x0).hash());
-    EXPECT_EQ(0x12345678, Int128(0x12345678, 0x0).hash());
-    EXPECT_EQ(0x12345678, Int128(0x0, 0x1234567800000000).hash());
-    EXPECT_EQ(0x12345678, Int128(0x0, 0x12345678).hash());
-  }
-
-  TEST(Int128, testFitsInLong) {
-    EXPECT_EQ(true, Int128(0x0, 0x7fffffffffffffff).fitsInLong());
-    EXPECT_EQ(true, !Int128(0x0, 0x8000000000000000).fitsInLong());
-    EXPECT_EQ(true, !Int128(-1, 0x7fffffffffffffff).fitsInLong());
-    EXPECT_EQ(true, Int128(-1, 0x8000000000000000).fitsInLong());
-    EXPECT_EQ(true, !Int128(1, 0x8000000000000000).fitsInLong());
-    EXPECT_EQ(true, !Int128(1, 0x7fffffffffffffff).fitsInLong());
-    EXPECT_EQ(true, !Int128(-2, 0x8000000000000000).fitsInLong());
-    EXPECT_EQ(true, !Int128(-2, 0x7fffffffffffffff).fitsInLong());
-
-    EXPECT_EQ(0x7fffffffffffffff, Int128(0x0, 0x7fffffffffffffff).toLong());
-    EXPECT_THROW(Int128(1,1).toLong(), std::runtime_error);
-    EXPECT_EQ(0x8000000000000000, Int128(-1, 0x8000000000000000).toLong());
-  }
-
-  TEST(Int128, testMultiply) {
-    Int128 x = 2;
-    x *= 3;
-    EXPECT_EQ(6, x.toLong());
-    x *= -4;
-    EXPECT_EQ(-24, x.toLong());
-    x *= 5;
-    EXPECT_EQ(-120, x.toLong());
-    x *= -7;
-    EXPECT_EQ(840, x.toLong());
-    x = Int128(0x0123456776543210,0x1111222233334444);
-    x *= 2;
-    EXPECT_EQ(0x02468aceeca86420, x.getHighBits());
-    EXPECT_EQ(0x2222444466668888, x.getLowBits());
-
-    x = Int128(0x0534AB4C, 0x59D109ADF9892FCA);
-    x *= Int128(0, 0x9033b8c7a);
-    EXPECT_EQ("0x2eead9afd0c6e0e929c18da753113e44", x.toHexString());
-  }
-
-  TEST(Int128, testMultiplyInt) {
-    Int128 x = 2;
-    x *= 1;
-    EXPECT_EQ(2, x.toLong());
-    x *= 2;
-    EXPECT_EQ(4, x.toLong());
-
-    x = 5;
-    x *= 6432346;
-    EXPECT_EQ(6432346 * 5, x.toLong());
-
-    x = (1L << 62) + (3L << 34) + 3L;
-    x *= 96;
-    EXPECT_EQ("0x00000000000000180000048000000120", x.toHexString());
-
-    x = 1;
-    x <<= 126;
-    EXPECT_EQ("0x40000000000000000000000000000000", x.toHexString());
-    x *= 2;
-    EXPECT_EQ("0x80000000000000000000000000000000", x.toHexString());
-    x *= 2;
-    EXPECT_EQ("0x00000000000000000000000000000000", x.toHexString());
-  }
-
-  TEST(Int128, testFillInArray) {
-    Int128 x(0x123456789abcdef0, 0x23456789abcdef01);
-    uint32_t array[4];
-    bool wasNegative;
-    EXPECT_EQ(4, x.fillInArray(array, wasNegative));
-    EXPECT_EQ(true, !wasNegative);
-    EXPECT_EQ(0x12345678, array[0]);
-    EXPECT_EQ(0x9abcdef0, array[1]);
-    EXPECT_EQ(0x23456789, array[2]);
-    EXPECT_EQ(0xabcdef01, array[3]);
-
-    x = 0;
-    EXPECT_EQ(0, x.fillInArray(array, wasNegative));
-    EXPECT_EQ(true, !wasNegative);
-
-    x = 1;
-    EXPECT_EQ(1, x.fillInArray(array, wasNegative));
-    EXPECT_EQ(true, !wasNegative);
-    EXPECT_EQ(1, array[0]);
-
-    x = -12345;
-    EXPECT_EQ(1, x.fillInArray(array, wasNegative));
-    EXPECT_EQ(true, wasNegative);
-    EXPECT_EQ(12345, array[0]);
-
-    x = 0x80000000;
-    EXPECT_EQ(1, x.fillInArray(array, wasNegative));
-    EXPECT_EQ(true, !wasNegative);
-    EXPECT_EQ(0x80000000, array[0]);
-
-    x = Int128(0, 0x8000000000000000);
-    EXPECT_EQ(2, x.fillInArray(array, wasNegative));
-    EXPECT_EQ(true, !wasNegative);
-    EXPECT_EQ(0x80000000, array[0]);
-    EXPECT_EQ(0x0, array[1]);
-
-    x = Int128(0x80000000, 0x123456789abcdef0);
-    EXPECT_EQ(3, x.fillInArray(array, wasNegative));
-    EXPECT_EQ(true, !wasNegative);
-    EXPECT_EQ(0x80000000, array[0]);
-    EXPECT_EQ(0x12345678, array[1]);
-    EXPECT_EQ(0x9abcdef0, array[2]);
-  }
-
-  int fls(uint32_t x);
-
-  TEST(Int128, testFindLastSet) {
-    EXPECT_EQ(0, fls(0));
-    EXPECT_EQ(1, fls(1));
-    EXPECT_EQ(8, fls(0xff));
-    EXPECT_EQ(9, fls(0x100));
-    EXPECT_EQ(29, fls(0x12345678));
-    EXPECT_EQ(31, fls(0x40000000));
-    EXPECT_EQ(32, fls(0x80000000));
-  }
-
-  void shiftArrayLeft(uint32_t* array, int64_t length, int64_t bits);
-
-  TEST(Int128, testShiftArrayLeft) {
-    uint32_t array[5];
-    // make sure nothing blows up
-    array[0] = 0x12345678;
-    shiftArrayLeft(0, 0, 30);
-    EXPECT_EQ(0x12345678, array[0]);
-
-    array[0] = 0x12345678;
-    shiftArrayLeft(array, 1, 0);
-    EXPECT_EQ(0x12345678, array[0]);
-
-    array[0] = 0x12345678;
-    array[1] = 0x9abcdef0;
-    shiftArrayLeft(array, 1, 3);
-    EXPECT_EQ(0x91a2b3c0, array[0]);
-    EXPECT_EQ(0x9abcdef0, array[1]);
-
-    array[0] = 0x12345678;
-    array[1] = 0x9abcdeff;
-    array[2] = 0xfedcba98;
-    array[3] = 0x76543210;
-    shiftArrayLeft(array, 4, 4);
-    EXPECT_EQ(0x23456789, array[0]);
-    EXPECT_EQ(0xabcdefff, array[1]);
-    EXPECT_EQ(0xedcba987, array[2]);
-    EXPECT_EQ(0x65432100, array[3]);
-
-    array[0] = 0;
-    array[1] = 0x12345678;
-    array[2] = 0x9abcdeff;
-    array[3] = 0xfedcba98;
-    array[4] = 0x76543210;
-    shiftArrayLeft(array, 5, 8);
-    EXPECT_EQ(0x00000012, array[0]);
-    EXPECT_EQ(0x3456789a, array[1]);
-    EXPECT_EQ(0xbcdefffe, array[2]);
-    EXPECT_EQ(0xdcba9876, array[3]);
-    EXPECT_EQ(0x54321000, array[4]);
-  }
-
-  void shiftArrayRight(uint32_t* array, int64_t length, int64_t bits);
-
-  TEST(Int128, testShiftArrayRight) {
-    uint32_t array[4];
-    // make sure nothing blows up
-    array[0] = 0x12345678;
-    shiftArrayRight(0, 0, 30);
-    EXPECT_EQ(0x12345678, array[0]);
-
-    array[0] = 0x12345678;
-    array[1] = 0x9abcdef0;
-    shiftArrayRight(array, 1, 3);
-    EXPECT_EQ(0x2468acf, array[0]);
-    EXPECT_EQ(0x9abcdef0, array[1]);
-
-    array[0] = 0x12345678;
-    array[1] = 0x9abcdeff;
-    array[2] = 0xfedcba98;
-    array[3] = 0x76543210;
-    shiftArrayRight(array, 4, 4);
-    EXPECT_EQ(0x01234567, array[0]);
-    EXPECT_EQ(0x89abcdef, array[1]);
-    EXPECT_EQ(0xffedcba9, array[2]);
-    EXPECT_EQ(0x87654321, array[3]);
-  }
-
-  void fixDivisionSigns(Int128 &result, Int128 &remainder,
-                        bool dividendWasNegative, bool divisorWasNegative);
-
-  TEST(Int128, testFixDivisionSigns) {
-    Int128 x = 123;
-    Int128 y = 456;
-    fixDivisionSigns(x, y, false, false);
-    EXPECT_EQ(123, x.toLong());
-    EXPECT_EQ(456, y.toLong());
-
-    x = 123;
-    y = 456;
-    fixDivisionSigns(x, y, false, true);
-    EXPECT_EQ(-123, x.toLong());
-    EXPECT_EQ(456, y.toLong());
-
-    x = 123;
-    y = 456;
-    fixDivisionSigns(x, y, true, false);
-    EXPECT_EQ(-123, x.toLong());
-    EXPECT_EQ(-456, y.toLong());
-
-    x = 123;
-    y = 456;
-    fixDivisionSigns(x, y, true, true);
-    EXPECT_EQ(123, x.toLong());
-    EXPECT_EQ(-456, y.toLong());
-
-  }
-
-  void buildFromArray(Int128& value, uint32_t* array, int64_t length);
-
-  TEST(Int128, testBuildFromArray) {
-    Int128 result;
-    uint32_t array[4]={0x12345678, 0x9abcdef0, 0xfedcba98, 0x76543210};
-
-    buildFromArray(result, array, 0);
-    EXPECT_EQ(0, result.toLong());
-
-    buildFromArray(result, array, 1);
-    EXPECT_EQ(0x12345678, result.toLong());
-
-    buildFromArray(result, array, 2);
-    EXPECT_EQ(0x123456789abcdef0, result.toLong());
-
-    buildFromArray(result, array, 3);
-    EXPECT_EQ("0x00000000123456789abcdef0fedcba98", result.toHexString());
-
-    buildFromArray(result, array, 4);
-    EXPECT_EQ("0x123456789abcdef0fedcba9876543210", result.toHexString());
-
-    EXPECT_THROW(buildFromArray(result, array, 5), std::logic_error);
-  }
-
-  Int128 singleDivide(uint32_t* dividend, int64_t dividendLength,
-                      uint32_t divisor, Int128& remainder,
-                      bool dividendWasNegative, bool divisorWasNegative);
-
-  TEST(Int128, testSingleDivide) {
-    Int128 remainder;
-    uint32_t dividend[4];
-
-    dividend[0] = 23;
-    Int128 result = singleDivide(dividend, 1, 5, remainder, true, false);
-    EXPECT_EQ(-4, result.toLong());
-    EXPECT_EQ(-3, remainder.toLong());
-
-    dividend[0] = 0x100;
-    dividend[1] = 0x120;
-    dividend[2] = 0x140;
-    dividend[3] = 0x160;
-    result = singleDivide(dividend, 4, 0x20, remainder, false, false);
-    EXPECT_EQ("0x00000008000000090000000a0000000b", result.toHexString());
-    EXPECT_EQ(0, remainder.toLong());
-
-    dividend[0] = 0x101;
-    dividend[1] = 0x122;
-    dividend[2] = 0x143;
-    dividend[3] = 0x164;
-    result = singleDivide(dividend, 4, 0x20, remainder, false, false);
-    EXPECT_EQ("0x00000008080000091000000a1800000b", result.toHexString());
-    EXPECT_EQ(4, remainder.toLong());
-
-    dividend[0] = 0x12345678;
-    dividend[1] = 0x9abcdeff;
-    dividend[2] = 0xfedcba09;
-    dividend[3] = 0x87654321;
-    result = singleDivide(dividend, 4, 123, remainder, false, false);
-    EXPECT_EQ("0x0025e390971c97aaaaa84c7077bc23ed", result.toHexString());
-    EXPECT_EQ(0x42, remainder.toLong());
-  }
-
-  TEST(Int128, testDivide) {
-    Int128 dividend;
-    Int128 result;
-    Int128 remainder;
-
-    dividend = 0x12345678;
-    result = dividend.divide(0x123456789abcdef0, remainder);
-    EXPECT_EQ(0, result.toLong());
-    EXPECT_EQ(0x12345678, remainder.toLong());
-
-    EXPECT_THROW(dividend.divide(0, remainder), std::runtime_error);
-
-    dividend = Int128(0x123456789abcdeff, 0xfedcba0987654321);
-    result = dividend.divide(123, remainder);
-    EXPECT_EQ("0x0025e390971c97aaaaa84c7077bc23ed", result.toHexString());
-    EXPECT_EQ(0x42, remainder.toLong());
-
-    dividend = Int128(0x111111112fffffff, 0xeeeeeeeedddddddd);
-    result = dividend.divide(0x1111111123456789, remainder);
-    EXPECT_EQ("0x000000000000000100000000beeeeef7", result.toHexString());
-    EXPECT_EQ("0x0000000000000000037d3b3d60479aae", remainder.toHexString());
-
-    dividend = 1234234662345;
-    result = dividend.divide(642337, remainder);
-    EXPECT_EQ(1921475, result.toLong());
-    EXPECT_EQ(175270, remainder.toLong());
-
-    dividend = Int128(0x42395ADC0534AB4C, 0x59D109ADF9892FCA);
-    result = dividend.divide(0x1234F09DC19A, remainder);
-    EXPECT_EQ("0x000000000003a327c1348bccd2f06c27", result.toHexString());
-    EXPECT_EQ("0x000000000000000000000cacef73b954", remainder.toHexString());
-
-    dividend = Int128(0xfffffffffffffff, 0xf000000000000000);
-    result = dividend.divide(Int128(0, 0x1000000000000000), remainder);
-    EXPECT_EQ("0x0000000000000000ffffffffffffffff", result.toHexString());
-    EXPECT_EQ(0, remainder.toLong());
-
-    dividend = Int128(0x4000000000000000, 0);
-    result = dividend.divide(Int128(0, 0x400000007fffffff), remainder);
-    EXPECT_EQ("0x0000000000000000fffffffe00000007", result.toHexString());
-    EXPECT_EQ("0x00000000000000003ffffffa80000007", remainder.toHexString());
-  }
-
-  TEST(Int128, testToString) {
-    Int128 num = Int128(0x123456789abcdef0, 0xfedcba0987654321);
-    EXPECT_EQ("24197857203266734881846307133640229665", num.toString());
-
-    num = Int128(0, 0xab54a98ceb1f0ad2);
-    EXPECT_EQ("12345678901234567890", num.toString());
-
-    num = 12345678;
-    EXPECT_EQ("12345678", num.toString());
-
-    num = -1234;
-    EXPECT_EQ("-1234", num.toString());
-
-    num = Int128(0x13f20d9c2, 0xfff89d38e1c70cb1);
-    EXPECT_EQ("98765432109876543210987654321", num.toString());
-    num.negate();
-    EXPECT_EQ("-98765432109876543210987654321", num.toString());
-
-    num = Int128("10000000000000000000000000000000000000");
-    EXPECT_EQ("10000000000000000000000000000000000000", num.toString());
-
-    num = Int128("-1234");
-    EXPECT_EQ("-1234", num.toString());
-
-    num = Int128("-12345678901122334455667788990011122233");
-    EXPECT_EQ("-12345678901122334455667788990011122233", num.toString());
-  }
-
-  TEST(Int128, testToDecimalString) {
-    Int128 num = Int128("98765432109876543210987654321098765432");
-    EXPECT_EQ("98765432109876543210987654321098765432",
-              num.toDecimalString(0));
-    EXPECT_EQ("987654321098765432109876543210987.65432",
-              num.toDecimalString(5));
-    num.negate();
-    EXPECT_EQ("-98765432109876543210987654321098765432",
-              num.toDecimalString(0));
-    EXPECT_EQ("-987654321098765432109876543210987.65432",
-              num.toDecimalString(5));
-    num = 123;
-    EXPECT_EQ("12.3", num.toDecimalString(1));
-    EXPECT_EQ("0.123", num.toDecimalString(3));
-    EXPECT_EQ("0.0123", num.toDecimalString(4));
-    EXPECT_EQ("0.00123", num.toDecimalString(5));
-
-    num = -123;
-    EXPECT_EQ("-123", num.toDecimalString(0));
-    EXPECT_EQ("-12.3", num.toDecimalString(1));
-    EXPECT_EQ("-0.123", num.toDecimalString(3));
-    EXPECT_EQ("-0.0123", num.toDecimalString(4));
-    EXPECT_EQ("-0.00123", num.toDecimalString(5));
-  }
-}  // namespace orc


[17/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/Exceptions.cc
----------------------------------------------------------------------
diff --git a/c++/src/orc/Exceptions.cc b/c++/src/orc/Exceptions.cc
deleted file mode 100644
index ae0e3d1..0000000
--- a/c++/src/orc/Exceptions.cc
+++ /dev/null
@@ -1,59 +0,0 @@
-/**
- * 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 "Exceptions.hh"
-
-namespace orc {
-
-  NotImplementedYet::NotImplementedYet(const std::string& what_arg
-                                       ) : logic_error(what_arg) {
-    // PASS
-  }
-
-  NotImplementedYet::NotImplementedYet(const char* what_arg
-                                       ) :logic_error(what_arg) {
-    // PASS
-  }
-
-  NotImplementedYet::NotImplementedYet(const NotImplementedYet& error
-                                       ): logic_error(error) {
-    // PASS
-  }
-
-  NotImplementedYet::~NotImplementedYet() noexcept {
-    // PASS
-  }
-
-  ParseError::ParseError(const std::string& what_arg
-                         ): runtime_error(what_arg) {
-    // PASS
-  }
-
-  ParseError::ParseError(const char* what_arg
-                         ): runtime_error(what_arg) {
-    // PASS
-  }
-
-  ParseError::ParseError(const ParseError& error): runtime_error(error) {
-    // PASS
-  }
-
-  ParseError::~ParseError() noexcept {
-    // PASS
-  }
-}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/Exceptions.hh
----------------------------------------------------------------------
diff --git a/c++/src/orc/Exceptions.hh b/c++/src/orc/Exceptions.hh
deleted file mode 100644
index ae23181..0000000
--- a/c++/src/orc/Exceptions.hh
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
- * 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 ORC_EXCEPTIONS_HH
-#define ORC_EXCEPTIONS_HH
-
-#include "orc/Adaptor.hh"
-
-#include <stdexcept>
-#include <string>
-
-namespace orc {
-
-  class NotImplementedYet: public std::logic_error {
-  public:
-    explicit NotImplementedYet(const std::string& what_arg);
-    explicit NotImplementedYet(const char* what_arg);
-    virtual ~NotImplementedYet() noexcept;
-    NotImplementedYet(const NotImplementedYet&);
-  private:
-    NotImplementedYet& operator=(const NotImplementedYet&);
-  };
-
-  class ParseError: public std::runtime_error {
-  public:
-    explicit ParseError(const std::string& what_arg);
-    explicit ParseError(const char* what_arg);
-    virtual ~ParseError() noexcept;
-    ParseError(const ParseError&);
-  private:
-    ParseError& operator=(const ParseError&);
-  };
-}
-
-#endif

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/Int128.cc
----------------------------------------------------------------------
diff --git a/c++/src/orc/Int128.cc b/c++/src/orc/Int128.cc
deleted file mode 100644
index 75cff49..0000000
--- a/c++/src/orc/Int128.cc
+++ /dev/null
@@ -1,438 +0,0 @@
-/**
- * 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 "orc/Int128.hh"
-#include "orc/Adaptor.hh"
-
-#include <iomanip>
-#include <iostream>
-#include <sstream>
-
-namespace orc {
-
-  Int128 Int128::maximumValue() {
-    return Int128(0x7fffffffffffffff, 0xfffffffffffffff);
-  }
-
-  Int128 Int128::minimumValue() {
-    return Int128(static_cast<int64_t>(0x8000000000000000), 0x0);
-  }
-
-  Int128::Int128(const std::string& str) {
-    lowbits = 0;
-    highbits = 0;
-    size_t length = str.length();
-    if (length > 0) {
-      bool isNegative = str[0] == '-';
-      size_t posn = isNegative ? 1 : 0;
-      while (posn < length) {
-        size_t group = std::min(18ul, length - posn);
-        int64_t chunk = std::stoll(str.substr(posn, group));
-        int64_t multiple = 1;
-        for(size_t i=0; i < group; ++i) {
-          multiple *= 10;
-        }
-        *this *= multiple;
-        *this += chunk;
-        posn += group;
-      }
-      if (isNegative) {
-        negate();
-      }
-    }
-  }
-
-  Int128& Int128::operator*=(const Int128 &right) {
-    const uint64_t INT_MASK = 0xffffffff;
-    const uint64_t CARRY_BIT = 1l << 32;
-
-    // Break the left and right numbers into 32 bit chunks
-    // so that we can multiply them without overflow.
-    uint64_t L0 = static_cast<uint64_t>(highbits) >> 32;
-    uint64_t L1 = static_cast<uint64_t>(highbits) & INT_MASK;
-    uint64_t L2 = lowbits >> 32;
-    uint64_t L3 = lowbits & INT_MASK;
-    uint64_t R0 = static_cast<uint64_t>(right.highbits) >> 32;
-    uint64_t R1 = static_cast<uint64_t>(right.highbits) & INT_MASK;
-    uint64_t R2 = right.lowbits >> 32;
-    uint64_t R3 = right.lowbits & INT_MASK;
-
-    uint64_t product = L3 * R3;
-    lowbits = product & INT_MASK;
-    uint64_t sum = product >> 32;
-    product = L2 * R3;
-    sum += product;
-    highbits = sum < product ? CARRY_BIT : 0;
-    product = L3 * R2;
-    sum += product;
-    if (sum < product) {
-      highbits += CARRY_BIT;
-    }
-    lowbits += sum << 32;
-    highbits += static_cast<int64_t>(sum >> 32);
-    highbits += L1 * R3 + L2 * R2 + L3 * R1;
-    highbits += (L0 * R3 + L1 * R2 + L2 * R1 + L3 * R0) << 32;
-    return *this;
-  }
-
-  /**
-   * Expands the given value into an array of ints so that we can work on
-   * it. The array will be converted to an absolute value and the wasNegative
-   * flag will be set appropriately. The array will remove leading zeros from
-   * the value.
-   * @param array an array of length 4 to set with the value
-   * @param wasNegative a flag for whether the value was original negative
-   * @result the output length of the array
-   */
-  int64_t Int128::fillInArray(uint32_t* array, bool &wasNegative) const {
-    uint64_t high;
-    uint64_t low;
-    if (highbits < 0) {
-      low = ~lowbits + 1;
-      high = static_cast<uint64_t>(~highbits);
-      if (low == 0) {
-        high += 1;
-      }
-      wasNegative = true;
-    } else {
-      low = lowbits;
-      high = static_cast<uint64_t>(highbits);
-      wasNegative = false;
-    }
-    if (high != 0) {
-      if (high > UINT32_MAX) {
-        array[0] = static_cast<uint32_t>(high >> 32);
-        array[1] = static_cast<uint32_t>(high);
-        array[2] = static_cast<uint32_t>(low >> 32);
-        array[3] = static_cast<uint32_t>(low);
-        return 4;
-      } else {
-        array[0] = static_cast<uint32_t>(high);
-        array[1] = static_cast<uint32_t>(low >> 32);
-        array[2] = static_cast<uint32_t>(low);
-        return 3;
-      }
-    } else if (low >= UINT32_MAX) {
-      array[0] = static_cast<uint32_t>(low >> 32);
-      array[1] = static_cast<uint32_t>(low);
-      return 2;
-    } else if (low == 0) {
-      return 0;
-    } else {
-      array[0] = static_cast<uint32_t>(low);
-      return 1;
-    }
-  }
-
-
-  /**
-   * Find last set bit in a 32 bit integer. Bit 1 is the LSB and bit 32 is
-   * the MSB. We can replace this with bsrq asm instruction on x64.
-   */
-  int64_t fls(uint32_t x) {
-    int64_t bitpos = 0;
-    while (x) {
-      x >>= 1;
-      bitpos += 1;
-    }
-    return bitpos;
-  }
-
-  /**
-   * Shift the number in the array left by bits positions.
-   * @param array the number to shift, must have length elements
-   * @param length the number of entries in the array
-   * @param bits the number of bits to shift (0 <= bits < 32)
-   */
-  void shiftArrayLeft(uint32_t* array, int64_t length, int64_t bits) {
-    if (length > 0 && bits != 0) {
-      for(int64_t i=0; i < length-1; ++i) {
-        array[i] = (array[i] << bits) | (array[i+1] >> (32 - bits));
-      }
-      array[length-1] <<= bits;
-    }
-  }
-
-  /**
-   * Shift the number in the array right by bits positions.
-   * @param array the number to shift, must have length elements
-   * @param length the number of entries in the array
-   * @param bits the number of bits to shift (0 <= bits < 32)
-   */
-  void shiftArrayRight(uint32_t* array, int64_t length, int64_t bits) {
-    if (length > 0 && bits != 0) {
-      for(int64_t i=length-1; i > 0; --i) {
-        array[i] = (array[i] >> bits) | (array[i-1] << (32 - bits));
-      }
-      array[0] >>= bits;
-    }
-  }
-
-  /**
-   * Fix the signs of the result and remainder at the end of the division
-   * based on the signs of the dividend and divisor.
-   */
-  void fixDivisionSigns(Int128 &result, Int128 &remainder,
-                        bool dividendWasNegative, bool divisorWasNegative) {
-    if (dividendWasNegative != divisorWasNegative) {
-      result.negate();
-    }
-    if (dividendWasNegative) {
-      remainder.negate();
-    }
-  }
-
-  /**
-   * Build a Int128 from a list of ints.
-   */
-  void buildFromArray(Int128& value, uint32_t* array, int64_t length) {
-    switch (length) {
-    case 0:
-      value = 0;
-      break;
-    case 1:
-      value = array[0];
-      break;
-    case 2:
-      value = Int128(0, (static_cast<uint64_t>(array[0]) << 32) + array[1]);
-      break;
-    case 3:
-      value = Int128(array[0],
-                     (static_cast<uint64_t>(array[1]) << 32) + array[2]);
-      break;
-    case 4:
-      value = Int128((static_cast<int64_t>(array[0]) << 32) + array[1],
-                     (static_cast<uint64_t>(array[2]) << 32) + array[3]);
-      break;
-    case 5:
-      if (array[0] != 0) {
-        throw std::logic_error("Can't build Int128 with 5 ints.");
-      }
-      value = Int128((static_cast<int64_t>(array[1]) << 32) + array[2],
-                     (static_cast<uint64_t>(array[3]) << 32) + array[4]);
-      break;
-    default:
-      throw std::logic_error("Unsupported length for building Int128");
-    }
-  }
-
-  /**
-   * Do a division where the divisor fits into a single 32 bit value.
-   */
-  Int128 singleDivide(uint32_t* dividend, int64_t dividendLength,
-                      uint32_t divisor, Int128& remainder,
-                      bool dividendWasNegative, bool divisorWasNegative) {
-    uint64_t r = 0;
-    uint32_t resultArray[5];
-    for(int64_t j=0; j < dividendLength; j++) {
-      r <<= 32;
-      r += dividend[j];
-      resultArray[j] = static_cast<uint32_t>(r / divisor);
-      r %= divisor;
-    }
-    Int128 result;
-    buildFromArray(result, resultArray, dividendLength);
-    remainder = static_cast<int64_t>(r);
-    fixDivisionSigns(result, remainder, dividendWasNegative,
-                     divisorWasNegative);
-    return result;
-  }
-
-  Int128 Int128::divide(const Int128 &divisor, Int128 &remainder) const {
-    // Split the dividend and divisor into integer pieces so that we can
-    // work on them.
-    uint32_t dividendArray[5];
-    uint32_t divisorArray[4];
-    bool dividendWasNegative;
-    bool divisorWasNegative;
-    // leave an extra zero before the dividend
-    dividendArray[0] = 0;
-    int64_t dividendLength = fillInArray(dividendArray + 1, dividendWasNegative)+1;
-    int64_t divisorLength = divisor.fillInArray(divisorArray, divisorWasNegative);
-
-    // Handle some of the easy cases.
-    if (dividendLength <= divisorLength) {
-      remainder = *this;
-      return 0;
-    } else if (divisorLength == 0) {
-      throw std::range_error("Division by 0 in Int128");
-    } else if (divisorLength == 1) {
-      return singleDivide(dividendArray, dividendLength, divisorArray[0],
-                          remainder, dividendWasNegative, divisorWasNegative);
-    }
-
-    int64_t resultLength = dividendLength - divisorLength;
-    uint32_t resultArray[4];
-
-    // Normalize by shifting both by a multiple of 2 so that
-    // the digit guessing is better. The requirement is that
-    // divisorArray[0] is greater than 2**31.
-    int64_t normalizeBits = 32 - fls(divisorArray[0]);
-    shiftArrayLeft(divisorArray, divisorLength, normalizeBits);
-    shiftArrayLeft(dividendArray, dividendLength, normalizeBits);
-
-    // compute each digit in the result
-    for(int64_t j=0; j < resultLength; ++j) {
-      // Guess the next digit. At worst it is two too large
-      uint32_t guess = UINT32_MAX;
-      uint64_t highDividend = static_cast<uint64_t>(dividendArray[j]) << 32 |
-        dividendArray[j+1];
-      if (dividendArray[j] != divisorArray[0]) {
-        guess = static_cast<uint32_t>(highDividend / divisorArray[0]);
-      }
-
-      // catch all of the cases where guess is two too large and most of the
-      // cases where it is one too large
-      uint32_t rhat =
-        static_cast<uint32_t>(highDividend - guess *
-                              static_cast<uint64_t>(divisorArray[0]));
-      while (static_cast<uint64_t>(divisorArray[1]) * guess >
-             (static_cast<uint64_t>(rhat) << 32) + dividendArray[j+2]) {
-        guess -= 1;
-        rhat += divisorArray[0];
-        if (static_cast<uint64_t>(rhat) < divisorArray[0]) {
-          break;
-        }
-      }
-
-      // subtract off the guess * divisor from the dividend
-      uint64_t mult = 0;
-      for(int64_t i=divisorLength-1; i >= 0; --i) {
-        mult += static_cast<uint64_t>(guess) * divisorArray[i];
-        uint32_t prev = dividendArray[j+i+1];
-        dividendArray[j+i+1] -= static_cast<uint32_t>(mult);
-        mult >>= 32;
-        if (dividendArray[j+i+1] > prev) {
-          mult += 1;
-        }
-      }
-      uint32_t prev = dividendArray[j];
-      dividendArray[j] -= static_cast<uint32_t>(mult);
-
-      // if guess was too big, we add back divisor
-      if (dividendArray[j] > prev) {
-        guess -= 1;
-        uint32_t carry = 0;
-        for(int64_t i=divisorLength-1; i >= 0; --i) {
-          uint64_t sum = static_cast<uint64_t>(divisorArray[i]) +
-            dividendArray[j+i+1] + carry;
-          dividendArray[j+i+1] = static_cast<uint32_t>(sum);
-          carry = static_cast<uint32_t>(sum >> 32);
-        }
-        dividendArray[j] += carry;
-      }
-
-      resultArray[j] = guess;
-    }
-
-    // denormalize the remainder
-    shiftArrayRight(dividendArray, dividendLength, normalizeBits);
-
-    // return result and remainder
-    Int128 result;
-    buildFromArray(result, resultArray, resultLength);
-    buildFromArray(remainder, dividendArray, dividendLength);
-    fixDivisionSigns(result, remainder,
-                     dividendWasNegative, divisorWasNegative);
-    return result;
-  }
-
-  std::string Int128::toString() const {
-    // 10**18 - the largest power of 10 less than 63 bits
-    const Int128 tenTo18(0xde0b6b3a7640000);
-    // 10**36
-    const Int128 tenTo36(0xc097ce7bc90715, 0xb34b9f1000000000);
-    Int128 remainder;
-    std::stringstream buf;
-    bool needFill = false;
-
-    // get anything above 10**36 and print it
-    Int128 top = divide(tenTo36, remainder);
-    if (top != 0) {
-      buf << top.toLong();
-      remainder.abs();
-      needFill = true;
-    }
-
-    // now get anything above 10**18 and print it
-    Int128 tail;
-    top = remainder.divide(tenTo18, tail);
-    if (needFill || top != 0) {
-      if (needFill) {
-        buf << std::setw(18) << std::setfill('0');
-      } else {
-        needFill = true;
-        tail.abs();
-      }
-      buf << top.toLong();
-    }
-
-    // finally print the tail, which is less than 10**18
-    if (needFill) {
-      buf << std::setw(18) << std::setfill('0');
-    }
-    buf << tail.toLong();
-    return buf.str();
-  }
-
-  std::string Int128::toDecimalString(int32_t scale) const {
-    std::string str = toString();
-    if (scale == 0) {
-      return str;
-    } else if (*this < 0) {
-      int32_t len = static_cast<int32_t>(str.length());
-      if (len - 1 > scale) {
-        return str.substr(0, static_cast<size_t>(len - scale)) + "." +
-          str.substr(static_cast<size_t>(len - scale),
-                     static_cast<size_t>(scale));
-      } else if (len - 1 == scale) {
-        return "-0." + str.substr(1, std::string::npos);
-      } else {
-        std::string result = "-0.";
-        for(int32_t i=0; i < scale - len + 1; ++i) {
-          result += "0";
-        }
-        return result + str.substr(1, std::string::npos);
-      }
-    } else {
-      int32_t len = static_cast<int32_t>(str.length());
-      if (len > scale) {
-        return str.substr(0, static_cast<size_t>(len - scale)) + "." +
-          str.substr(static_cast<size_t>(len - scale),
-                     static_cast<size_t>(scale));
-      } else if (len == scale) {
-        return "0." + str;
-      } else {
-        std::string result = "0.";
-        for(int32_t i=0; i < scale - len; ++i) {
-          result += "0";
-        }
-        return result + str;
-      }
-    }
-  }
-
-  std::string Int128::toHexString() const {
-    std::stringstream buf;
-    buf << std::hex << "0x"
-        << std::setw(16) << std::setfill('0') << highbits
-        << std::setw(16) << std::setfill('0') << lowbits;
-    return buf.str();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/MemoryPool.cc
----------------------------------------------------------------------
diff --git a/c++/src/orc/MemoryPool.cc b/c++/src/orc/MemoryPool.cc
deleted file mode 100644
index 98aa296..0000000
--- a/c++/src/orc/MemoryPool.cc
+++ /dev/null
@@ -1,231 +0,0 @@
-/**
- * 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 "orc/Adaptor.hh"
-#include "orc/Int128.hh"
-#include "orc/MemoryPool.hh"
-
-#include <cstdlib>
-#include <iostream>
-#include <string.h>
-
-namespace orc {
-
-  MemoryPool::~MemoryPool() {
-    // PASS
-  }
-
-  class MemoryPoolImpl: public MemoryPool {
-  public:
-    virtual ~MemoryPoolImpl();
-
-    char* malloc(uint64_t size) override;
-    void free(char* p) override;
-  };
-
-  char* MemoryPoolImpl::malloc(uint64_t size) {
-    return static_cast<char*>(std::malloc(size));
-  }
-
-  void MemoryPoolImpl::free(char* p) {
-    std::free(p);
-  }
-
-  MemoryPoolImpl::~MemoryPoolImpl() {
-    // PASS
-  }
-
-  template <class T>
-  DataBuffer<T>::DataBuffer(MemoryPool& pool,
-                            uint64_t newSize
-                            ): memoryPool(pool),
-                               buf(nullptr),
-                               currentSize(0),
-                               currentCapacity(0) {
-    resize(newSize);
-  }
-
-  template <class T>
-  DataBuffer<T>::~DataBuffer(){
-    for(uint64_t i=currentSize; i > 0; --i) {
-      (buf + i - 1)->~T();
-    }
-    if (buf) {
-      memoryPool.free(reinterpret_cast<char*>(buf));
-    }
-  }
-
-  template <class T>
-  void DataBuffer<T>::resize(uint64_t newSize) {
-    reserve(newSize);
-    if (currentSize > newSize) {
-      for(uint64_t i=currentSize; i > newSize; --i) {
-        (buf + i - 1)->~T();
-      }
-    } else if (newSize > currentSize) {
-      for(uint64_t i=currentSize; i < newSize; ++i) {
-        new (buf + i) T();
-      }
-    }
-    currentSize = newSize;
-  }
-
-  template <class T>
-  void DataBuffer<T>::reserve(uint64_t newCapacity){
-    if (newCapacity > currentCapacity) {
-      if (buf) {
-        T* buf_old = buf;
-        buf = reinterpret_cast<T*>(memoryPool.malloc(sizeof(T) * newCapacity));
-        memcpy(buf, buf_old, sizeof(T) * currentSize);
-        memoryPool.free(reinterpret_cast<char*>(buf_old));
-      } else {
-        buf = reinterpret_cast<T*>(memoryPool.malloc(sizeof(T) * newCapacity));
-      }
-      currentCapacity = newCapacity;
-    }
-  }
-
-  // Specializations for char
-
-  template <>
-  DataBuffer<char>::~DataBuffer(){
-    if (buf) {
-      memoryPool.free(reinterpret_cast<char*>(buf));
-    }
-  }
-
-  template <>
-  void DataBuffer<char>::resize(uint64_t newSize) {
-    reserve(newSize);
-    if (newSize > currentSize) {
-      memset(buf + currentSize, 0, newSize - currentSize);
-    }
-    currentSize = newSize;
-  }
-
-  // Specializations for char*
-
-  template <>
-  DataBuffer<char*>::~DataBuffer(){
-    if (buf) {
-      memoryPool.free(reinterpret_cast<char*>(buf));
-    }
-  }
-
-  template <>
-  void DataBuffer<char*>::resize(uint64_t newSize) {
-    reserve(newSize);
-    if (newSize > currentSize) {
-      memset(buf + currentSize, 0, (newSize - currentSize) * sizeof(char*));
-    }
-    currentSize = newSize;
-  }
-
-  // Specializations for double
-
-  template <>
-  DataBuffer<double>::~DataBuffer(){
-    if (buf) {
-      memoryPool.free(reinterpret_cast<char*>(buf));
-    }
-  }
-
-  template <>
-  void DataBuffer<double>::resize(uint64_t newSize) {
-    reserve(newSize);
-    if (newSize > currentSize) {
-      memset(buf + currentSize, 0, (newSize - currentSize) * sizeof(double));
-    }
-    currentSize = newSize;
-  }
-
-  // Specializations for int64_t
-
-  template <>
-  DataBuffer<int64_t>::~DataBuffer(){
-    if (buf) {
-      memoryPool.free(reinterpret_cast<char*>(buf));
-    }
-  }
-
-  template <>
-  void DataBuffer<int64_t>::resize(uint64_t newSize) {
-    reserve(newSize);
-    if (newSize > currentSize) {
-      memset(buf + currentSize, 0, (newSize - currentSize) * sizeof(int64_t));
-    }
-    currentSize = newSize;
-  }
-
-  // Specializations for uint64_t
-
-  template <>
-  DataBuffer<uint64_t>::~DataBuffer(){
-    if (buf) {
-      memoryPool.free(reinterpret_cast<char*>(buf));
-    }
-  }
-
-  template <>
-  void DataBuffer<uint64_t>::resize(uint64_t newSize) {
-    reserve(newSize);
-    if (newSize > currentSize) {
-      memset(buf + currentSize, 0, (newSize - currentSize) * sizeof(uint64_t));
-    }
-    currentSize = newSize;
-  }
-
-  // Specializations for unsigned char
-
-  template <>
-  DataBuffer<unsigned char>::~DataBuffer(){
-    if (buf) {
-      memoryPool.free(reinterpret_cast<char*>(buf));
-    }
-  }
-
-  template <>
-  void DataBuffer<unsigned char>::resize(uint64_t newSize) {
-    reserve(newSize);
-    if (newSize > currentSize) {
-      memset(buf + currentSize, 0, newSize - currentSize);
-    }
-    currentSize = newSize;
-  }
-
-  #ifdef __clang__
-    #pragma clang diagnostic ignored "-Wweak-template-vtables"
-  #endif
-
-  template class DataBuffer<char>;
-  template class DataBuffer<char*>;
-  template class DataBuffer<double>;
-  template class DataBuffer<Int128>;
-  template class DataBuffer<int64_t>;
-  template class DataBuffer<uint64_t>;
-  template class DataBuffer<unsigned char>;
-
-  #ifdef __clang__
-    #pragma clang diagnostic ignored "-Wexit-time-destructors"
-  #endif
-
-  MemoryPool* getDefaultPool() {
-    static MemoryPoolImpl internal;
-    return &internal;
-  }
-} // namespace orc

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/OrcFile.cc
----------------------------------------------------------------------
diff --git a/c++/src/orc/OrcFile.cc b/c++/src/orc/OrcFile.cc
deleted file mode 100644
index 9ae9c56..0000000
--- a/c++/src/orc/OrcFile.cc
+++ /dev/null
@@ -1,101 +0,0 @@
-/**
- * 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 "orc/Adaptor.hh"
-#include "orc/OrcFile.hh"
-#include "Exceptions.hh"
-
-#include <errno.h>
-#include <fcntl.h>
-#include <stdio.h>
-#include <sys/mman.h>
-#include <sys/stat.h>
-#include <sys/types.h>
-#include <unistd.h>
-
-namespace orc {
-
-  class FileInputStream : public InputStream {
-  private:
-    std::string filename ;
-    int file;
-    uint64_t totalLength;
-
-  public:
-    FileInputStream(std::string _filename) {
-      filename = _filename ;
-      file = open(filename.c_str(), O_RDONLY);
-      if (file == -1) {
-        throw ParseError("Can't open " + filename);
-      }
-      struct stat fileStat;
-      if (fstat(file, &fileStat) == -1) {
-        throw ParseError("Can't stat " + filename);
-      }
-      totalLength = static_cast<uint64_t>(fileStat.st_size);
-    }
-
-    ~FileInputStream();
-
-    uint64_t getLength() const override {
-      return totalLength;
-    }
-
-    void read(void* buf,
-              uint64_t length,
-              uint64_t offset) override {
-      if (!buf) {
-        throw ParseError("Buffer is null");
-      }
-      ssize_t bytesRead = pread(file, buf, length, static_cast<off_t>(offset));
-
-      if (bytesRead == -1) {
-        throw ParseError("Bad read of " + filename);
-      }
-      if (static_cast<uint64_t>(bytesRead) != length) {
-        throw ParseError("Short read of " + filename);
-      }
-    }
-
-    const std::string& getName() const override {
-      return filename;
-    }
-  };
-
-  FileInputStream::~FileInputStream() {
-    close(file);
-  }
-
-  std::unique_ptr<InputStream> readLocalFile(const std::string& path) {
-    return std::unique_ptr<InputStream>(new FileInputStream(path));
-  }
-}
-
-#ifndef HAS_STOLL
-
-  #include <sstream>
-
-  int64_t std::stoll(std::string str) {
-    int64_t val = 0;
-    stringstream ss ;
-    ss << str ;
-    ss >> val ;
-    return val;
-  }
-
-#endif

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/RLE.cc
----------------------------------------------------------------------
diff --git a/c++/src/orc/RLE.cc b/c++/src/orc/RLE.cc
deleted file mode 100644
index 51bd628..0000000
--- a/c++/src/orc/RLE.cc
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
-* 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 "RLEv1.hh"
-#include "RLEv2.hh"
-#include "Exceptions.hh"
-
-namespace orc {
-
-  RleDecoder::~RleDecoder() {
-    // PASS
-  }
-
-  std::unique_ptr<RleDecoder> createRleDecoder
-                         (std::unique_ptr<SeekableInputStream> input,
-                          bool isSigned,
-                          RleVersion version,
-                          MemoryPool& pool) {
-    switch (static_cast<int64_t>(version)) {
-    case RleVersion_1:
-      // We don't have std::make_unique() yet.
-      return std::unique_ptr<RleDecoder>(new RleDecoderV1(std::move(input),
-                                                          isSigned));
-    case RleVersion_2:
-      return std::unique_ptr<RleDecoder>(new RleDecoderV2(std::move(input),
-                                                          isSigned, pool));
-    default:
-      throw NotImplementedYet("Not implemented yet");
-    }
-  }
-
-}  // namespace orc

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/RLE.hh
----------------------------------------------------------------------
diff --git a/c++/src/orc/RLE.hh b/c++/src/orc/RLE.hh
deleted file mode 100644
index 0a44c95..0000000
--- a/c++/src/orc/RLE.hh
+++ /dev/null
@@ -1,78 +0,0 @@
-/**
- * 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 ORC_RLE_HH
-#define ORC_RLE_HH
-
-#include "Compression.hh"
-
-#include <memory>
-
-namespace orc {
-
-  inline int64_t unZigZag(uint64_t value) {
-    return value >> 1 ^ -(value & 1);
-  }
-
-  class RleDecoder {
-  public:
-    // must be non-inline!
-    virtual ~RleDecoder();
-
-    /**
-     * Seek to a particular spot.
-     */
-    virtual void seek(PositionProvider&) = 0;
-
-    /**
-     * Seek over a given number of values.
-     */
-    virtual void skip(uint64_t numValues) = 0;
-
-    /**
-     * Read a number of values into the batch.
-     * @param data the array to read into
-     * @param numValues the number of values to read
-     * @param notNull If the pointer is null, all values are read. If the
-     *    pointer is not null, positions that are false are skipped.
-     */
-    virtual void next(int64_t* data, uint64_t numValues,
-                      const char* notNull) = 0;
-  };
-
-  enum RleVersion {
-    RleVersion_1,
-    RleVersion_2
-  };
-
-  /**
-   * Create an RLE decoder.
-   * @param input the input stream to read from
-   * @param isSigned true if the number sequence is signed
-   * @param version version of RLE decoding to do
-   * @param pool memory pool to use for allocation
-   */
-  std::unique_ptr<RleDecoder> createRleDecoder
-                      (std::unique_ptr<SeekableInputStream> input,
-                       bool isSigned,
-                       RleVersion version,
-                       MemoryPool& pool);
-
-}  // namespace orc
-
-#endif  // ORC_RLE_HH

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/RLEv1.cc
----------------------------------------------------------------------
diff --git a/c++/src/orc/RLEv1.cc b/c++/src/orc/RLEv1.cc
deleted file mode 100644
index 55e52d2..0000000
--- a/c++/src/orc/RLEv1.cc
+++ /dev/null
@@ -1,190 +0,0 @@
-/**
- * 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 "orc/Adaptor.hh"
-#include "RLEv1.hh"
-#include "Compression.hh"
-#include "Exceptions.hh"
-
-#include <algorithm>
-
-namespace orc {
-
-const uint64_t MINIMUM_REPEAT = 3;
-const uint64_t BASE_128_MASK = 0x7f;
-
-signed char RleDecoderV1::readByte() {
-  if (bufferStart == bufferEnd) {
-    int bufferLength;
-    const void* bufferPointer;
-    if (!inputStream->Next(&bufferPointer, &bufferLength)) {
-      throw ParseError("bad read in readByte");
-    }
-    bufferStart = static_cast<const char*>(bufferPointer);
-    bufferEnd = bufferStart + bufferLength;
-  }
-  return *(bufferStart++);
-}
-
-uint64_t RleDecoderV1::readLong() {
-  uint64_t result = 0;
-  int64_t offset = 0;
-  signed char ch = readByte();
-  if (ch >= 0) {
-    result = static_cast<uint64_t>(ch);
-  } else {
-    result = static_cast<uint64_t>(ch) & BASE_128_MASK;
-    while ((ch = readByte()) < 0) {
-      offset += 7;
-      result |= (static_cast<uint64_t>(ch) & BASE_128_MASK) << offset;
-    }
-    result |= static_cast<uint64_t>(ch) << (offset + 7);
-  }
-  return result;
-}
-
-void RleDecoderV1::skipLongs(uint64_t numValues) {
-  while (numValues > 0) {
-    if (readByte() >= 0) {
-      --numValues;
-    }
-  }
-}
-
-void RleDecoderV1::readHeader() {
-  signed char ch = readByte();
-  if (ch < 0) {
-    remainingValues = static_cast<uint64_t>(-ch);
-    repeating = false;
-  } else {
-    remainingValues = static_cast<uint64_t>(ch) + MINIMUM_REPEAT;
-    repeating = true;
-    delta = readByte();
-    value = isSigned
-        ? unZigZag(readLong())
-        : static_cast<int64_t>(readLong());
-  }
-}
-
-RleDecoderV1::RleDecoderV1(std::unique_ptr<SeekableInputStream> input,
-                           bool hasSigned)
-    : inputStream(std::move(input)),
-      isSigned(hasSigned),
-      remainingValues(0),
-      bufferStart(nullptr),
-      bufferEnd(bufferStart) {
-}
-
-void RleDecoderV1::seek(PositionProvider& location) {
-  // move the input stream
-  inputStream->seek(location);
-  // force a re-read from the stream
-  bufferEnd = bufferStart;
-  // read a new header
-  readHeader();
-  // skip ahead the given number of records
-  skip(location.next());
-}
-
-void RleDecoderV1::skip(uint64_t numValues) {
-  while (numValues > 0) {
-    if (remainingValues == 0) {
-      readHeader();
-    }
-    uint64_t count = std::min(numValues, remainingValues);
-    remainingValues -= count;
-    numValues -= count;
-    if (repeating) {
-      value += delta * static_cast<int64_t>(count);
-    } else {
-      skipLongs(count);
-    }
-  }
-}
-
-void RleDecoderV1::next(int64_t* const data,
-                        const uint64_t numValues,
-                        const char* const notNull) {
-  uint64_t position = 0;
-  // skipNulls()
-  if (notNull) {
-    // Skip over null values.
-    while (position < numValues && !notNull[position]) {
-      ++position;
-    }
-  }
-  while (position < numValues) {
-    // If we are out of values, read more.
-    if (remainingValues == 0) {
-      readHeader();
-    }
-    // How many do we read out of this block?
-    uint64_t count = std::min(numValues - position, remainingValues);
-    uint64_t consumed = 0;
-    if (repeating) {
-      if (notNull) {
-        for (uint64_t i = 0; i < count; ++i) {
-          if (notNull[position + i]) {
-            data[position + i] = value + static_cast<int64_t>(consumed) * delta;
-            consumed += 1;
-          }
-        }
-      } else {
-        for (uint64_t i = 0; i < count; ++i) {
-          data[position + i] = value + static_cast<int64_t>(i) * delta;
-        }
-        consumed = count;
-      }
-      value += static_cast<int64_t>(consumed) * delta;
-    } else {
-      if (notNull) {
-        for (uint64_t i = 0 ; i < count; ++i) {
-          if (notNull[i]) {
-            data[position + i] = isSigned
-                ? unZigZag(readLong())
-                : static_cast<int64_t>(readLong());
-            ++consumed;
-          }
-        }
-      } else {
-        if (isSigned) {
-          for (uint64_t i = 0; i < count; ++i) {
-            data[position + i] = unZigZag(readLong());
-          }
-        } else {
-          for (uint64_t i = 0; i < count; ++i) {
-            data[position + i] = static_cast<int64_t>(readLong());
-          }
-        }
-        consumed = count;
-      }
-    }
-    remainingValues -= consumed;
-    position += count;
-
-    // skipNulls()
-    if (notNull) {
-      // Skip over null values.
-      while (position < numValues && !notNull[position]) {
-        ++position;
-      }
-    }
-  }
-}
-
-}  // namespace orc

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/RLEv1.hh
----------------------------------------------------------------------
diff --git a/c++/src/orc/RLEv1.hh b/c++/src/orc/RLEv1.hh
deleted file mode 100644
index df43a16..0000000
--- a/c++/src/orc/RLEv1.hh
+++ /dev/null
@@ -1,70 +0,0 @@
-/**
-* 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 ORC_RLEV1_HH
-#define ORC_RLEV1_HH
-
-#include "orc/Adaptor.hh"
-#include "RLE.hh"
-
-#include <memory>
-
-namespace orc {
-
-class RleDecoderV1 : public RleDecoder {
-public:
-    RleDecoderV1(std::unique_ptr<SeekableInputStream> input,
-                 bool isSigned);
-
-    /**
-    * Seek to a particular spot.
-    */
-    void seek(PositionProvider&) override;
-
-    /**
-    * Seek over a given number of values.
-    */
-    void skip(uint64_t numValues) override;
-
-    /**
-    * Read a number of values into the batch.
-    */
-    void next(int64_t* data, uint64_t numValues,
-              const char* notNull) override;
-
-private:
-    inline signed char readByte();
-
-    inline void readHeader();
-
-    inline uint64_t readLong();
-
-    inline void skipLongs(uint64_t numValues);
-
-    const std::unique_ptr<SeekableInputStream> inputStream;
-    const bool isSigned;
-    uint64_t remainingValues;
-    int64_t value;
-    const char *bufferStart;
-    const char *bufferEnd;
-    int64_t delta;
-    bool repeating;
-};
-}  // namespace orc
-
-#endif  // ORC_RLEV1_HH

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/RLEv2.cc
----------------------------------------------------------------------
diff --git a/c++/src/orc/RLEv2.cc b/c++/src/orc/RLEv2.cc
deleted file mode 100644
index f54fa3e..0000000
--- a/c++/src/orc/RLEv2.cc
+++ /dev/null
@@ -1,482 +0,0 @@
-/**
- * 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 "orc/Adaptor.hh"
-#include "RLEv2.hh"
-#include "Compression.hh"
-
-#define MIN_REPEAT 3
-
-namespace orc {
-
-struct FixedBitSizes {
-  enum FBS {
-    ONE = 0, TWO, THREE, FOUR, FIVE, SIX, SEVEN, EIGHT, NINE, TEN, ELEVEN, TWELVE,
-    THIRTEEN, FOURTEEN, FIFTEEN, SIXTEEN, SEVENTEEN, EIGHTEEN, NINETEEN,
-    TWENTY, TWENTYONE, TWENTYTWO, TWENTYTHREE, TWENTYFOUR, TWENTYSIX,
-    TWENTYEIGHT, THIRTY, THIRTYTWO, FORTY, FORTYEIGHT, FIFTYSIX, SIXTYFOUR
-  };
-};
-
-inline uint32_t decodeBitWidth(uint32_t n) {
-  if (n >= FixedBitSizes::ONE &&
-      n <= FixedBitSizes::TWENTYFOUR) {
-    return n + 1;
-  } else if (n == FixedBitSizes::TWENTYSIX) {
-    return 26;
-  } else if (n == FixedBitSizes::TWENTYEIGHT) {
-    return 28;
-  } else if (n == FixedBitSizes::THIRTY) {
-    return 30;
-  } else if (n == FixedBitSizes::THIRTYTWO) {
-    return 32;
-  } else if (n == FixedBitSizes::FORTY) {
-    return 40;
-  } else if (n == FixedBitSizes::FORTYEIGHT) {
-    return 48;
-  } else if (n == FixedBitSizes::FIFTYSIX) {
-    return 56;
-  } else {
-    return 64;
-  }
-}
-
-inline uint32_t getClosestFixedBits(uint32_t n) {
-  if (n == 0) {
-    return 1;
-  }
-
-  if (n >= 1 && n <= 24) {
-    return n;
-  } else if (n > 24 && n <= 26) {
-    return 26;
-  } else if (n > 26 && n <= 28) {
-    return 28;
-  } else if (n > 28 && n <= 30) {
-    return 30;
-  } else if (n > 30 && n <= 32) {
-    return 32;
-  } else if (n > 32 && n <= 40) {
-    return 40;
-  } else if (n > 40 && n <= 48) {
-    return 48;
-  } else if (n > 48 && n <= 56) {
-    return 56;
-  } else {
-    return 64;
-  }
-}
-
-int64_t RleDecoderV2::readLongBE(uint64_t bsz) {
-  int64_t ret = 0, val;
-  uint64_t n = bsz;
-  while (n > 0) {
-    n--;
-    val = readByte();
-    ret |= (val << (n * 8));
-  }
-  return ret;
-}
-
-inline int64_t RleDecoderV2::readVslong() {
-  return unZigZag(readVulong());
-}
-
-uint64_t RleDecoderV2::readVulong() {
-  uint64_t ret = 0, b;
-  uint64_t offset = 0;
-  do {
-    b = readByte();
-    ret |= (0x7f & b) << offset;
-    offset += 7;
-  } while (b >= 0x80);
-  return ret;
-}
-
-RleDecoderV2::RleDecoderV2(std::unique_ptr<SeekableInputStream> input,
-                           bool _isSigned, MemoryPool& pool
-                           ): inputStream(std::move(input)),
-                              isSigned(_isSigned),
-                              firstByte(0),
-                              runLength(0),
-                              runRead(0),
-                              bufferStart(nullptr),
-                              bufferEnd(bufferStart),
-                              deltaBase(0),
-                              byteSize(0),
-                              firstValue(0),
-                              prevValue(0),
-                              bitSize(0),
-                              bitsLeft(0),
-                              curByte(0),
-                              patchBitSize(0),
-                              base(0),
-                              curGap(0),
-                              patchMask(0),
-                              actualGap(0),
-                              unpacked(pool, 0),
-                              unpackedPatch(pool, 0) {
-  // PASS
-}
-
-void RleDecoderV2::seek(PositionProvider& location) {
-  // move the input stream
-  inputStream->seek(location);
-  // clear state
-  bufferEnd = bufferStart = 0;
-  runRead = runLength = 0;
-  // skip ahead the given number of records
-  skip(location.next());
-}
-
-void RleDecoderV2::skip(uint64_t numValues) {
-  // simple for now, until perf tests indicate something encoding specific is
-  // needed
-  const uint64_t N = 64;
-  int64_t dummy[N];
-
-  while (numValues) {
-    uint64_t nRead = std::min(N, numValues);
-    next(dummy, nRead, nullptr);
-    numValues -= nRead;
-  }
-}
-
-void RleDecoderV2::next(int64_t* const data,
-                        const uint64_t numValues,
-                        const char* const notNull) {
-  uint64_t nRead = 0;
-
-  while (nRead < numValues) {
-    // Skip any nulls before attempting to read first byte.
-    while (notNull && !notNull[nRead]) {
-      if (++nRead == numValues) {
-        return; // ended with null values
-      }
-    }
-
-    if (runRead == runLength) {
-      resetRun();
-      firstByte = readByte();
-    }
-
-    uint64_t offset = nRead, length = numValues - nRead;
-
-    EncodingType enc = static_cast<EncodingType>
-        ((firstByte >> 6) & 0x03);
-    switch(static_cast<int64_t>(enc)) {
-    case SHORT_REPEAT:
-      nRead += nextShortRepeats(data, offset, length, notNull);
-      break;
-    case DIRECT:
-      nRead += nextDirect(data, offset, length, notNull);
-      break;
-    case PATCHED_BASE:
-      nRead += nextPatched(data, offset, length, notNull);
-      break;
-    case DELTA:
-      nRead += nextDelta(data, offset, length, notNull);
-      break;
-    default:
-      throw ParseError("unknown encoding");
-    }
-  }
-}
-
-uint64_t RleDecoderV2::nextShortRepeats(int64_t* const data,
-                                        uint64_t offset,
-                                        uint64_t numValues,
-                                        const char* const notNull) {
-  if (runRead == runLength) {
-    // extract the number of fixed bytes
-    byteSize = (firstByte >> 3) & 0x07;
-    byteSize += 1;
-
-    runLength = firstByte & 0x07;
-    // run lengths values are stored only after MIN_REPEAT value is met
-    runLength += MIN_REPEAT;
-    runRead = 0;
-
-    // read the repeated value which is store using fixed bytes
-    firstValue = readLongBE(byteSize);
-
-    if (isSigned) {
-      firstValue = unZigZag(static_cast<uint64_t>(firstValue));
-    }
-  }
-
-  uint64_t nRead = std::min(runLength - runRead, numValues);
-
-  if (notNull) {
-    for(uint64_t pos = offset; pos < offset + nRead; ++pos) {
-      if (notNull[pos]) {
-        data[pos] = firstValue;
-        ++runRead;
-      }
-    }
-  } else {
-    for(uint64_t pos = offset; pos < offset + nRead; ++pos) {
-      data[pos] = firstValue;
-      ++runRead;
-    }
-  }
-
-  return nRead;
-}
-
-uint64_t RleDecoderV2::nextDirect(int64_t* const data,
-                                  uint64_t offset,
-                                  uint64_t numValues,
-                                  const char* const notNull) {
-  if (runRead == runLength) {
-    // extract the number of fixed bits
-    unsigned char fbo = (firstByte >> 1) & 0x1f;
-    bitSize = decodeBitWidth(fbo);
-
-    // extract the run length
-    runLength = static_cast<uint64_t>(firstByte & 0x01) << 8;
-    runLength |= readByte();
-    // runs are one off
-    runLength += 1;
-    runRead = 0;
-  }
-
-  uint64_t nRead = std::min(runLength - runRead, numValues);
-
-  runRead += readLongs(data, offset, nRead, bitSize, notNull);
-
-  if (isSigned) {
-    if (notNull) {
-      for (uint64_t pos = offset; pos < offset + nRead; ++pos) {
-        if (notNull[pos]) {
-          data[pos] = unZigZag(static_cast<uint64_t>(data[pos]));
-        }
-      }
-    } else {
-      for (uint64_t pos = offset; pos < offset + nRead; ++pos) {
-        data[pos] = unZigZag(static_cast<uint64_t>(data[pos]));
-      }
-    }
-  }
-
-  return nRead;
-}
-
-uint64_t RleDecoderV2::nextPatched(int64_t* const data,
-                                   uint64_t offset,
-                                   uint64_t numValues,
-                                   const char* const notNull) {
-  if (runRead == runLength) {
-    // extract the number of fixed bits
-    unsigned char fbo = (firstByte >> 1) & 0x1f;
-    bitSize = decodeBitWidth(fbo);
-
-    // extract the run length
-    runLength = static_cast<uint64_t>(firstByte & 0x01) << 8;
-    runLength |= readByte();
-    // runs are one off
-    runLength += 1;
-    runRead = 0;
-
-    // extract the number of bytes occupied by base
-    uint64_t thirdByte = readByte();
-    byteSize = (thirdByte >> 5) & 0x07;
-    // base width is one off
-    byteSize += 1;
-
-    // extract patch width
-    uint32_t pwo = thirdByte & 0x1f;
-    patchBitSize = decodeBitWidth(pwo);
-
-    // read fourth byte and extract patch gap width
-    uint64_t fourthByte = readByte();
-    uint32_t pgw = (fourthByte >> 5) & 0x07;
-    // patch gap width is one off
-    pgw += 1;
-
-    // extract the length of the patch list
-    size_t pl = fourthByte & 0x1f;
-    if (pl == 0) {
-      throw ParseError("Corrupt PATCHED_BASE encoded data (pl==0)!");
-    }
-
-    // read the next base width number of bytes to extract base value
-    base = readLongBE(byteSize);
-    int64_t mask = (static_cast<int64_t>(1) << ((byteSize * 8) - 1));
-    // if mask of base value is 1 then base is negative value else positive
-    if ((base & mask) != 0) {
-      base = base & ~mask;
-      base = -base;
-    }
-
-    // TODO: something more efficient than resize
-    unpacked.resize(runLength);
-    unpackedIdx = 0;
-    readLongs(unpacked.data(), 0, runLength, bitSize);
-    // any remaining bits are thrown out
-    resetReadLongs();
-
-    // TODO: something more efficient than resize
-    unpackedPatch.resize(pl);
-    patchIdx = 0;
-    // TODO: Skip corrupt?
-    //    if ((patchBitSize + pgw) > 64 && !skipCorrupt) {
-    if ((patchBitSize + pgw) > 64) {
-      throw ParseError("Corrupt PATCHED_BASE encoded data "
-                       "(patchBitSize + pgw > 64)!");
-    }
-    uint32_t cfb = getClosestFixedBits(patchBitSize + pgw);
-    readLongs(unpackedPatch.data(), 0, pl, cfb);
-    // any remaining bits are thrown out
-    resetReadLongs();
-
-    // apply the patch directly when decoding the packed data
-    patchMask = ((static_cast<int64_t>(1) << patchBitSize) - 1);
-
-    adjustGapAndPatch();
-  }
-
-  uint64_t nRead = std::min(runLength - runRead, numValues);
-
-  for(uint64_t pos = offset; pos < offset + nRead; ++pos) {
-    // skip null positions
-    if (notNull && !notNull[pos]) {
-      continue;
-    }
-    if (static_cast<int64_t>(unpackedIdx) != actualGap) {
-      // no patching required. add base to unpacked value to get final value
-      data[pos] = base + unpacked[unpackedIdx];
-    } else {
-      // extract the patch value
-      int64_t patchedVal = unpacked[unpackedIdx] | (curPatch << bitSize);
-
-      // add base to patched value
-      data[pos] = base + patchedVal;
-
-      // increment the patch to point to next entry in patch list
-      ++patchIdx;
-
-      if (patchIdx < unpackedPatch.size()) {
-        adjustGapAndPatch();
-
-        // next gap is relative to the current gap
-        actualGap += unpackedIdx;
-      }
-    }
-
-    ++runRead;
-    ++unpackedIdx;
-  }
-
-  return nRead;
-}
-
-uint64_t RleDecoderV2::nextDelta(int64_t* const data,
-                                 uint64_t offset,
-                                 uint64_t numValues,
-                                 const char* const notNull) {
-  if (runRead == runLength) {
-    // extract the number of fixed bits
-    unsigned char fbo = (firstByte >> 1) & 0x1f;
-    if (fbo != 0) {
-      bitSize = decodeBitWidth(fbo);
-    } else {
-      bitSize = 0;
-    }
-
-    // extract the run length
-    runLength = static_cast<uint64_t>(firstByte & 0x01) << 8;
-    runLength |= readByte();
-    ++runLength; // account for first value
-    runRead = deltaBase = 0;
-
-    // read the first value stored as vint
-    if (isSigned) {
-      firstValue = static_cast<int64_t>(readVslong());
-    } else {
-      firstValue = static_cast<int64_t>(readVulong());
-    }
-
-    prevValue = firstValue;
-
-    // read the fixed delta value stored as vint (deltas can be negative even
-    // if all number are positive)
-    deltaBase = static_cast<int64_t>(readVslong());
-  }
-
-  uint64_t nRead = std::min(runLength - runRead, numValues);
-
-  uint64_t pos = offset;
-  for ( ; pos < offset + nRead; ++pos) {
-    // skip null positions
-    if (!notNull || notNull[pos]) break;
-  }
-  if (runRead == 0 && pos < offset + nRead) {
-    data[pos++] = firstValue;
-    ++runRead;
-  }
-
-  if (bitSize == 0) {
-    // add fixed deltas to adjacent values
-    for ( ; pos < offset + nRead; ++pos) {
-      // skip null positions
-      if (notNull && !notNull[pos]) {
-        continue;
-      }
-      prevValue = data[pos] = prevValue + deltaBase;
-      ++runRead;
-    }
-  } else {
-    for ( ; pos < offset + nRead; ++pos) {
-      // skip null positions
-      if (!notNull || notNull[pos]) break;
-    }
-    if (runRead < 2 && pos < offset + nRead) {
-      // add delta base and first value
-      prevValue = data[pos++] = firstValue + deltaBase;
-      ++runRead;
-    }
-
-    // write the unpacked values, add it to previous value and store final
-    // value to result buffer. if the delta base value is negative then it
-    // is a decreasing sequence else an increasing sequence
-    uint64_t remaining = (offset + nRead) - pos;
-    runRead += readLongs(data, pos, remaining, bitSize, notNull);
-
-    if (deltaBase < 0) {
-      for ( ; pos < offset + nRead; ++pos) {
-        // skip null positions
-        if (notNull && !notNull[pos]) {
-          continue;
-        }
-        prevValue = data[pos] = prevValue - data[pos];
-      }
-    } else {
-      for ( ; pos < offset + nRead; ++pos) {
-        // skip null positions
-        if (notNull && !notNull[pos]) {
-          continue;
-        }
-        prevValue = data[pos] = prevValue + data[pos];
-      }
-    }
-  }
-  return nRead;
-}
-
-}  // namespace orc

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/RLEv2.hh
----------------------------------------------------------------------
diff --git a/c++/src/orc/RLEv2.hh b/c++/src/orc/RLEv2.hh
deleted file mode 100644
index 9e5d6d5..0000000
--- a/c++/src/orc/RLEv2.hh
+++ /dev/null
@@ -1,175 +0,0 @@
-/**
-* 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 ORC_RLEV2_HH
-#define ORC_RLEV2_HH
-
-#include "orc/Adaptor.hh"
-#include "RLE.hh"
-#include "Exceptions.hh"
-
-#include <vector>
-
-namespace orc {
-
-class RleDecoderV2 : public RleDecoder {
-public:
-
-  enum EncodingType { SHORT_REPEAT=0, DIRECT=1, PATCHED_BASE=2, DELTA=3 };
-
-  RleDecoderV2(std::unique_ptr<SeekableInputStream> input,
-               bool isSigned, MemoryPool& pool);
-
-  /**
-  * Seek to a particular spot.
-  */
-  void seek(PositionProvider&) override;
-
-  /**
-  * Seek over a given number of values.
-  */
-  void skip(uint64_t numValues) override;
-
-  /**
-  * Read a number of values into the batch.
-  */
-  void next(int64_t* data, uint64_t numValues,
-            const char* notNull) override;
-
-private:
-
-  // Used by PATCHED_BASE
-  void adjustGapAndPatch() {
-    curGap = static_cast<uint64_t>(unpackedPatch[patchIdx]) >>
-      patchBitSize;
-    curPatch = unpackedPatch[patchIdx] & patchMask;
-    actualGap = 0;
-
-    // special case: gap is >255 then patch value will be 0.
-    // if gap is <=255 then patch value cannot be 0
-    while (curGap == 255 && curPatch == 0) {
-      actualGap += 255;
-      ++patchIdx;
-      curGap = static_cast<uint64_t>(unpackedPatch[patchIdx]) >>
-        patchBitSize;
-      curPatch = unpackedPatch[patchIdx] & patchMask;
-    }
-    // add the left over gap
-    actualGap += curGap;
-  }
-
-  void resetReadLongs() {
-    bitsLeft = 0;
-    curByte = 0;
-  }
-
-  void resetRun() {
-    resetReadLongs();
-    bitSize = 0;
-  }
-
-  unsigned char readByte() {
-  if (bufferStart == bufferEnd) {
-    int bufferLength;
-    const void* bufferPointer;
-    if (!inputStream->Next(&bufferPointer, &bufferLength)) {
-      throw ParseError("bad read in RleDecoderV2::readByte");
-    }
-    bufferStart = static_cast<const char*>(bufferPointer);
-    bufferEnd = bufferStart + bufferLength;
-  }
-
-  unsigned char result = static_cast<unsigned char>(*bufferStart++);
-  return result;
-}
-
-  int64_t readLongBE(uint64_t bsz);
-  int64_t readVslong();
-  uint64_t readVulong();
-  uint64_t readLongs(int64_t *data, uint64_t offset, uint64_t len,
-                     uint64_t fb, const char* notNull = nullptr) {
-  uint64_t ret = 0;
-
-  // TODO: unroll to improve performance
-  for(uint64_t i = offset; i < (offset + len); i++) {
-    // skip null positions
-    if (notNull && !notNull[i]) {
-      continue;
-    }
-    uint64_t result = 0;
-    uint64_t bitsLeftToRead = fb;
-    while (bitsLeftToRead > bitsLeft) {
-      result <<= bitsLeft;
-      result |= curByte & ((1 << bitsLeft) - 1);
-      bitsLeftToRead -= bitsLeft;
-      curByte = readByte();
-      bitsLeft = 8;
-    }
-
-    // handle the left over bits
-    if (bitsLeftToRead > 0) {
-      result <<= bitsLeftToRead;
-      bitsLeft -= static_cast<uint32_t>(bitsLeftToRead);
-      result |= (curByte >> bitsLeft) & ((1 << bitsLeftToRead) - 1);
-    }
-    data[i] = static_cast<int64_t>(result);
-    ++ret;
-  }
-
-  return ret;
-}
-
-
-  uint64_t nextShortRepeats(int64_t* data, uint64_t offset, uint64_t numValues,
-                            const char* notNull);
-  uint64_t nextDirect(int64_t* data, uint64_t offset, uint64_t numValues,
-                      const char* notNull);
-  uint64_t nextPatched(int64_t* data, uint64_t offset, uint64_t numValues,
-                       const char* notNull);
-  uint64_t nextDelta(int64_t* data, uint64_t offset, uint64_t numValues,
-                     const char* notNull);
-
-  const std::unique_ptr<SeekableInputStream> inputStream;
-  const bool isSigned;
-
-  unsigned char firstByte;
-  uint64_t runLength;
-  uint64_t runRead;
-  const char *bufferStart;
-  const char *bufferEnd;
-  int64_t deltaBase; // Used by DELTA
-  uint64_t byteSize; // Used by SHORT_REPEAT and PATCHED_BASE
-  int64_t firstValue; // Used by SHORT_REPEAT and DELTA
-  int64_t prevValue; // Used by DELTA
-  uint32_t bitSize; // Used by DIRECT, PATCHED_BASE and DELTA
-  uint32_t bitsLeft; // Used by anything that uses readLongs
-  uint32_t curByte; // Used by anything that uses readLongs
-  uint32_t patchBitSize; // Used by PATCHED_BASE
-  uint64_t unpackedIdx; // Used by PATCHED_BASE
-  uint64_t patchIdx; // Used by PATCHED_BASE
-  int64_t base; // Used by PATCHED_BASE
-  uint64_t curGap; // Used by PATCHED_BASE
-  int64_t curPatch; // Used by PATCHED_BASE
-  int64_t patchMask; // Used by PATCHED_BASE
-  int64_t actualGap; // Used by PATCHED_BASE
-  DataBuffer<int64_t> unpacked; // Used by PATCHED_BASE
-  DataBuffer<int64_t> unpackedPatch; // Used by PATCHED_BASE
-};
-}  // namespace orc
-
-#endif  // ORC_RLEV2_HH


[03/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools-c++/test/ToolTest.cc
----------------------------------------------------------------------
diff --git a/tools-c++/test/ToolTest.cc b/tools-c++/test/ToolTest.cc
deleted file mode 100644
index b4ddfbc..0000000
--- a/tools-c++/test/ToolTest.cc
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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 "orc/orc-config.hh"
-#include "ToolTest.hh"
-
-#include "wrap/orc-proto-wrapper.hh"
-#include "wrap/gtest-wrapper.h"
-
-
-#include <iostream>
-
-const char *exampleDirectory = 0;
-
-GTEST_API_ int main(int argc, char **argv) {
-  GOOGLE_PROTOBUF_VERIFY_VERSION;
-  std::cout << "ORC version: " << ORC_VERSION << "\n";
-  if (argc == 2) {
-    exampleDirectory = argv[1];
-  } else {
-    exampleDirectory = "../examples";
-  }
-  std::cout << "example dir = " << exampleDirectory << "\n";
-  testing::InitGoogleTest(&argc, argv);
-  int result = RUN_ALL_TESTS();
-  return result;
-}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools-c++/test/ToolTest.hh
----------------------------------------------------------------------
diff --git a/tools-c++/test/ToolTest.hh b/tools-c++/test/ToolTest.hh
deleted file mode 100644
index 48720b0..0000000
--- a/tools-c++/test/ToolTest.hh
+++ /dev/null
@@ -1,21 +0,0 @@
-/**
- * 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 <string>
-
-extern const char* exampleDirectory;

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools-c++/test/gzip.cc
----------------------------------------------------------------------
diff --git a/tools-c++/test/gzip.cc b/tools-c++/test/gzip.cc
deleted file mode 100644
index 585393c..0000000
--- a/tools-c++/test/gzip.cc
+++ /dev/null
@@ -1,115 +0,0 @@
-/**
- * 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 "orc/Adaptor.hh"
-#include "gzip.hh"
-
-#include <iostream>
-#include <stdexcept>
-
-#ifdef __clang__
-#pragma clang diagnostic ignored "-Wold-style-cast"
-#endif
-
-namespace orc {
-
-  GzipTextReader::GzipTextReader(const std::string& _filename
-                                 ): filename(_filename) {
-    file = fopen(filename.c_str(), "rb");
-    if (file == nullptr) {
-      throw std::runtime_error("can't open " + filename);
-    }
-    stream.zalloc = Z_NULL;
-    stream.zfree = Z_NULL;
-    stream.opaque = Z_NULL;
-    stream.avail_in = 0;
-    stream.avail_out = 1;
-    stream.next_in = Z_NULL;
-    int ret = inflateInit2(&stream, 16+MAX_WBITS);
-    if (ret != Z_OK) {
-      throw std::runtime_error("zlib failed initialization for " + filename);
-    }
-    outPtr = nullptr;
-    outEnd = nullptr;
-    isDone = false;
-  }
-
-  bool GzipTextReader::nextBuffer() {
-    // if we are done, return
-    if (isDone) {
-      return false;
-    }
-    // if the last read is done, read more
-    if (stream.avail_in == 0 && stream.avail_out != 0) {
-      stream.next_in = input;
-      stream.avail_in = static_cast<uint>(fread(input, 1, sizeof(input),
-                                                file));
-      if (ferror(file)) {
-        throw std::runtime_error("failure reading " + filename);
-      }
-    }
-    stream.avail_out = sizeof(output);
-    stream.next_out = output;
-    int ret = inflate(&stream, Z_NO_FLUSH);
-    switch (ret) {
-    case Z_OK:
-      break;
-    case Z_STREAM_END:
-      isDone = true;
-      break;
-    case Z_STREAM_ERROR:
-      throw std::runtime_error("zlib stream problem");
-    case Z_NEED_DICT:
-    case Z_DATA_ERROR:
-      throw std::runtime_error("zlib data problem");
-    case Z_MEM_ERROR:
-      throw std::runtime_error("zlib memory problem");
-    case Z_BUF_ERROR:
-      throw std::runtime_error("zlib buffer problem");
-    default:
-      throw std::runtime_error("zlib unknown problem");
-    }
-    outPtr = output;
-    outEnd = output + (sizeof(output) - stream.avail_out);
-    return true;
-  }
-
-  bool GzipTextReader::nextLine(std::string& line) {
-    bool result = false;
-    line.clear();
-    while (true) {
-      if (outPtr == outEnd) {
-        if (!nextBuffer()) {
-          return result;
-        }
-      }
-      unsigned char ch = *(outPtr++);
-      if (ch == '\n') {
-        return true;
-      }
-      line += static_cast<char>(ch);
-    }
-  }
-
-  GzipTextReader::~GzipTextReader() {
-    inflateEnd(&stream);
-    if (fclose(file) != 0) {
-      std::cerr << "can't close file " << filename;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools-c++/test/gzip.hh
----------------------------------------------------------------------
diff --git a/tools-c++/test/gzip.hh b/tools-c++/test/gzip.hh
deleted file mode 100644
index 4efebed..0000000
--- a/tools-c++/test/gzip.hh
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * 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 ORC_GZIP
-#define ORC_GZIP
-
-#include <string>
-#include <stdio.h>
-#include "zlib.h"
-
-namespace orc {
-
-  class GzipTextReader {
-  private:
-    std::string filename;
-    FILE* file;
-    z_stream stream;
-    unsigned char input[64 * 1024];
-    unsigned char output[64 * 1024];
-    unsigned char *outPtr;
-    unsigned char *outEnd;
-    bool isDone;
-
-    bool nextBuffer();
-
-    // NOT IMPLEMENTED
-    GzipTextReader(const GzipTextReader&);
-    GzipTextReader& operator=(const GzipTextReader&);
-
-  public:
-    GzipTextReader(const std::string& filename);
-    ~GzipTextReader();
-    bool nextLine(std::string& line);
-  };
-
-}
-#endif

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/tools/CMakeLists.txt b/tools/CMakeLists.txt
new file mode 100644
index 0000000..9606d3b
--- /dev/null
+++ b/tools/CMakeLists.txt
@@ -0,0 +1,14 @@
+# Licensed 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.
+
+add_subdirectory(src)
+add_subdirectory(test)

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/tools/src/CMakeLists.txt b/tools/src/CMakeLists.txt
new file mode 100644
index 0000000..bdcbba2
--- /dev/null
+++ b/tools/src/CMakeLists.txt
@@ -0,0 +1,64 @@
+# Licensed 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_directories (
+  ${PROJECT_SOURCE_DIR}/c++/include
+  ${PROJECT_SOURCE_DIR}/c++/src
+  ${PROJECT_BINARY_DIR}/c++/include
+  ${PROJECT_BINARY_DIR}/c++/src
+  ${PROTOBUF_INCLUDE_DIRS}
+  )
+
+set (CMAKE_CXX_FLAGS "-O0 ${CMAKE_CXX_FLAGS} -g ${CXX11_FLAGS} ${WARN_FLAGS}")
+
+add_executable (file-contents
+  FileContents.cc
+  )
+
+target_link_libraries (file-contents
+  orc
+  ${PROTOBUF_LIBRARIES}
+  )
+
+add_executable (file-scan
+  FileScan.cc
+  )
+
+target_link_libraries (file-scan
+  orc
+  ${PROTOBUF_LIBRARIES}
+  )
+
+add_executable (file-metadata
+  FileMetadata.cc
+  )
+
+target_link_libraries (file-metadata
+  orc
+  ${PROTOBUF_LIBRARIES}
+  )
+  
+ add_executable (file-statistics
+  FileStatistics.cc
+  )
+
+target_link_libraries (file-statistics
+  orc
+  ${PROTOBUF_LIBRARIES}
+  ) 
+
+install(TARGETS
+   file-contents
+   file-metadata
+   file-scan
+   file-statistics
+   DESTINATION bin)

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools/src/FileContents.cc
----------------------------------------------------------------------
diff --git a/tools/src/FileContents.cc b/tools/src/FileContents.cc
new file mode 100644
index 0000000..afda84a
--- /dev/null
+++ b/tools/src/FileContents.cc
@@ -0,0 +1,63 @@
+/**
+ * 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 "orc/ColumnPrinter.hh"
+
+#include "Exceptions.hh"
+
+#include <memory>
+#include <string>
+#include <iostream>
+#include <string>
+
+int main(int argc, char* argv[]) {
+  if (argc < 2) {
+    std::cout << "Usage: file-contents <filename>\n";
+    return 1;
+  }
+  orc::ReaderOptions opts;
+  std::list<int64_t> cols;
+  cols.push_back(0);
+  opts.include(cols);
+
+  std::unique_ptr<orc::Reader> reader;
+  try{
+    reader = orc::createReader(orc::readLocalFile(std::string(argv[1])), opts);
+  } catch (orc::ParseError e) {
+    std::cout << "Error reading file " << argv[1] << "! "
+              << e.what() << std::endl;
+    return -1;
+  }
+
+  std::unique_ptr<orc::ColumnVectorBatch> batch = reader->createRowBatch(1000);
+  std::string line;
+  std::unique_ptr<orc::ColumnPrinter> printer =
+    createColumnPrinter(line, reader->getType());
+
+  while (reader->next(*batch)) {
+    printer->reset(*batch);
+    for(unsigned long i=0; i < batch->numElements; ++i) {
+      line.clear();
+      printer->printRow(i);
+      line += "\n";
+      const char* str = line.c_str();
+      fwrite(str, 1, strlen(str), stdout);
+    }
+  }
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools/src/FileMetadata.cc
----------------------------------------------------------------------
diff --git a/tools/src/FileMetadata.cc b/tools/src/FileMetadata.cc
new file mode 100644
index 0000000..d193ad8
--- /dev/null
+++ b/tools/src/FileMetadata.cc
@@ -0,0 +1,181 @@
+/**
+ * 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 <iostream>
+#include <fstream>
+#include <vector>
+#include <string>
+#include <sstream>
+#include <iomanip>
+
+#include "wrap/orc-proto-wrapper.hh"
+#include "orc/OrcFile.hh"
+
+using namespace orc::proto;
+
+uint64_t getTotalPaddingSize(Footer footer);
+
+int main(int argc, char* argv[])
+{
+  std::ifstream input;
+
+  GOOGLE_PROTOBUF_VERIFY_VERSION;
+
+  if (argc < 2) {
+    std::cout << "Usage: file-metadata <filename>\n";
+  }
+
+  std::cout << "Structure for " << argv[1] << std::endl;
+
+  input.open(argv[1], std::ios::in | std::ios::binary);
+  input.seekg(0,input.end);
+  std::streamoff fileSize = input.tellg();
+
+  // Read the postscript size
+  input.seekg(fileSize-1);
+  int result = input.get();
+  if (result == EOF) {
+    std::cerr << "Failed to read postscript size\n";
+    return -1;
+  }
+  std::streamoff postscriptSize = result;
+
+  // Read the postscript
+  input.seekg(fileSize - postscriptSize-1);
+  std::vector<char> buffer(static_cast<size_t>(postscriptSize));
+  input.read(buffer.data(), postscriptSize);
+  PostScript postscript ;
+  postscript.ParseFromArray(buffer.data(),
+                            static_cast<int>(postscriptSize));
+  std::cout << std::endl << " === Postscript === " << std::endl ;
+  postscript.PrintDebugString();
+
+  // Everything but the postscript is compressed
+  switch (static_cast<int>(postscript.compression())) {
+  case NONE:
+      break;
+  case ZLIB:
+  case SNAPPY:
+  case LZO:
+  default:
+      std::cout << "ORC files with compression are not supported" << std::endl ;
+      input.close();
+      return -1;
+  };
+
+  std::streamoff footerSize =
+    static_cast<std::streamoff>(postscript.footerlength());
+  std::streamoff metadataSize =
+    static_cast<std::streamoff>(postscript.metadatalength());
+
+  // Read the metadata
+  input.seekg(fileSize - 1 - postscriptSize - footerSize - metadataSize);
+  buffer.resize(static_cast<size_t>(metadataSize));
+  input.read(buffer.data(), metadataSize);
+  Metadata metadata ;
+  metadata.ParseFromArray(buffer.data(), static_cast<int>(metadataSize));
+
+  // Read the footer
+  //input.seekg(fileSize -1 - postscriptSize-footerSize);
+  buffer.resize(static_cast<size_t>(footerSize));
+  input.read(buffer.data(), footerSize);
+  Footer footer ;
+  footer.ParseFromArray(buffer.data(), static_cast<int>(footerSize));
+  std::cout << std::endl << " === Footer === " << std::endl ;
+  footer.PrintDebugString();
+
+  std::cout << std::endl << "=== Stripe Statistics ===" << std::endl;
+
+  StripeInformation stripe ;
+  Stream section;
+  ColumnEncoding encoding;
+  for (int stripeIx=0; stripeIx<footer.stripes_size(); stripeIx++)
+  {
+      std::cout << "Stripe " << stripeIx+1 <<": " << std::endl ;
+      stripe = footer.stripes(stripeIx);
+      stripe.PrintDebugString();
+
+      std::streamoff offset =
+        static_cast<std::streamoff>(stripe.offset() + stripe.indexlength() +
+                                    stripe.datalength());
+      std::streamoff tailLength =
+        static_cast<std::streamoff>(stripe.footerlength());
+
+      // read the stripe footer
+      input.seekg(offset);
+      buffer.resize(static_cast<size_t>(tailLength));
+      input.read(buffer.data(), tailLength);
+
+      StripeFooter stripeFooter;
+      stripeFooter.ParseFromArray(buffer.data(), static_cast<int>(tailLength));
+      //stripeFooter.PrintDebugString();
+      uint64_t stripeStart = stripe.offset();
+      uint64_t sectionStart = stripeStart;
+      for (int streamIx=0; streamIx<stripeFooter.streams_size(); streamIx++) {
+          section = stripeFooter.streams(streamIx);
+          std::cout << "    Stream: column " << section.column()
+                    << " section "
+                    << section.kind() << " start: " << sectionStart
+                    << " length " << section.length() << std::endl;
+          sectionStart += section.length();
+      };
+      for (int columnIx=0; columnIx<stripeFooter.columns_size();
+           columnIx++) {
+          encoding = stripeFooter.columns(columnIx);
+          std::cout << "    Encoding column " << columnIx << ": "
+                    << encoding.kind() ;
+          if (encoding.kind() == ColumnEncoding_Kind_DICTIONARY ||
+              encoding.kind() == ColumnEncoding_Kind_DICTIONARY_V2)
+              std::cout << "[" << encoding.dictionarysize() << "]";
+          std::cout << std::endl;
+      };
+  };
+
+  uint64_t paddedBytes = getTotalPaddingSize(footer);
+  // empty ORC file is ~45 bytes. Assumption here is file length always >0
+  double percentPadding =
+    static_cast<double>(paddedBytes) * 100 / static_cast<double>(fileSize);
+  std::cout << "File length: " << fileSize << " bytes" << std::endl;
+  std::cout <<"Padding length: " << paddedBytes << " bytes" << std::endl;
+  std::cout <<"Padding ratio: " << std::fixed << std::setprecision(2)
+            << percentPadding << " %" << std::endl;
+
+  input.close();
+
+
+
+  google::protobuf::ShutdownProtobufLibrary();
+
+  return 0;
+}
+
+uint64_t getTotalPaddingSize(Footer footer) {
+  uint64_t paddedBytes = 0;
+  StripeInformation stripe;
+  for (int stripeIx=1; stripeIx<footer.stripes_size(); stripeIx++) {
+      stripe = footer.stripes(stripeIx-1);
+      uint64_t prevStripeOffset = stripe.offset();
+      uint64_t prevStripeLen = stripe.datalength() + stripe.indexlength() +
+        stripe.footerlength();
+      paddedBytes += footer.stripes(stripeIx).offset() -
+        (prevStripeOffset + prevStripeLen);
+  };
+  return paddedBytes;
+}
+
+

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools/src/FileScan.cc
----------------------------------------------------------------------
diff --git a/tools/src/FileScan.cc b/tools/src/FileScan.cc
new file mode 100644
index 0000000..85b7617
--- /dev/null
+++ b/tools/src/FileScan.cc
@@ -0,0 +1,57 @@
+/**
+ * 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 "orc/ColumnPrinter.hh"
+
+#include "Exceptions.hh"
+
+#include <string>
+#include <memory>
+#include <iostream>
+#include <string>
+
+int main(int argc, char* argv[]) {
+  if (argc < 2) {
+    std::cout << "Usage: file-scan <filename>\n";
+  }
+
+  orc::ReaderOptions opts;
+  std::list<int64_t> cols;
+  cols.push_back(0);
+  opts.include(cols);
+
+  std::unique_ptr<orc::Reader> reader;
+  try{
+    reader = orc::createReader(orc::readLocalFile(std::string(argv[1])), opts);
+  } catch (orc::ParseError e) {
+    std::cout << "Error reading file " << argv[1] << "! "
+              << e.what() << std::endl;
+    return -1;
+  }
+
+  std::unique_ptr<orc::ColumnVectorBatch> batch = reader->createRowBatch(1000);
+  unsigned long rows = 0;
+  unsigned long batches = 0;
+  while (reader->next(*batch)) {
+    batches += 1;
+    rows += batch->numElements;
+  }
+  std::cout << "Rows: " << rows << std::endl;
+  std::cout << "Batches: " << batches << std::endl;
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools/src/FileStatistics.cc
----------------------------------------------------------------------
diff --git a/tools/src/FileStatistics.cc b/tools/src/FileStatistics.cc
new file mode 100644
index 0000000..5ae677b
--- /dev/null
+++ b/tools/src/FileStatistics.cc
@@ -0,0 +1,75 @@
+/**
+ * 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 "orc/ColumnPrinter.hh"
+#include "Exceptions.hh"
+
+#include <string>
+#include <memory>
+#include <iostream>
+#include <string>
+
+int main(int argc, char* argv[]) {
+  if (argc < 2) {
+    std::cout << "Usage: file-metadata <filename>\n";
+  }
+
+  orc::ReaderOptions opts;
+  std::list<int64_t> cols;
+  cols.push_back(0);
+  opts.include(cols);
+
+  std::unique_ptr<orc::Reader> reader;
+  try{
+    reader = orc::createReader(orc::readLocalFile(std::string(argv[1])), opts);
+  } catch (orc::ParseError e) {
+    std::cout << "Error reading file " << argv[1] << "! "
+              << e.what() << std::endl;
+    return -1;
+  }
+
+  // print out all selected columns statistics.
+  std::unique_ptr<orc::Statistics> colStats = reader->getStatistics();
+  std::cout << "File " << argv[1] << " has "
+            << colStats->getNumberOfColumns() << " columns"  << std::endl;
+  for(uint32_t i=0; i < colStats->getNumberOfColumns(); ++i) {
+    std::cout << "*** Column " << i << " ***" << std::endl;
+    std::cout << colStats->getColumnStatistics(i)->toString() << std::endl;
+  }
+
+  // test stripe statistics
+  std::unique_ptr<orc::Statistics> stripeStats;
+  std::cout << "File " << argv[1] << " has " << reader->getNumberOfStripes()
+            << " stripes"  << std::endl;
+  if(reader->getNumberOfStripeStatistics() == 0){
+    std::cout << "File " << argv[1] << " doesn't have stripe statistics"  << std::endl;
+  }else{
+    for (unsigned int j = 0; j < reader->getNumberOfStripeStatistics(); j++) {
+      stripeStats = reader->getStripeStatistics(j);
+      std::cout << "*** Stripe " << j << " ***" << std::endl << std::endl ;
+
+      for(unsigned int k = 0; k < stripeStats->getNumberOfColumns(); ++k) {
+        std::cout << "--- Column " << k << " ---" << std::endl;
+        std::cout << stripeStats->getColumnStatistics(k)->toString()
+                  << std::endl;
+      }
+    }
+  }
+
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools/test/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/tools/test/CMakeLists.txt b/tools/test/CMakeLists.txt
new file mode 100644
index 0000000..ade81ff
--- /dev/null
+++ b/tools/test/CMakeLists.txt
@@ -0,0 +1,40 @@
+# Licensed 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_directories(
+  ${PROJECT_SOURCE_DIR}/c++/include
+  ${PROJECT_SOURCE_DIR}/c++/src
+  ${PROJECT_SOURCE_DIR}/tools-c++/src
+  ${PROJECT_BINARY_DIR}/c++/include
+  ${PROJECT_BINARY_DIR}/c++/src
+  ${PROTOBUF_INCLUDE_DIRS}
+  ${ZLIB_INCLUDE_DIRS}
+  ${GMOCK_INCLUDE_DIRS}
+)
+
+set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${CXX11_FLAGS} ${WARN_FLAGS}")
+
+add_executable (tool-test
+  gzip.cc
+  TestReader.cc
+  ToolTest.cc
+)
+
+target_link_libraries (tool-test
+  orc
+  ${PROTOBUF_LIBRARIES}
+  ${GMOCK_LIBRARIES}
+  ${ZLIB_LIBRARIES}
+  ${SNAPPY_LIBRARIES}
+)
+
+add_test (tool-test tool-test ${EXAMPLE_DIRECTORY})


[10/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/test/orc/OrcTest.hh
----------------------------------------------------------------------
diff --git a/c++/test/orc/OrcTest.hh b/c++/test/orc/OrcTest.hh
deleted file mode 100644
index adef344..0000000
--- a/c++/test/orc/OrcTest.hh
+++ /dev/null
@@ -1,25 +0,0 @@
-/**
- * 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 ORC_TEST_HH
-#define ORC_TEST_HH
-
-// determine the size of an array
-#define ARRAY_SIZE(array) (sizeof(array)/sizeof(*array))
-
-#endif

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/test/orc/TestByteRle.cc
----------------------------------------------------------------------
diff --git a/c++/test/orc/TestByteRle.cc b/c++/test/orc/TestByteRle.cc
deleted file mode 100644
index 4f6faee..0000000
--- a/c++/test/orc/TestByteRle.cc
+++ /dev/null
@@ -1,1385 +0,0 @@
-/**
- * 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 "orc/Adaptor.hh"
-#include "orc/ByteRLE.hh"
-#include "wrap/gtest-wrapper.h"
-#include "OrcTest.hh"
-
-#include <iostream>
-#include <vector>
-
-namespace orc {
-
-TEST(ByteRle, simpleTest) {
-  const unsigned char buffer[] = {0x61, 0x00, 0xfd, 0x44, 0x45, 0x46};
-  std::unique_ptr<ByteRleDecoder> rle =
-      createByteRleDecoder(
-        std::unique_ptr<SeekableInputStream>
-        (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer))));
-  std::vector<char> data(103);
-  rle->next(data.data(), data.size(), nullptr);
-
-  for(size_t i = 0; i < 100; ++i) {
-    EXPECT_EQ(0, data[i]) << "Output wrong at " << i;
-  }
-  EXPECT_EQ(0x44, data[100]);
-  EXPECT_EQ(0x45, data[101]);
-  EXPECT_EQ(0x46, data[102]);
-}
-
-TEST(ByteRle, nullTest) {
-  char buffer[258];
-  char notNull[266];
-  char result[266];
-  buffer[0] = -128;
-  buffer[129] = -128;
-  for(int i=0; i < 128; ++i) {
-    buffer[1 + i] = static_cast<char>(i);
-    buffer[130 + i] = static_cast<char>(128 + i);
-  }
-  for(int i=0; i < 266; ++i) {
-    notNull[i] = static_cast<char>(i >= 10);
-  }
-  std::unique_ptr<ByteRleDecoder> rle =
-    createByteRleDecoder(std::unique_ptr<SeekableInputStream>
-                         (new SeekableArrayInputStream(buffer,
-                                                       sizeof(buffer))));
-  rle->next(result, sizeof(result), notNull);
-  for(size_t i = 0; i < sizeof(result); ++i) {
-    if (i >= 10) {
-      EXPECT_EQ((i - 10) & 0xff,
-                static_cast<int>(result[i]) & 0xff) << "Output wrong at " << i;
-    }
-  }
-}
-
-TEST(ByteRle, literalCrossBuffer) {
-  const unsigned char buffer[] = {0xf6, 0x00, 0x01, 0x02, 0x03, 0x04,
-				  0x05, 0x06, 0x07, 0x08, 0x09, 0x07, 0x10};
-  std::unique_ptr<ByteRleDecoder> rle =
-      createByteRleDecoder(
-        std::unique_ptr<SeekableInputStream>
-        (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer), 6)));
-  std::vector<char> data(20);
-  rle->next(data.data(), data.size(), nullptr);
-
-  for(size_t i = 0; i < 10; ++i) {
-    EXPECT_EQ(i, data[i]) << "Output wrong at " << i;
-  }
-  for(size_t i = 10; i < 20; ++i) {
-    EXPECT_EQ(16, data[i]) << "Output wrong at " << i;
-  }
-}
-
-TEST(ByteRle, skipLiteralBufferUnderflowTest) {
-  const unsigned char buffer[] =
-    {0xf8, 0x0, 0x1, 0x2, 0x3, 0x4, 0x5, 0x6, 0x7};
-  std::unique_ptr<ByteRleDecoder> rle =
-      createByteRleDecoder(
-        std::unique_ptr<SeekableInputStream>(
-          new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer), 4)));
-  std::vector<char> data(8);
-  rle->next(data.data(), 3, nullptr);
-  EXPECT_EQ(0x0, data[0]);
-  EXPECT_EQ(0x1, data[1]);
-  EXPECT_EQ(0x2, data[2]);
-
-  rle->skip(2);
-  rle->next(data.data(), 3, nullptr);
-  EXPECT_EQ(0x5, data[0]);
-  EXPECT_EQ(0x6, data[1]);
-  EXPECT_EQ(0x7, data[2]);
-}
-
-TEST(ByteRle, simpleRuns) {
-  const unsigned char buffer[] = {0x0d, 0xff, 0x0d, 0xfe, 0x0d, 0xfd};
-  std::unique_ptr<ByteRleDecoder> rle =
-      createByteRleDecoder(
-        std::unique_ptr<SeekableInputStream>
-	(new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer))));
-  std::vector<char> data(16);
-  for (size_t i = 0; i < 3; ++i) {
-    rle->next(data.data(), data.size(), nullptr);
-    for (size_t j = 0; j < data.size(); ++j) {
-      EXPECT_EQ(static_cast<char>(-1 - static_cast<int>(i)), data[j])
-        << "Output wrong at " << (16 * i + j);
-    }
-  }
-}
-
-TEST(ByteRle, splitHeader) {
-  const unsigned char buffer[] = {0x00, 0x01, 0xe0, 0x01, 0x02, 0x03, 0x04,
-				  0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b,
-				  0x0c, 0x0d, 0x0e, 0x0f, 0x10, 0x11, 0x12,
-				  0x13, 0x14, 0x15, 0x16, 0x17, 0x18, 0x19,
-				  0x1a, 0x1b, 0x1c, 0x1d, 0x1e, 0x1f, 0x20};
-  std::unique_ptr<ByteRleDecoder> rle =
-      createByteRleDecoder(
-        std::unique_ptr<orc::SeekableInputStream>
-	(new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer), 1)));
-  std::vector<char> data(35);
-  rle->next(data.data(), data.size(), nullptr);
-  for (size_t i = 0; i < 3; ++i) {
-    EXPECT_EQ(1, data[i]) << "Output wrong at " << i;
-  }
-  for (size_t i = 3; i < data.size(); ++i) {
-    EXPECT_EQ(i-2, data[i]) << "Output wrong at " << i;
-  }
-}
-
-TEST(ByteRle, splitRuns) {
-  const unsigned char buffer[] =
-    {0x0d, 0x02, 0xf0, 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08,
-     0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10};
-  std::unique_ptr<ByteRleDecoder> rle =
-      createByteRleDecoder(
-        std::unique_ptr<orc::SeekableInputStream>
-	(new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer))));
-  std::vector<char> data(5);
-  for (size_t i = 0; i < 3; ++i) {
-    rle->next(data.data(), data.size(), nullptr);
-    for (size_t j = 0; j < data.size(); ++j) {
-      EXPECT_EQ(2, data[j]) << "Output wrong at " << (i * data.size() + j);
-    }
-  }
-  rle->next(data.data(), data.size(), nullptr);
-  EXPECT_EQ(2, data[0]) << "Output wrong at 15";
-  for (size_t i = 1; i < data.size(); ++i) {
-    EXPECT_EQ(i, data[i]) << "Output wrong at " << (15 + i);
-  }
-  for(size_t i = 0; i < 2; ++i) {
-    rle->next(data.data(), data.size(), nullptr);
-    for(size_t j = 0; j < data.size(); ++j) {
-      EXPECT_EQ(5 * i + j + data.size(), data[j])
-        << "Output wrong at " << (20 + data.size() * i + j);
-    }
-  }
-  rle->next(data.data(), 2, nullptr);
-  EXPECT_EQ(15, data[0]) << "Output wrong at 30" ;
-  EXPECT_EQ(16, data[1]) << "Output wrong at 31" ;
-}
-
-TEST(ByteRle, testNulls) {
-  const unsigned char buffer[] = 
-    {0xf0, 0x00, 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07,
-     0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x3d, 0xdc};
-  std::unique_ptr<ByteRleDecoder> rle =
-      createByteRleDecoder(
-        std::unique_ptr<orc::SeekableInputStream>
-	(new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer), 3)));
-  std::vector<char> data(16, -1);
-  std::vector<char> notNull(data.size());
-  for (size_t i = 0; i < data.size(); ++i) {
-    notNull[i] = (i + 1) % 2;
-  }
-  for (size_t i = 0; i < 2; ++i) {
-    rle->next(data.data(), data.size(), notNull.data());
-    for (size_t j = 0; j < data.size(); ++j) {
-      if (j % 2 == 0) {
-        EXPECT_EQ((i*data.size() + j)/2, data[j]) << "Output wrong at "
-                                                  << (i * data.size() + j);
-      } else {
-        EXPECT_EQ(-1, data[j]) << "Output wrong at "
-                               << (i * data.size() + j);
-      }
-    }
-  }
-  for (size_t i = 0; i < 8; ++i) {
-    rle->next(data.data(), data.size(), notNull.data());
-    for (size_t j = 0; j < data.size(); ++j) {
-      EXPECT_EQ(j % 2 == 0 ? -36 : -1,
-                data[j])
-          << "Output wrong at " << (i * data.size() + j + 32);
-    }
-  }
-}
-
-TEST(ByteRle, testAllNulls) {
-  const unsigned char buffer[] =
-    {0xf0, 0x00, 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07,
-     0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x3d, 0xdc};
-  std::unique_ptr<ByteRleDecoder> rle =
-      createByteRleDecoder(
-        std::unique_ptr<orc::SeekableInputStream>
-	(new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer))));
-  std::vector<char> data(16, -1);
-  std::vector<char> allNull(data.size(), 0);
-  std::vector<char> noNull(data.size(), 1);
-  rle->next(data.data(), data.size(), allNull.data());
-  for (size_t i = 0; i < data.size(); ++i) {
-    EXPECT_EQ(-1, data[i]) << "Output wrong at " << i;
-  }
-  rle->next(data.data(), data.size(), noNull.data());
-  for (size_t i = 0; i < data.size(); ++i) {
-    EXPECT_EQ(i, data[i]) << "Output wrong at " << i;
-    data[i] = -1;
-  }
-  rle->next(data.data(), data.size(), allNull.data());
-  for (size_t i = 0; i < data.size(); ++i) {
-    EXPECT_EQ(-1, data[i]) << "Output wrong at " << i;
-  }
-  for (size_t i = 0; i < 4; ++i) {
-    rle->next(data.data(), data.size(), noNull.data());
-    for (size_t j = 0; j < data.size(); ++j) {
-      EXPECT_EQ(-36, data[j]) << "Output wrong at " << i;
-    }
-  }
-  rle->next(data.data(), data.size(), allNull.data());
-}
-
-TEST(ByteRle, testSkip) {
-  // the stream generated by Java's TestRunLengthByteReader.testSkips
-  // for (int i = 0; i < 2048; ++i) {
-  //   if (i < 1024) {
-  //     out.write(i / 16);
-  //   } else {
-  //     out.write(i % 256);
-  //   }
-  // }
-  const unsigned char buffer[] = {
- 0xd,  0x0,  0xd,  0x1,  0xd,  0x2,  0xd,  0x3,  0xd,  0x4,  0xd,  0x5,  0xd,
- 0x6,  0xd,  0x7,  0xd,  0x8,  0xd,  0x9,  0xd,  0xa,  0xd,  0xb,  0xd,  0xc,
- 0xd,  0xd,  0xd,  0xe,  0xd,  0xf,  0xd, 0x10,  0xd, 0x11,  0xd, 0x12,  0xd,
-0x13,  0xd, 0x14,  0xd, 0x15,  0xd, 0x16,  0xd, 0x17,  0xd, 0x18,  0xd, 0x19,
- 0xd, 0x1a,  0xd, 0x1b,  0xd, 0x1c,  0xd, 0x1d,  0xd, 0x1e,  0xd, 0x1f,  0xd,
-0x20,  0xd, 0x21,  0xd, 0x22,  0xd, 0x23,  0xd, 0x24,  0xd, 0x25,  0xd, 0x26,
- 0xd, 0x27,  0xd, 0x28,  0xd, 0x29,  0xd, 0x2a,  0xd, 0x2b,  0xd, 0x2c,  0xd,
-0x2d,  0xd, 0x2e,  0xd, 0x2f,  0xd, 0x30,  0xd, 0x31,  0xd, 0x32,  0xd, 0x33,
- 0xd, 0x34,  0xd, 0x35,  0xd, 0x36,  0xd, 0x37,  0xd, 0x38,  0xd, 0x39,  0xd,
-0x3a,  0xd, 0x3b,  0xd, 0x3c,  0xd, 0x3d,  0xd, 0x3e,  0xd, 0x3f, 0x80,  0x0,
- 0x1,  0x2,  0x3,  0x4,  0x5,  0x6,  0x7,  0x8,  0x9,  0xa,  0xb,  0xc,  0xd,
- 0xe,  0xf, 0x10, 0x11, 0x12, 0x13, 0x14, 0x15, 0x16, 0x17, 0x18, 0x19, 0x1a,
-0x1b, 0x1c, 0x1d, 0x1e, 0x1f, 0x20, 0x21, 0x22, 0x23, 0x24, 0x25, 0x26, 0x27,
-0x28, 0x29, 0x2a, 0x2b, 0x2c, 0x2d, 0x2e, 0x2f, 0x30, 0x31, 0x32, 0x33, 0x34,
-0x35, 0x36, 0x37, 0x38, 0x39, 0x3a, 0x3b, 0x3c, 0x3d, 0x3e, 0x3f, 0x40, 0x41,
-0x42, 0x43, 0x44, 0x45, 0x46, 0x47, 0x48, 0x49, 0x4a, 0x4b, 0x4c, 0x4d, 0x4e,
-0x4f, 0x50, 0x51, 0x52, 0x53, 0x54, 0x55, 0x56, 0x57, 0x58, 0x59, 0x5a, 0x5b,
-0x5c, 0x5d, 0x5e, 0x5f, 0x60, 0x61, 0x62, 0x63, 0x64, 0x65, 0x66, 0x67, 0x68,
-0x69, 0x6a, 0x6b, 0x6c, 0x6d, 0x6e, 0x6f, 0x70, 0x71, 0x72, 0x73, 0x74, 0x75,
-0x76, 0x77, 0x78, 0x79, 0x7a, 0x7b, 0x7c, 0x7d, 0x7e, 0x7f, 0x80, 0x80, 0x81,
-0x82, 0x83, 0x84, 0x85, 0x86, 0x87, 0x88, 0x89, 0x8a, 0x8b, 0x8c, 0x8d, 0x8e,
-0x8f, 0x90, 0x91, 0x92, 0x93, 0x94, 0x95, 0x96, 0x97, 0x98, 0x99, 0x9a, 0x9b,
-0x9c, 0x9d, 0x9e, 0x9f, 0xa0, 0xa1, 0xa2, 0xa3, 0xa4, 0xa5, 0xa6, 0xa7, 0xa8,
-0xa9, 0xaa, 0xab, 0xac, 0xad, 0xae, 0xaf, 0xb0, 0xb1, 0xb2, 0xb3, 0xb4, 0xb5,
-0xb6, 0xb7, 0xb8, 0xb9, 0xba, 0xbb, 0xbc, 0xbd, 0xbe, 0xbf, 0xc0, 0xc1, 0xc2,
-0xc3, 0xc4, 0xc5, 0xc6, 0xc7, 0xc8, 0xc9, 0xca, 0xcb, 0xcc, 0xcd, 0xce, 0xcf,
-0xd0, 0xd1, 0xd2, 0xd3, 0xd4, 0xd5, 0xd6, 0xd7, 0xd8, 0xd9, 0xda, 0xdb, 0xdc,
-0xdd, 0xde, 0xdf, 0xe0, 0xe1, 0xe2, 0xe3, 0xe4, 0xe5, 0xe6, 0xe7, 0xe8, 0xe9,
-0xea, 0xeb, 0xec, 0xed, 0xee, 0xef, 0xf0, 0xf1, 0xf2, 0xf3, 0xf4, 0xf5, 0xf6,
-0xf7, 0xf8, 0xf9, 0xfa, 0xfb, 0xfc, 0xfd, 0xfe, 0xff, 0x80,  0x0,  0x1,  0x2,
- 0x3,  0x4,  0x5,  0x6,  0x7,  0x8,  0x9,  0xa,  0xb,  0xc,  0xd,  0xe,  0xf,
-0x10, 0x11, 0x12, 0x13, 0x14, 0x15, 0x16, 0x17, 0x18, 0x19, 0x1a, 0x1b, 0x1c,
-0x1d, 0x1e, 0x1f, 0x20, 0x21, 0x22, 0x23, 0x24, 0x25, 0x26, 0x27, 0x28, 0x29,
-0x2a, 0x2b, 0x2c, 0x2d, 0x2e, 0x2f, 0x30, 0x31, 0x32, 0x33, 0x34, 0x35, 0x36,
-0x37, 0x38, 0x39, 0x3a, 0x3b, 0x3c, 0x3d, 0x3e, 0x3f, 0x40, 0x41, 0x42, 0x43,
-0x44, 0x45, 0x46, 0x47, 0x48, 0x49, 0x4a, 0x4b, 0x4c, 0x4d, 0x4e, 0x4f, 0x50,
-0x51, 0x52, 0x53, 0x54, 0x55, 0x56, 0x57, 0x58, 0x59, 0x5a, 0x5b, 0x5c, 0x5d,
-0x5e, 0x5f, 0x60, 0x61, 0x62, 0x63, 0x64, 0x65, 0x66, 0x67, 0x68, 0x69, 0x6a,
-0x6b, 0x6c, 0x6d, 0x6e, 0x6f, 0x70, 0x71, 0x72, 0x73, 0x74, 0x75, 0x76, 0x77,
-0x78, 0x79, 0x7a, 0x7b, 0x7c, 0x7d, 0x7e, 0x7f, 0x80, 0x80, 0x81, 0x82, 0x83,
-0x84, 0x85, 0x86, 0x87, 0x88, 0x89, 0x8a, 0x8b, 0x8c, 0x8d, 0x8e, 0x8f, 0x90,
-0x91, 0x92, 0x93, 0x94, 0x95, 0x96, 0x97, 0x98, 0x99, 0x9a, 0x9b, 0x9c, 0x9d,
-0x9e, 0x9f, 0xa0, 0xa1, 0xa2, 0xa3, 0xa4, 0xa5, 0xa6, 0xa7, 0xa8, 0xa9, 0xaa,
-0xab, 0xac, 0xad, 0xae, 0xaf, 0xb0, 0xb1, 0xb2, 0xb3, 0xb4, 0xb5, 0xb6, 0xb7,
-0xb8, 0xb9, 0xba, 0xbb, 0xbc, 0xbd, 0xbe, 0xbf, 0xc0, 0xc1, 0xc2, 0xc3, 0xc4,
-0xc5, 0xc6, 0xc7, 0xc8, 0xc9, 0xca, 0xcb, 0xcc, 0xcd, 0xce, 0xcf, 0xd0, 0xd1,
-0xd2, 0xd3, 0xd4, 0xd5, 0xd6, 0xd7, 0xd8, 0xd9, 0xda, 0xdb, 0xdc, 0xdd, 0xde,
-0xdf, 0xe0, 0xe1, 0xe2, 0xe3, 0xe4, 0xe5, 0xe6, 0xe7, 0xe8, 0xe9, 0xea, 0xeb,
-0xec, 0xed, 0xee, 0xef, 0xf0, 0xf1, 0xf2, 0xf3, 0xf4, 0xf5, 0xf6, 0xf7, 0xf8,
-0xf9, 0xfa, 0xfb, 0xfc, 0xfd, 0xfe, 0xff, 0x80,  0x0,  0x1,  0x2,  0x3,  0x4,
- 0x5,  0x6,  0x7,  0x8,  0x9,  0xa,  0xb,  0xc,  0xd,  0xe,  0xf, 0x10, 0x11,
-0x12, 0x13, 0x14, 0x15, 0x16, 0x17, 0x18, 0x19, 0x1a, 0x1b, 0x1c, 0x1d, 0x1e,
-0x1f, 0x20, 0x21, 0x22, 0x23, 0x24, 0x25, 0x26, 0x27, 0x28, 0x29, 0x2a, 0x2b,
-0x2c, 0x2d, 0x2e, 0x2f, 0x30, 0x31, 0x32, 0x33, 0x34, 0x35, 0x36, 0x37, 0x38,
-0x39, 0x3a, 0x3b, 0x3c, 0x3d, 0x3e, 0x3f, 0x40, 0x41, 0x42, 0x43, 0x44, 0x45,
-0x46, 0x47, 0x48, 0x49, 0x4a, 0x4b, 0x4c, 0x4d, 0x4e, 0x4f, 0x50, 0x51, 0x52,
-0x53, 0x54, 0x55, 0x56, 0x57, 0x58, 0x59, 0x5a, 0x5b, 0x5c, 0x5d, 0x5e, 0x5f,
-0x60, 0x61, 0x62, 0x63, 0x64, 0x65, 0x66, 0x67, 0x68, 0x69, 0x6a, 0x6b, 0x6c,
-0x6d, 0x6e, 0x6f, 0x70, 0x71, 0x72, 0x73, 0x74, 0x75, 0x76, 0x77, 0x78, 0x79,
-0x7a, 0x7b, 0x7c, 0x7d, 0x7e, 0x7f, 0x80, 0x80, 0x81, 0x82, 0x83, 0x84, 0x85,
-0x86, 0x87, 0x88, 0x89, 0x8a, 0x8b, 0x8c, 0x8d, 0x8e, 0x8f, 0x90, 0x91, 0x92,
-0x93, 0x94, 0x95, 0x96, 0x97, 0x98, 0x99, 0x9a, 0x9b, 0x9c, 0x9d, 0x9e, 0x9f,
-0xa0, 0xa1, 0xa2, 0xa3, 0xa4, 0xa5, 0xa6, 0xa7, 0xa8, 0xa9, 0xaa, 0xab, 0xac,
-0xad, 0xae, 0xaf, 0xb0, 0xb1, 0xb2, 0xb3, 0xb4, 0xb5, 0xb6, 0xb7, 0xb8, 0xb9,
-0xba, 0xbb, 0xbc, 0xbd, 0xbe, 0xbf, 0xc0, 0xc1, 0xc2, 0xc3, 0xc4, 0xc5, 0xc6,
-0xc7, 0xc8, 0xc9, 0xca, 0xcb, 0xcc, 0xcd, 0xce, 0xcf, 0xd0, 0xd1, 0xd2, 0xd3,
-0xd4, 0xd5, 0xd6, 0xd7, 0xd8, 0xd9, 0xda, 0xdb, 0xdc, 0xdd, 0xde, 0xdf, 0xe0,
-0xe1, 0xe2, 0xe3, 0xe4, 0xe5, 0xe6, 0xe7, 0xe8, 0xe9, 0xea, 0xeb, 0xec, 0xed,
-0xee, 0xef, 0xf0, 0xf1, 0xf2, 0xf3, 0xf4, 0xf5, 0xf6, 0xf7, 0xf8, 0xf9, 0xfa,
-0xfb, 0xfc, 0xfd, 0xfe, 0xff, 0x80,  0x0,  0x1,  0x2,  0x3,  0x4,  0x5,  0x6,
- 0x7,  0x8,  0x9,  0xa,  0xb,  0xc,  0xd,  0xe,  0xf, 0x10, 0x11, 0x12, 0x13,
-0x14, 0x15, 0x16, 0x17, 0x18, 0x19, 0x1a, 0x1b, 0x1c, 0x1d, 0x1e, 0x1f, 0x20,
-0x21, 0x22, 0x23, 0x24, 0x25, 0x26, 0x27, 0x28, 0x29, 0x2a, 0x2b, 0x2c, 0x2d,
-0x2e, 0x2f, 0x30, 0x31, 0x32, 0x33, 0x34, 0x35, 0x36, 0x37, 0x38, 0x39, 0x3a,
-0x3b, 0x3c, 0x3d, 0x3e, 0x3f, 0x40, 0x41, 0x42, 0x43, 0x44, 0x45, 0x46, 0x47,
-0x48, 0x49, 0x4a, 0x4b, 0x4c, 0x4d, 0x4e, 0x4f, 0x50, 0x51, 0x52, 0x53, 0x54,
-0x55, 0x56, 0x57, 0x58, 0x59, 0x5a, 0x5b, 0x5c, 0x5d, 0x5e, 0x5f, 0x60, 0x61,
-0x62, 0x63, 0x64, 0x65, 0x66, 0x67, 0x68, 0x69, 0x6a, 0x6b, 0x6c, 0x6d, 0x6e,
-0x6f, 0x70, 0x71, 0x72, 0x73, 0x74, 0x75, 0x76, 0x77, 0x78, 0x79, 0x7a, 0x7b,
-0x7c, 0x7d, 0x7e, 0x7f, 0x80, 0x80, 0x81, 0x82, 0x83, 0x84, 0x85, 0x86, 0x87,
-0x88, 0x89, 0x8a, 0x8b, 0x8c, 0x8d, 0x8e, 0x8f, 0x90, 0x91, 0x92, 0x93, 0x94,
-0x95, 0x96, 0x97, 0x98, 0x99, 0x9a, 0x9b, 0x9c, 0x9d, 0x9e, 0x9f, 0xa0, 0xa1,
-0xa2, 0xa3, 0xa4, 0xa5, 0xa6, 0xa7, 0xa8, 0xa9, 0xaa, 0xab, 0xac, 0xad, 0xae,
-0xaf, 0xb0, 0xb1, 0xb2, 0xb3, 0xb4, 0xb5, 0xb6, 0xb7, 0xb8, 0xb9, 0xba, 0xbb,
-0xbc, 0xbd, 0xbe, 0xbf, 0xc0, 0xc1, 0xc2, 0xc3, 0xc4, 0xc5, 0xc6, 0xc7, 0xc8,
-0xc9, 0xca, 0xcb, 0xcc, 0xcd, 0xce, 0xcf, 0xd0, 0xd1, 0xd2, 0xd3, 0xd4, 0xd5,
-0xd6, 0xd7, 0xd8, 0xd9, 0xda, 0xdb, 0xdc, 0xdd, 0xde, 0xdf, 0xe0, 0xe1, 0xe2,
-0xe3, 0xe4, 0xe5, 0xe6, 0xe7, 0xe8, 0xe9, 0xea, 0xeb, 0xec, 0xed, 0xee, 0xef,
-0xf0, 0xf1, 0xf2, 0xf3, 0xf4, 0xf5, 0xf6, 0xf7, 0xf8, 0xf9, 0xfa, 0xfb, 0xfc,
-0xfd, 0xfe, 0xff,
-};
-  SeekableInputStream* const stream =
-    new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer));
-  std::unique_ptr<ByteRleDecoder> rle =
-      createByteRleDecoder(std::unique_ptr<orc::SeekableInputStream>(stream));
-  std::vector<char> data(1);
-  for (size_t i = 0; i < 2048; i += 10) {
-    rle->next(data.data(), data.size(), nullptr);
-    EXPECT_EQ(static_cast<char>(i < 1024 ? i/16 : i & 0xff),
-              data[0])
-        << "Output wrong at " << i;
-    if (i < 2038) {
-      rle->skip(9);
-    }
-    rle->skip(0);
-  }
-}
-
-TEST(ByteRle, testSeek) {
-  // the stream generated by Java's
-  // TestRunLengthByteReader.testUncompressedSeek
-  // for (int i = 0; i < 2048; ++i) {
-  //   if (i < 1024) {
-  //     out.write(i / 4);
-  //   } else {
-  //     out.write(i % 256);
-  //   }
-  // }
-  const unsigned char buffer[] = {
- 0x1,  0x0,  0x1,  0x1,  0x1,  0x2,  0x1,  0x3,  0x1,  0x4,  0x1,  0x5,  0x1,
- 0x6,  0x1,  0x7,  0x1,  0x8,  0x1,  0x9,  0x1,  0xa,  0x1,  0xb,  0x1,  0xc,
- 0x1,  0xd,  0x1,  0xe,  0x1,  0xf,  0x1, 0x10,  0x1, 0x11,  0x1, 0x12,  0x1,
-0x13,  0x1, 0x14,  0x1, 0x15,  0x1, 0x16,  0x1, 0x17,  0x1, 0x18,  0x1, 0x19,
- 0x1, 0x1a,  0x1, 0x1b,  0x1, 0x1c,  0x1, 0x1d,  0x1, 0x1e,  0x1, 0x1f,  0x1,
-0x20,  0x1, 0x21,  0x1, 0x22,  0x1, 0x23,  0x1, 0x24,  0x1, 0x25,  0x1, 0x26,
- 0x1, 0x27,  0x1, 0x28,  0x1, 0x29,  0x1, 0x2a,  0x1, 0x2b,  0x1, 0x2c,  0x1,
-0x2d,  0x1, 0x2e,  0x1, 0x2f,  0x1, 0x30,  0x1, 0x31,  0x1, 0x32,  0x1, 0x33,
- 0x1, 0x34,  0x1, 0x35,  0x1, 0x36,  0x1, 0x37,  0x1, 0x38,  0x1, 0x39,  0x1,
-0x3a,  0x1, 0x3b,  0x1, 0x3c,  0x1, 0x3d,  0x1, 0x3e,  0x1, 0x3f,  0x1, 0x40,
- 0x1, 0x41,  0x1, 0x42,  0x1, 0x43,  0x1, 0x44,  0x1, 0x45,  0x1, 0x46,  0x1,
-0x47,  0x1, 0x48,  0x1, 0x49,  0x1, 0x4a,  0x1, 0x4b,  0x1, 0x4c,  0x1, 0x4d,
- 0x1, 0x4e,  0x1, 0x4f,  0x1, 0x50,  0x1, 0x51,  0x1, 0x52,  0x1, 0x53,  0x1,
-0x54,  0x1, 0x55,  0x1, 0x56,  0x1, 0x57,  0x1, 0x58,  0x1, 0x59,  0x1, 0x5a,
- 0x1, 0x5b,  0x1, 0x5c,  0x1, 0x5d,  0x1, 0x5e,  0x1, 0x5f,  0x1, 0x60,  0x1,
-0x61,  0x1, 0x62,  0x1, 0x63,  0x1, 0x64,  0x1, 0x65,  0x1, 0x66,  0x1, 0x67,
- 0x1, 0x68,  0x1, 0x69,  0x1, 0x6a,  0x1, 0x6b,  0x1, 0x6c,  0x1, 0x6d,  0x1,
-0x6e,  0x1, 0x6f,  0x1, 0x70,  0x1, 0x71,  0x1, 0x72,  0x1, 0x73,  0x1, 0x74,
- 0x1, 0x75,  0x1, 0x76,  0x1, 0x77,  0x1, 0x78,  0x1, 0x79,  0x1, 0x7a,  0x1,
-0x7b,  0x1, 0x7c,  0x1, 0x7d,  0x1, 0x7e,  0x1, 0x7f,  0x1, 0x80,  0x1, 0x81,
- 0x1, 0x82,  0x1, 0x83,  0x1, 0x84,  0x1, 0x85,  0x1, 0x86,  0x1, 0x87,  0x1,
-0x88,  0x1, 0x89,  0x1, 0x8a,  0x1, 0x8b,  0x1, 0x8c,  0x1, 0x8d,  0x1, 0x8e,
- 0x1, 0x8f,  0x1, 0x90,  0x1, 0x91,  0x1, 0x92,  0x1, 0x93,  0x1, 0x94,  0x1,
-0x95,  0x1, 0x96,  0x1, 0x97,  0x1, 0x98,  0x1, 0x99,  0x1, 0x9a,  0x1, 0x9b,
- 0x1, 0x9c,  0x1, 0x9d,  0x1, 0x9e,  0x1, 0x9f,  0x1, 0xa0,  0x1, 0xa1,  0x1,
-0xa2,  0x1, 0xa3,  0x1, 0xa4,  0x1, 0xa5,  0x1, 0xa6,  0x1, 0xa7,  0x1, 0xa8,
- 0x1, 0xa9,  0x1, 0xaa,  0x1, 0xab,  0x1, 0xac,  0x1, 0xad,  0x1, 0xae,  0x1,
-0xaf,  0x1, 0xb0,  0x1, 0xb1,  0x1, 0xb2,  0x1, 0xb3,  0x1, 0xb4,  0x1, 0xb5,
- 0x1, 0xb6,  0x1, 0xb7,  0x1, 0xb8,  0x1, 0xb9,  0x1, 0xba,  0x1, 0xbb,  0x1,
-0xbc,  0x1, 0xbd,  0x1, 0xbe,  0x1, 0xbf,  0x1, 0xc0,  0x1, 0xc1,  0x1, 0xc2,
- 0x1, 0xc3,  0x1, 0xc4,  0x1, 0xc5,  0x1, 0xc6,  0x1, 0xc7,  0x1, 0xc8,  0x1,
-0xc9,  0x1, 0xca,  0x1, 0xcb,  0x1, 0xcc,  0x1, 0xcd,  0x1, 0xce,  0x1, 0xcf,
- 0x1, 0xd0,  0x1, 0xd1,  0x1, 0xd2,  0x1, 0xd3,  0x1, 0xd4,  0x1, 0xd5,  0x1,
-0xd6,  0x1, 0xd7,  0x1, 0xd8,  0x1, 0xd9,  0x1, 0xda,  0x1, 0xdb,  0x1, 0xdc,
- 0x1, 0xdd,  0x1, 0xde,  0x1, 0xdf,  0x1, 0xe0,  0x1, 0xe1,  0x1, 0xe2,  0x1,
-0xe3,  0x1, 0xe4,  0x1, 0xe5,  0x1, 0xe6,  0x1, 0xe7,  0x1, 0xe8,  0x1, 0xe9,
- 0x1, 0xea,  0x1, 0xeb,  0x1, 0xec,  0x1, 0xed,  0x1, 0xee,  0x1, 0xef,  0x1,
-0xf0,  0x1, 0xf1,  0x1, 0xf2,  0x1, 0xf3,  0x1, 0xf4,  0x1, 0xf5,  0x1, 0xf6,
- 0x1, 0xf7,  0x1, 0xf8,  0x1, 0xf9,  0x1, 0xfa,  0x1, 0xfb,  0x1, 0xfc,  0x1,
-0xfd,  0x1, 0xfe,  0x1, 0xff, 0x80,  0x0,  0x1,  0x2,  0x3,  0x4,  0x5,  0x6,
- 0x7,  0x8,  0x9,  0xa,  0xb,  0xc,  0xd,  0xe,  0xf, 0x10, 0x11, 0x12, 0x13,
-0x14, 0x15, 0x16, 0x17, 0x18, 0x19, 0x1a, 0x1b, 0x1c, 0x1d, 0x1e, 0x1f, 0x20,
-0x21, 0x22, 0x23, 0x24, 0x25, 0x26, 0x27, 0x28, 0x29, 0x2a, 0x2b, 0x2c, 0x2d,
-0x2e, 0x2f, 0x30, 0x31, 0x32, 0x33, 0x34, 0x35, 0x36, 0x37, 0x38, 0x39, 0x3a,
-0x3b, 0x3c, 0x3d, 0x3e, 0x3f, 0x40, 0x41, 0x42, 0x43, 0x44, 0x45, 0x46, 0x47,
-0x48, 0x49, 0x4a, 0x4b, 0x4c, 0x4d, 0x4e, 0x4f, 0x50, 0x51, 0x52, 0x53, 0x54,
-0x55, 0x56, 0x57, 0x58, 0x59, 0x5a, 0x5b, 0x5c, 0x5d, 0x5e, 0x5f, 0x60, 0x61,
-0x62, 0x63, 0x64, 0x65, 0x66, 0x67, 0x68, 0x69, 0x6a, 0x6b, 0x6c, 0x6d, 0x6e,
-0x6f, 0x70, 0x71, 0x72, 0x73, 0x74, 0x75, 0x76, 0x77, 0x78, 0x79, 0x7a, 0x7b,
-0x7c, 0x7d, 0x7e, 0x7f, 0x80, 0x80, 0x81, 0x82, 0x83, 0x84, 0x85, 0x86, 0x87,
-0x88, 0x89, 0x8a, 0x8b, 0x8c, 0x8d, 0x8e, 0x8f, 0x90, 0x91, 0x92, 0x93, 0x94,
-0x95, 0x96, 0x97, 0x98, 0x99, 0x9a, 0x9b, 0x9c, 0x9d, 0x9e, 0x9f, 0xa0, 0xa1,
-0xa2, 0xa3, 0xa4, 0xa5, 0xa6, 0xa7, 0xa8, 0xa9, 0xaa, 0xab, 0xac, 0xad, 0xae,
-0xaf, 0xb0, 0xb1, 0xb2, 0xb3, 0xb4, 0xb5, 0xb6, 0xb7, 0xb8, 0xb9, 0xba, 0xbb,
-0xbc, 0xbd, 0xbe, 0xbf, 0xc0, 0xc1, 0xc2, 0xc3, 0xc4, 0xc5, 0xc6, 0xc7, 0xc8,
-0xc9, 0xca, 0xcb, 0xcc, 0xcd, 0xce, 0xcf, 0xd0, 0xd1, 0xd2, 0xd3, 0xd4, 0xd5,
-0xd6, 0xd7, 0xd8, 0xd9, 0xda, 0xdb, 0xdc, 0xdd, 0xde, 0xdf, 0xe0, 0xe1, 0xe2,
-0xe3, 0xe4, 0xe5, 0xe6, 0xe7, 0xe8, 0xe9, 0xea, 0xeb, 0xec, 0xed, 0xee, 0xef,
-0xf0, 0xf1, 0xf2, 0xf3, 0xf4, 0xf5, 0xf6, 0xf7, 0xf8, 0xf9, 0xfa, 0xfb, 0xfc,
-0xfd, 0xfe, 0xff, 0x80,  0x0,  0x1,  0x2,  0x3,  0x4,  0x5,  0x6,  0x7,  0x8,
- 0x9,  0xa,  0xb,  0xc,  0xd,  0xe,  0xf, 0x10, 0x11, 0x12, 0x13, 0x14, 0x15,
-0x16, 0x17, 0x18, 0x19, 0x1a, 0x1b, 0x1c, 0x1d, 0x1e, 0x1f, 0x20, 0x21, 0x22,
-0x23, 0x24, 0x25, 0x26, 0x27, 0x28, 0x29, 0x2a, 0x2b, 0x2c, 0x2d, 0x2e, 0x2f,
-0x30, 0x31, 0x32, 0x33, 0x34, 0x35, 0x36, 0x37, 0x38, 0x39, 0x3a, 0x3b, 0x3c,
-0x3d, 0x3e, 0x3f, 0x40, 0x41, 0x42, 0x43, 0x44, 0x45, 0x46, 0x47, 0x48, 0x49,
-0x4a, 0x4b, 0x4c, 0x4d, 0x4e, 0x4f, 0x50, 0x51, 0x52, 0x53, 0x54, 0x55, 0x56,
-0x57, 0x58, 0x59, 0x5a, 0x5b, 0x5c, 0x5d, 0x5e, 0x5f, 0x60, 0x61, 0x62, 0x63,
-0x64, 0x65, 0x66, 0x67, 0x68, 0x69, 0x6a, 0x6b, 0x6c, 0x6d, 0x6e, 0x6f, 0x70,
-0x71, 0x72, 0x73, 0x74, 0x75, 0x76, 0x77, 0x78, 0x79, 0x7a, 0x7b, 0x7c, 0x7d,
-0x7e, 0x7f, 0x80, 0x80, 0x81, 0x82, 0x83, 0x84, 0x85, 0x86, 0x87, 0x88, 0x89,
-0x8a, 0x8b, 0x8c, 0x8d, 0x8e, 0x8f, 0x90, 0x91, 0x92, 0x93, 0x94, 0x95, 0x96,
-0x97, 0x98, 0x99, 0x9a, 0x9b, 0x9c, 0x9d, 0x9e, 0x9f, 0xa0, 0xa1, 0xa2, 0xa3,
-0xa4, 0xa5, 0xa6, 0xa7, 0xa8, 0xa9, 0xaa, 0xab, 0xac, 0xad, 0xae, 0xaf, 0xb0,
-0xb1, 0xb2, 0xb3, 0xb4, 0xb5, 0xb6, 0xb7, 0xb8, 0xb9, 0xba, 0xbb, 0xbc, 0xbd,
-0xbe, 0xbf, 0xc0, 0xc1, 0xc2, 0xc3, 0xc4, 0xc5, 0xc6, 0xc7, 0xc8, 0xc9, 0xca,
-0xcb, 0xcc, 0xcd, 0xce, 0xcf, 0xd0, 0xd1, 0xd2, 0xd3, 0xd4, 0xd5, 0xd6, 0xd7,
-0xd8, 0xd9, 0xda, 0xdb, 0xdc, 0xdd, 0xde, 0xdf, 0xe0, 0xe1, 0xe2, 0xe3, 0xe4,
-0xe5, 0xe6, 0xe7, 0xe8, 0xe9, 0xea, 0xeb, 0xec, 0xed, 0xee, 0xef, 0xf0, 0xf1,
-0xf2, 0xf3, 0xf4, 0xf5, 0xf6, 0xf7, 0xf8, 0xf9, 0xfa, 0xfb, 0xfc, 0xfd, 0xfe,
-0xff, 0x80,  0x0,  0x1,  0x2,  0x3,  0x4,  0x5,  0x6,  0x7,  0x8,  0x9,  0xa,
- 0xb,  0xc,  0xd,  0xe,  0xf, 0x10, 0x11, 0x12, 0x13, 0x14, 0x15, 0x16, 0x17,
-0x18, 0x19, 0x1a, 0x1b, 0x1c, 0x1d, 0x1e, 0x1f, 0x20, 0x21, 0x22, 0x23, 0x24,
-0x25, 0x26, 0x27, 0x28, 0x29, 0x2a, 0x2b, 0x2c, 0x2d, 0x2e, 0x2f, 0x30, 0x31,
-0x32, 0x33, 0x34, 0x35, 0x36, 0x37, 0x38, 0x39, 0x3a, 0x3b, 0x3c, 0x3d, 0x3e,
-0x3f, 0x40, 0x41, 0x42, 0x43, 0x44, 0x45, 0x46, 0x47, 0x48, 0x49, 0x4a, 0x4b,
-0x4c, 0x4d, 0x4e, 0x4f, 0x50, 0x51, 0x52, 0x53, 0x54, 0x55, 0x56, 0x57, 0x58,
-0x59, 0x5a, 0x5b, 0x5c, 0x5d, 0x5e, 0x5f, 0x60, 0x61, 0x62, 0x63, 0x64, 0x65,
-0x66, 0x67, 0x68, 0x69, 0x6a, 0x6b, 0x6c, 0x6d, 0x6e, 0x6f, 0x70, 0x71, 0x72,
-0x73, 0x74, 0x75, 0x76, 0x77, 0x78, 0x79, 0x7a, 0x7b, 0x7c, 0x7d, 0x7e, 0x7f,
-0x80, 0x80, 0x81, 0x82, 0x83, 0x84, 0x85, 0x86, 0x87, 0x88, 0x89, 0x8a, 0x8b,
-0x8c, 0x8d, 0x8e, 0x8f, 0x90, 0x91, 0x92, 0x93, 0x94, 0x95, 0x96, 0x97, 0x98,
-0x99, 0x9a, 0x9b, 0x9c, 0x9d, 0x9e, 0x9f, 0xa0, 0xa1, 0xa2, 0xa3, 0xa4, 0xa5,
-0xa6, 0xa7, 0xa8, 0xa9, 0xaa, 0xab, 0xac, 0xad, 0xae, 0xaf, 0xb0, 0xb1, 0xb2,
-0xb3, 0xb4, 0xb5, 0xb6, 0xb7, 0xb8, 0xb9, 0xba, 0xbb, 0xbc, 0xbd, 0xbe, 0xbf,
-0xc0, 0xc1, 0xc2, 0xc3, 0xc4, 0xc5, 0xc6, 0xc7, 0xc8, 0xc9, 0xca, 0xcb, 0xcc,
-0xcd, 0xce, 0xcf, 0xd0, 0xd1, 0xd2, 0xd3, 0xd4, 0xd5, 0xd6, 0xd7, 0xd8, 0xd9,
-0xda, 0xdb, 0xdc, 0xdd, 0xde, 0xdf, 0xe0, 0xe1, 0xe2, 0xe3, 0xe4, 0xe5, 0xe6,
-0xe7, 0xe8, 0xe9, 0xea, 0xeb, 0xec, 0xed, 0xee, 0xef, 0xf0, 0xf1, 0xf2, 0xf3,
-0xf4, 0xf5, 0xf6, 0xf7, 0xf8, 0xf9, 0xfa, 0xfb, 0xfc, 0xfd, 0xfe, 0xff, 0x80,
- 0x0,  0x1,  0x2,  0x3,  0x4,  0x5,  0x6,  0x7,  0x8,  0x9,  0xa,  0xb,  0xc,
- 0xd,  0xe,  0xf, 0x10, 0x11, 0x12, 0x13, 0x14, 0x15, 0x16, 0x17, 0x18, 0x19,
-0x1a, 0x1b, 0x1c, 0x1d, 0x1e, 0x1f, 0x20, 0x21, 0x22, 0x23, 0x24, 0x25, 0x26,
-0x27, 0x28, 0x29, 0x2a, 0x2b, 0x2c, 0x2d, 0x2e, 0x2f, 0x30, 0x31, 0x32, 0x33,
-0x34, 0x35, 0x36, 0x37, 0x38, 0x39, 0x3a, 0x3b, 0x3c, 0x3d, 0x3e, 0x3f, 0x40,
-0x41, 0x42, 0x43, 0x44, 0x45, 0x46, 0x47, 0x48, 0x49, 0x4a, 0x4b, 0x4c, 0x4d,
-0x4e, 0x4f, 0x50, 0x51, 0x52, 0x53, 0x54, 0x55, 0x56, 0x57, 0x58, 0x59, 0x5a,
-0x5b, 0x5c, 0x5d, 0x5e, 0x5f, 0x60, 0x61, 0x62, 0x63, 0x64, 0x65, 0x66, 0x67,
-0x68, 0x69, 0x6a, 0x6b, 0x6c, 0x6d, 0x6e, 0x6f, 0x70, 0x71, 0x72, 0x73, 0x74,
-0x75, 0x76, 0x77, 0x78, 0x79, 0x7a, 0x7b, 0x7c, 0x7d, 0x7e, 0x7f, 0x80, 0x80,
-0x81, 0x82, 0x83, 0x84, 0x85, 0x86, 0x87, 0x88, 0x89, 0x8a, 0x8b, 0x8c, 0x8d,
-0x8e, 0x8f, 0x90, 0x91, 0x92, 0x93, 0x94, 0x95, 0x96, 0x97, 0x98, 0x99, 0x9a,
-0x9b, 0x9c, 0x9d, 0x9e, 0x9f, 0xa0, 0xa1, 0xa2, 0xa3, 0xa4, 0xa5, 0xa6, 0xa7,
-0xa8, 0xa9, 0xaa, 0xab, 0xac, 0xad, 0xae, 0xaf, 0xb0, 0xb1, 0xb2, 0xb3, 0xb4,
-0xb5, 0xb6, 0xb7, 0xb8, 0xb9, 0xba, 0xbb, 0xbc, 0xbd, 0xbe, 0xbf, 0xc0, 0xc1,
-0xc2, 0xc3, 0xc4, 0xc5, 0xc6, 0xc7, 0xc8, 0xc9, 0xca, 0xcb, 0xcc, 0xcd, 0xce,
-0xcf, 0xd0, 0xd1, 0xd2, 0xd3, 0xd4, 0xd5, 0xd6, 0xd7, 0xd8, 0xd9, 0xda, 0xdb,
-0xdc, 0xdd, 0xde, 0xdf, 0xe0, 0xe1, 0xe2, 0xe3, 0xe4, 0xe5, 0xe6, 0xe7, 0xe8,
-0xe9, 0xea, 0xeb, 0xec, 0xed, 0xee, 0xef, 0xf0, 0xf1, 0xf2, 0xf3, 0xf4, 0xf5,
-0xf6, 0xf7, 0xf8, 0xf9, 0xfa, 0xfb, 0xfc, 0xfd, 0xfe, 0xff,
-  };
-  std::unique_ptr<SeekableInputStream> stream
-    (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer)));
-  const uint64_t fileLocs[] = {
-   0,    0,    0,    0,    0,    2,    2,    2,    2,    4,    4,    4,    4,
-   6,    6,    6,    6,    8,    8,    8,    8,   10,   10,   10,   10,   12,
-  12,   12,   12,   14,   14,   14,   14,   16,   16,   16,   16,   18,   18,
-  18,   18,   20,   20,   20,   20,   22,   22,   22,   22,   24,   24,   24,
-  24,   26,   26,   26,   26,   28,   28,   28,   28,   30,   30,   30,   30,
-  32,   32,   32,   32,   34,   34,   34,   34,   36,   36,   36,   36,   38,
-  38,   38,   38,   40,   40,   40,   40,   42,   42,   42,   42,   44,   44,
-  44,   44,   46,   46,   46,   46,   48,   48,   48,   48,   50,   50,   50,
-  50,   52,   52,   52,   52,   54,   54,   54,   54,   56,   56,   56,   56,
-  58,   58,   58,   58,   60,   60,   60,   60,   62,   62,   62,   62,   64,
-  64,   64,   64,   66,   66,   66,   66,   68,   68,   68,   68,   70,   70,
-  70,   70,   72,   72,   72,   72,   74,   74,   74,   74,   76,   76,   76,
-  76,   78,   78,   78,   78,   80,   80,   80,   80,   82,   82,   82,   82,
-  84,   84,   84,   84,   86,   86,   86,   86,   88,   88,   88,   88,   90,
-  90,   90,   90,   92,   92,   92,   92,   94,   94,   94,   94,   96,   96,
-  96,   96,   98,   98,   98,   98,  100,  100,  100,  100,  102,  102,  102,
- 102,  104,  104,  104,  104,  106,  106,  106,  106,  108,  108,  108,  108,
- 110,  110,  110,  110,  112,  112,  112,  112,  114,  114,  114,  114,  116,
- 116,  116,  116,  118,  118,  118,  118,  120,  120,  120,  120,  122,  122,
- 122,  122,  124,  124,  124,  124,  126,  126,  126,  126,  128,  128,  128,
- 128,  130,  130,  130,  130,  132,  132,  132,  132,  134,  134,  134,  134,
- 136,  136,  136,  136,  138,  138,  138,  138,  140,  140,  140,  140,  142,
- 142,  142,  142,  144,  144,  144,  144,  146,  146,  146,  146,  148,  148,
- 148,  148,  150,  150,  150,  150,  152,  152,  152,  152,  154,  154,  154,
- 154,  156,  156,  156,  156,  158,  158,  158,  158,  160,  160,  160,  160,
- 162,  162,  162,  162,  164,  164,  164,  164,  166,  166,  166,  166,  168,
- 168,  168,  168,  170,  170,  170,  170,  172,  172,  172,  172,  174,  174,
- 174,  174,  176,  176,  176,  176,  178,  178,  178,  178,  180,  180,  180,
- 180,  182,  182,  182,  182,  184,  184,  184,  184,  186,  186,  186,  186,
- 188,  188,  188,  188,  190,  190,  190,  190,  192,  192,  192,  192,  194,
- 194,  194,  194,  196,  196,  196,  196,  198,  198,  198,  198,  200,  200,
- 200,  200,  202,  202,  202,  202,  204,  204,  204,  204,  206,  206,  206,
- 206,  208,  208,  208,  208,  210,  210,  210,  210,  212,  212,  212,  212,
- 214,  214,  214,  214,  216,  216,  216,  216,  218,  218,  218,  218,  220,
- 220,  220,  220,  222,  222,  222,  222,  224,  224,  224,  224,  226,  226,
- 226,  226,  228,  228,  228,  228,  230,  230,  230,  230,  232,  232,  232,
- 232,  234,  234,  234,  234,  236,  236,  236,  236,  238,  238,  238,  238,
- 240,  240,  240,  240,  242,  242,  242,  242,  244,  244,  244,  244,  246,
- 246,  246,  246,  248,  248,  248,  248,  250,  250,  250,  250,  252,  252,
- 252,  252,  254,  254,  254,  254,  256,  256,  256,  256,  258,  258,  258,
- 258,  260,  260,  260,  260,  262,  262,  262,  262,  264,  264,  264,  264,
- 266,  266,  266,  266,  268,  268,  268,  268,  270,  270,  270,  270,  272,
- 272,  272,  272,  274,  274,  274,  274,  276,  276,  276,  276,  278,  278,
- 278,  278,  280,  280,  280,  280,  282,  282,  282,  282,  284,  284,  284,
- 284,  286,  286,  286,  286,  288,  288,  288,  288,  290,  290,  290,  290,
- 292,  292,  292,  292,  294,  294,  294,  294,  296,  296,  296,  296,  298,
- 298,  298,  298,  300,  300,  300,  300,  302,  302,  302,  302,  304,  304,
- 304,  304,  306,  306,  306,  306,  308,  308,  308,  308,  310,  310,  310,
- 310,  312,  312,  312,  312,  314,  314,  314,  314,  316,  316,  316,  316,
- 318,  318,  318,  318,  320,  320,  320,  320,  322,  322,  322,  322,  324,
- 324,  324,  324,  326,  326,  326,  326,  328,  328,  328,  328,  330,  330,
- 330,  330,  332,  332,  332,  332,  334,  334,  334,  334,  336,  336,  336,
- 336,  338,  338,  338,  338,  340,  340,  340,  340,  342,  342,  342,  342,
- 344,  344,  344,  344,  346,  346,  346,  346,  348,  348,  348,  348,  350,
- 350,  350,  350,  352,  352,  352,  352,  354,  354,  354,  354,  356,  356,
- 356,  356,  358,  358,  358,  358,  360,  360,  360,  360,  362,  362,  362,
- 362,  364,  364,  364,  364,  366,  366,  366,  366,  368,  368,  368,  368,
- 370,  370,  370,  370,  372,  372,  372,  372,  374,  374,  374,  374,  376,
- 376,  376,  376,  378,  378,  378,  378,  380,  380,  380,  380,  382,  382,
- 382,  382,  384,  384,  384,  384,  386,  386,  386,  386,  388,  388,  388,
- 388,  390,  390,  390,  390,  392,  392,  392,  392,  394,  394,  394,  394,
- 396,  396,  396,  396,  398,  398,  398,  398,  400,  400,  400,  400,  402,
- 402,  402,  402,  404,  404,  404,  404,  406,  406,  406,  406,  408,  408,
- 408,  408,  410,  410,  410,  410,  412,  412,  412,  412,  414,  414,  414,
- 414,  416,  416,  416,  416,  418,  418,  418,  418,  420,  420,  420,  420,
- 422,  422,  422,  422,  424,  424,  424,  424,  426,  426,  426,  426,  428,
- 428,  428,  428,  430,  430,  430,  430,  432,  432,  432,  432,  434,  434,
- 434,  434,  436,  436,  436,  436,  438,  438,  438,  438,  440,  440,  440,
- 440,  442,  442,  442,  442,  444,  444,  444,  444,  446,  446,  446,  446,
- 448,  448,  448,  448,  450,  450,  450,  450,  452,  452,  452,  452,  454,
- 454,  454,  454,  456,  456,  456,  456,  458,  458,  458,  458,  460,  460,
- 460,  460,  462,  462,  462,  462,  464,  464,  464,  464,  466,  466,  466,
- 466,  468,  468,  468,  468,  470,  470,  470,  470,  472,  472,  472,  472,
- 474,  474,  474,  474,  476,  476,  476,  476,  478,  478,  478,  478,  480,
- 480,  480,  480,  482,  482,  482,  482,  484,  484,  484,  484,  486,  486,
- 486,  486,  488,  488,  488,  488,  490,  490,  490,  490,  492,  492,  492,
- 492,  494,  494,  494,  494,  496,  496,  496,  496,  498,  498,  498,  498,
- 500,  500,  500,  500,  502,  502,  502,  502,  504,  504,  504,  504,  506,
- 506,  506,  506,  508,  508,  508,  508,  510,  510,  510,  510,  512,  512,
- 512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,
- 512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,
- 512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,
- 512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,
- 512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,
- 512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,
- 512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,
- 512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,
- 512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,  512,
- 512,  512,  512,  512,  512,  512,  512,  512,  641,  641,  641,  641,  641,
- 641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,
- 641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,
- 641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,
- 641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,
- 641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,
- 641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,
- 641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,
- 641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,
- 641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,  641,
- 641,  641,  641,  641,  641,  641,  770,  770,  770,  770,  770,  770,  770,
- 770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,
- 770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,
- 770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,
- 770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,
- 770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,
- 770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,
- 770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,
- 770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,
- 770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,  770,
- 770,  770,  770,  770,  899,  899,  899,  899,  899,  899,  899,  899,  899,
- 899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,
- 899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,
- 899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,
- 899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,
- 899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,
- 899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,
- 899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,
- 899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,
- 899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,  899,
- 899,  899, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
-1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
-1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
-1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
-1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
-1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
-1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
-1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
-1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
-1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028,
-1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157,
-1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157,
-1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157,
-1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157,
-1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157,
-1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157,
-1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157,
-1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157,
-1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157,
-1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1286, 1286,
-1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286,
-1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286,
-1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286,
-1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286,
-1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286,
-1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286,
-1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286,
-1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286,
-1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286,
-1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1415, 1415, 1415, 1415,
-1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415,
-1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415,
-1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415,
-1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415,
-1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415,
-1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415,
-1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415,
-1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415,
-1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415,
-1415, 1415, 1415, 1415, 1415, 1415, 1415,
-    };
-  const uint64_t rleLocs[] = {
-  0,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,   4,   1,   2,   3,
-  4,   1,   2,   3,   4,   5,   6,   7,   8,   9,  10,  11,  12,  13,  14,  15,
- 16,  17,  18,  19,  20,  21,  22,  23,  24,  25,  26,  27,  28,  29,  30,  31,
- 32,  33,  34,  35,  36,  37,  38,  39,  40,  41,  42,  43,  44,  45,  46,  47,
- 48,  49,  50,  51,  52,  53,  54,  55,  56,  57,  58,  59,  60,  61,  62,  63,
- 64,  65,  66,  67,  68,  69,  70,  71,  72,  73,  74,  75,  76,  77,  78,  79,
- 80,  81,  82,  83,  84,  85,  86,  87,  88,  89,  90,  91,  92,  93,  94,  95,
- 96,  97,  98,  99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111,
-112, 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126, 127,
-  0,   1,   2,   3,   4,   5,   6,   7,   8,   9,  10,  11,  12,  13,  14,  15,
- 16,  17,  18,  19,  20,  21,  22,  23,  24,  25,  26,  27,  28,  29,  30,  31,
- 32,  33,  34,  35,  36,  37,  38,  39,  40,  41,  42,  43,  44,  45,  46,  47,
- 48,  49,  50,  51,  52,  53,  54,  55,  56,  57,  58,  59,  60,  61,  62,  63,
- 64,  65,  66,  67,  68,  69,  70,  71,  72,  73,  74,  75,  76,  77,  78,  79,
- 80,  81,  82,  83,  84,  85,  86,  87,  88,  89,  90,  91,  92,  93,  94,  95,
- 96,  97,  98,  99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111,
-112, 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126, 127,
-  0,   1,   2,   3,   4,   5,   6,   7,   8,   9,  10,  11,  12,  13,  14,  15,
- 16,  17,  18,  19,  20,  21,  22,  23,  24,  25,  26,  27,  28,  29,  30,  31,
- 32,  33,  34,  35,  36,  37,  38,  39,  40,  41,  42,  43,  44,  45,  46,  47,
- 48,  49,  50,  51,  52,  53,  54,  55,  56,  57,  58,  59,  60,  61,  62,  63,
- 64,  65,  66,  67,  68,  69,  70,  71,  72,  73,  74,  75,  76,  77,  78,  79,
- 80,  81,  82,  83,  84,  85,  86,  87,  88,  89,  90,  91,  92,  93,  94,  95,
- 96,  97,  98,  99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111,
-112, 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126, 127,
-  0,   1,   2,   3,   4,   5,   6,   7,   8,   9,  10,  11,  12,  13,  14,  15,
- 16,  17,  18,  19,  20,  21,  22,  23,  24,  25,  26,  27,  28,  29,  30,  31,
- 32,  33,  34,  35,  36,  37,  38,  39,  40,  41,  42,  43,  44,  45,  46,  47,
- 48,  49,  50,  51,  52,  53,  54,  55,  56,  57,  58,  59,  60,  61,  62,  63,
- 64,  65,  66,  67,  68,  69,  70,  71,  72,  73,  74,  75,  76,  77,  78,  79,
- 80,  81,  82,  83,  84,  85,  86,  87,  88,  89,  90,  91,  92,  93,  94,  95,
- 96,  97,  98,  99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111,
-112, 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126, 127,
-  0,   1,   2,   3,   4,   5,   6,   7,   8,   9,  10,  11,  12,  13,  14,  15,
- 16,  17,  18,  19,  20,  21,  22,  23,  24,  25,  26,  27,  28,  29,  30,  31,
- 32,  33,  34,  35,  36,  37,  38,  39,  40,  41,  42,  43,  44,  45,  46,  47,
- 48,  49,  50,  51,  52,  53,  54,  55,  56,  57,  58,  59,  60,  61,  62,  63,
- 64,  65,  66,  67,  68,  69,  70,  71,  72,  73,  74,  75,  76,  77,  78,  79,
- 80,  81,  82,  83,  84,  85,  86,  87,  88,  89,  90,  91,  92,  93,  94,  95,
- 96,  97,  98,  99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111,
-112, 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126, 127,
-  0,   1,   2,   3,   4,   5,   6,   7,   8,   9,  10,  11,  12,  13,  14,  15,
- 16,  17,  18,  19,  20,  21,  22,  23,  24,  25,  26,  27,  28,  29,  30,  31,
- 32,  33,  34,  35,  36,  37,  38,  39,  40,  41,  42,  43,  44,  45,  46,  47,
- 48,  49,  50,  51,  52,  53,  54,  55,  56,  57,  58,  59,  60,  61,  62,  63,
- 64,  65,  66,  67,  68,  69,  70,  71,  72,  73,  74,  75,  76,  77,  78,  79,
- 80,  81,  82,  83,  84,  85,  86,  87,  88,  89,  90,  91,  92,  93,  94,  95,
- 96,  97,  98,  99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111,
-112, 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126, 127,
-  0,   1,   2,   3,   4,   5,   6,   7,   8,   9,  10,  11,  12,  13,  14,  15,
- 16,  17,  18,  19,  20,  21,  22,  23,  24,  25,  26,  27,  28,  29,  30,  31,
- 32,  33,  34,  35,  36,  37,  38,  39,  40,  41,  42,  43,  44,  45,  46,  47,
- 48,  49,  50,  51,  52,  53,  54,  55,  56,  57,  58,  59,  60,  61,  62,  63,
- 64,  65,  66,  67,  68,  69,  70,  71,  72,  73,  74,  75,  76,  77,  78,  79,
- 80,  81,  82,  83,  84,  85,  86,  87,  88,  89,  90,  91,  92,  93,  94,  95,
- 96,  97,  98,  99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111,
-112, 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126, 127,
-  0,   1,   2,   3,   4,   5,   6,   7,   8,   9,  10,  11,  12,  13,  14,  15,
- 16,  17,  18,  19,  20,  21,  22,  23,  24,  25,  26,  27,  28,  29,  30,  31,
- 32,  33,  34,  35,  36,  37,  38,  39,  40,  41,  42,  43,  44,  45,  46,  47,
- 48,  49,  50,  51,  52,  53,  54,  55,  56,  57,  58,  59,  60,  61,  62,  63,
- 64,  65,  66,  67,  68,  69,  70,  71,  72,  73,  74,  75,  76,  77,  78,  79,
- 80,  81,  82,  83,  84,  85,  86,  87,  88,  89,  90,  91,  92,  93,  94,  95,
- 96,  97,  98,  99, 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111,
-112, 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126, 127,
-};
-  std::list<uint64_t> positions[2048];
-  for (size_t i = 0; i < 2048; ++i) {
-    positions[i].push_back(fileLocs[i]);
-    positions[i].push_back(rleLocs[i]);
-  }
-  std::unique_ptr<ByteRleDecoder> rle =
-      createByteRleDecoder(std::move(stream));
-  std::vector<char> data(1);
-  for (size_t i = 0; i < 2048; ++i) {
-    rle->next(data.data(), 1, nullptr);
-    EXPECT_EQ(static_cast<char>(i < 1024 ? i / 4 : i & 0xff),
-              data[0])
-        << "Output wrong at " << i;
-  }
-  size_t i = 2048;
-  do {
-    --i;
-    PositionProvider location(positions[i]);
-    rle->seek(location);
-    rle->next(data.data(), 1, nullptr);
-    EXPECT_EQ(static_cast<char>(i < 1024 ? i / 4 : i & 0xff),
-              data[0])
-        << "Output wrong at " << i;
-  } while (i != 0);
-}
-
-TEST(BooleanRle, simpleTest) {
-  const unsigned char buffer[] = {0x61, 0xf0, 0xfd, 0x55, 0xAA, 0x55};
-  std::unique_ptr<SeekableInputStream> stream
-    (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer)));
-  std::unique_ptr<ByteRleDecoder> rle =
-      createBooleanRleDecoder(std::move(stream));
-  std::vector<char> data(50);
-  for (size_t i = 0; i < 16; ++i) {
-    rle->next(data.data(), data.size(), nullptr);
-    for (size_t j = 0; j < data.size(); ++j) {
-      const int bitPosn = static_cast<int>(50 * i + j);
-      EXPECT_EQ((bitPosn & 0x4) == 0 ? 1 : 0,
-                data[j])
-          << "Output wrong at " << i << ", " << j;
-    }
-  }
-  rle->next(data.data(), 24, nullptr);
-  for (size_t i = 0; i < 3; ++i) {
-    for (size_t j = 0; j < 8; ++j) {
-      EXPECT_EQ((i % 2) == (j % 2) ? 0 : 1,
-                data[i * 8 + j])
-          << "Output wrong at " << i << "," << j;
-    }
-  }
-}
-
-TEST(BooleanRle, runsTest) {
-  const unsigned char buffer[] = {0xf7, 0xff, 0x80, 0x3f, 0xe0, 0x0f,
-				  0xf8, 0x03, 0xfe, 0x00};
-  std::unique_ptr<SeekableInputStream> stream
-    (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer)));
-  std::unique_ptr<ByteRleDecoder> rle =
-      createBooleanRleDecoder(std::move(stream));
-  std::vector<char> data(72);
-  rle->next(data.data(), data.size(), nullptr);
-  for (size_t i = 0; i < data.size(); ++i) {
-    EXPECT_EQ(i % 18 < 9 ? 1 : 0,
-              data[i])
-        << "Output wrong at " << i;
-  }
-  std::list<uint64_t> position(3, 0);
-  PositionProvider location(position);
-  rle->seek(location);
-  for (size_t i = 0; i < data.size(); ++i) {
-    rle->next(data.data(), 1, nullptr);
-    EXPECT_EQ(i % 18 < 9 ? 1 : 0,
-              data[0])
-        << "Output wrong at " << i;
-  }
-}
-
-TEST(BooleanRle, runsTestWithNull) {
-  const unsigned char buffer[] = {0xf7, 0xff, 0x80, 0x3f, 0xe0, 0x0f,
-				  0xf8, 0x03, 0xfe, 0x00};
-  std::unique_ptr<SeekableInputStream> stream
-    (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer)));
-  std::unique_ptr<ByteRleDecoder> rle =
-      createBooleanRleDecoder(std::move(stream));
-  std::vector<char> data(72);
-  std::vector<char> notNull(data.size(), 1);
-  rle->next(data.data(), data.size(), notNull.data());
-  for (size_t i = 0; i < data.size(); ++i) {
-    EXPECT_EQ(i % 18 < 9 ? 1 : 0,
-              data[i])
-        << "Output wrong at " << i;
-  }
-  std::list<uint64_t> position(3, 0);
-  PositionProvider location(position);
-  rle->seek(location);
-  for (size_t i = 0; i < data.size(); ++i) {
-    rle->next(data.data(), 1, notNull.data());
-    EXPECT_EQ(i % 18 < 9 ? 1 : 0,
-              data[0])
-        << "Output wrong at " << i;
-  }
-}
-
-TEST(BooleanRle, skipTest) {
-  // stream copied from Java's TestBitFieldReader.testSkips
-  // for i in 0..16383
-  //   if i < 8192
-  //     out.write(i & 1)
-  //   else
-  //     out.write((i / 3) & 1)
-  const unsigned char buffer[] = {
-0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f,
-0x55, 0x6f, 0x55, 0x80, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x80, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0x80, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x80, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x80,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0x80, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x80, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x80, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71
-  };
-  std::unique_ptr<SeekableInputStream> stream
-    (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer)));
-  std::unique_ptr<ByteRleDecoder> rle =
-      createBooleanRleDecoder(std::move(stream));
-  std::vector<char> data(1);
-  for (size_t i = 0; i < 16384; i += 5) {
-    rle->next(data.data(), data.size(), nullptr);
-    EXPECT_EQ(i < 8192 ? i & 1 : (i / 3) & 1,
-              data[0])
-        << "Output wrong at " << i;
-    if (i < 16379) {
-      rle->skip(4);
-    }
-    rle->skip(0);
-  }
-}
-
-TEST(BooleanRle, skipTestWithNulls) {
-  // stream copied from Java's TestBitFieldReader.testSkips
-  // for i in 0..16383
-  //   if i < 8192
-  //     out.write(i & 1)
-  //   else
-  //     out.write((i / 3) & 1)
-  const unsigned char buffer[] = {
-0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f,
-0x55, 0x6f, 0x55, 0x80, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x80, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0x80, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x80, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x80,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0x80, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x80, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x80, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71
-  };
-  std::unique_ptr<SeekableInputStream> stream
-    (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer)));
-  std::unique_ptr<ByteRleDecoder> rle =
-      createBooleanRleDecoder(std::move(stream));
-  std::vector<char> data(3);
-  std::vector<char> someNull(data.size(), 0);
-  someNull[1] = 1;
-  std::vector<char> allNull(data.size(), 0);
-  for (size_t i = 0; i < 16384; i += 5) {
-    data.assign(data.size(), -1);
-    rle->next(data.data(), data.size(), someNull.data());
-    EXPECT_EQ(0, data[0]) << "Output wrong at " << i;
-    EXPECT_EQ(0, data[2]) << "Output wrong at " << i;
-    EXPECT_EQ(i < 8192 ? i & 1 : (i / 3) & 1,
-              data[1])
-        << "Output wrong at " << i;
-    if (i < 16379) {
-      rle->skip(4);
-    }
-    rle->skip(0);
-    data.assign(data.size(), -1);;
-    rle->next(data.data(), data.size(), allNull.data());
-    for (size_t j = 0; j < data.size(); ++j) {
-      EXPECT_EQ(0, data[j]) << "Output wrong at " << i << ", " << j;
-    }
-  }
-}
-
-TEST(BooleanRle, seekTest) {
-  // stream copied from Java's TestBitFieldReader.testUncompressedSeek
-  // for i in 0..16383
-  //   if i < 8192
-  //     out.write(i & 1)
-  //   else
-  //     out.write((i / 3) & 1)
-  const unsigned char buffer[] = {
-0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f,
-0x55, 0x6f, 0x55, 0x80, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x80, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0x80, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x80, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x80,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0x80, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x80, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x80, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71
-  };
-  std::unique_ptr<SeekableInputStream> stream
-    (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer)));
-  std::unique_ptr<ByteRleDecoder> rle =
-      createBooleanRleDecoder(std::move(stream));
-  std::vector<char> data(16384);
-  rle->next(data.data(), data.size(), nullptr);
-  for (size_t i = 0; i < data.size(); ++i) {
-    EXPECT_EQ(i < 8192 ? i & 1 : (i / 3) & 1,
-              data[i])
-        << "Output wrong at " << i;
-  }
-  // set up all of the positions
-  std::list<uint64_t> positions[16384];
-  for (uint64_t i = 0; i < 16384; ++i) {
-    const uint64_t bytePosn = i / 8;
-    // add the stream position
-    positions[i].push_back(bytePosn < 1025
-                           ? 2 * (bytePosn / 130)
-                           : (bytePosn < 1152
-                              ? 16
-                              : 145 + 129 * ((bytePosn - 1152)/ 128)));
-    // add the byte RLE position
-    positions[i].push_back(bytePosn < 1025
-                           ? bytePosn % 130
-                           : (bytePosn - 1024) % 128);
-    // add the bit position
-    positions[i].push_back(i % 8);
-  }
-  size_t i = 16384;
-  do {
-    --i;
-    PositionProvider location(positions[i]);
-    rle->seek(location);
-    rle->next(data.data(), 1, nullptr);
-    EXPECT_EQ(i < 8192 ? i & 1 : (i / 3) & 1,
-              data[i])
-        << "Output wrong at " << i;
-  } while (i != 0);
-}
-
-TEST(BooleanRle, seekTestWithNulls) {
-  // stream copied from Java's TestBitFieldReader.testUncompressedSeek
-  // for i in 0..16383
-  //   if i < 8192
-  //     out.write(i & 1)
-  //   else
-  //     out.write((i / 3) & 1)
-  const unsigned char buffer[] = {
-0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f, 0x55, 0x7f,
-0x55, 0x6f, 0x55, 0x80, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x80, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0x80, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x80, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x80,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0x80, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x80, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x80, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c,
-0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71,
-0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7,
-0x1c, 0x71, 0xc7, 0x1c, 0x71, 0xc7, 0x1c, 0x71
-};
-  std::unique_ptr<SeekableInputStream> stream
-    (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer)));
-  std::unique_ptr<ByteRleDecoder> rle =
-      createBooleanRleDecoder(std::move(stream));
-  std::vector<char> data(16384);
-  std::vector<char> allNull(data.size(), 0);
-  std::vector<char> noNull(data.size(), 1);
-  rle->next(data.data(), data.size(), allNull.data());
-  for (size_t i = 0; i < data.size(); ++i) {
-    EXPECT_EQ(0, data[i]) << "Output wrong at " << i;
-  }
-  rle->next(data.data(), data.size(), noNull.data());
-  for (size_t i = 0; i < data.size(); ++i) {
-    EXPECT_EQ(i < 8192 ? i & 1 : (i / 3) & 1,
-              data[i])
-        << "Output wrong at " << i;
-  }
-  // set up all of the positions
-  std::list<uint64_t> positions[16384];
-  for (uint64_t i = 0; i < 16384; ++i) {
-    const uint64_t bytePosn = i / 8;
-    // add the stream position
-    // add the stream position
-    positions[i].push_back(bytePosn < 1025
-                           ? 2 * (bytePosn / 130)
-                           : (bytePosn < 1152
-                              ? 16
-                              : 145 + 129 * ((bytePosn - 1152)/ 128)));
-    // add the byte RLE position
-    positions[i].push_back(bytePosn < 1025
-                           ? bytePosn % 130
-                           : (bytePosn - 1024) % 128);
-    // add the bit position
-    positions[i].push_back(i % 8);
-  }
-  size_t i = 16384;
-  do {
-    --i;
-    PositionProvider location(positions[i]);
-    rle->seek(location);
-    rle->next(data.data(), 1, noNull.data());
-    EXPECT_EQ(i < 8192 ? i & 1 : (i / 3) & 1,
-              data[i])
-        << "Output wrong at " << i;
-    data[0] = -1;
-    rle->next(data.data(), 1, allNull.data());
-    EXPECT_EQ(0, data[0]) << "Output wrong at " << i;
-  } while (i != 0);
-}
-
-}  // namespace orc


[22/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/ColumnReader.cc
----------------------------------------------------------------------
diff --git a/c++/src/ColumnReader.cc b/c++/src/ColumnReader.cc
new file mode 100644
index 0000000..78fb15f
--- /dev/null
+++ b/c++/src/ColumnReader.cc
@@ -0,0 +1,1558 @@
+/**
+ * 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 "orc/Int128.hh"
+
+#include "Adaptor.hh"
+#include "ByteRLE.hh"
+#include "ColumnReader.hh"
+#include "Exceptions.hh"
+#include "RLE.hh"
+
+#include <math.h>
+#include <iostream>
+
+namespace orc {
+
+  StripeStreams::~StripeStreams() {
+    // PASS
+  }
+
+  inline RleVersion convertRleVersion(proto::ColumnEncoding_Kind kind) {
+    switch (static_cast<int64_t>(kind)) {
+    case proto::ColumnEncoding_Kind_DIRECT:
+    case proto::ColumnEncoding_Kind_DICTIONARY:
+      return RleVersion_1;
+    case proto::ColumnEncoding_Kind_DIRECT_V2:
+    case proto::ColumnEncoding_Kind_DICTIONARY_V2:
+      return RleVersion_2;
+    default:
+      throw ParseError("Unknown encoding in convertRleVersion");
+    }
+  }
+
+  ColumnReader::ColumnReader(const Type& type,
+                             StripeStreams& stripe
+                             ): columnId(type.getColumnId()),
+                                memoryPool(stripe.getMemoryPool()) {
+    std::unique_ptr<SeekableInputStream> stream =
+      stripe.getStream(columnId, proto::Stream_Kind_PRESENT, true);
+    if (stream.get()) {
+      notNullDecoder = createBooleanRleDecoder(std::move(stream));
+    }
+  }
+
+  ColumnReader::~ColumnReader() {
+    // PASS
+  }
+
+  uint64_t ColumnReader::skip(uint64_t numValues) {
+    ByteRleDecoder* decoder = notNullDecoder.get();
+    if (decoder) {
+      // page through the values that we want to skip
+      // and count how many are non-null
+      const size_t MAX_BUFFER_SIZE = 32768;
+      size_t bufferSize = std::min(MAX_BUFFER_SIZE,
+                                   static_cast<size_t>(numValues));
+      char buffer[MAX_BUFFER_SIZE];
+      uint64_t remaining = numValues;
+      while (remaining > 0) {
+        uint64_t chunkSize =
+          std::min(remaining,
+                   static_cast<uint64_t>(bufferSize));
+        decoder->next(buffer, chunkSize, 0);
+        remaining -= chunkSize;
+        for(uint64_t i=0; i < chunkSize; ++i) {
+          if (!buffer[i]) {
+            numValues -= 1;
+          }
+        }
+      }
+    }
+    return numValues;
+  }
+
+  void ColumnReader::next(ColumnVectorBatch& rowBatch,
+                          uint64_t numValues,
+                          char* incomingMask) {
+    if (numValues > rowBatch.capacity) {
+      rowBatch.resize(numValues);
+    }
+    rowBatch.numElements = numValues;
+    ByteRleDecoder* decoder = notNullDecoder.get();
+    if (decoder) {
+      char* notNullArray = rowBatch.notNull.data();
+      decoder->next(notNullArray, numValues, incomingMask);
+      // check to see if there are nulls in this batch
+      for(uint64_t i=0; i < numValues; ++i) {
+        if (!notNullArray[i]) {
+          rowBatch.hasNulls = true;
+          return;
+        }
+      }
+    } else if (incomingMask) {
+      // If we don't have a notNull stream, copy the incomingMask
+      rowBatch.hasNulls = true;
+      memcpy(rowBatch.notNull.data(), incomingMask, numValues);
+      return;
+    }
+    rowBatch.hasNulls = false;
+  }
+
+  /**
+   * Expand an array of bytes in place to the corresponding array of longs.
+   * Has to work backwards so that they data isn't clobbered during the
+   * expansion.
+   * @param buffer the array of chars and array of longs that need to be
+   *        expanded
+   * @param numValues the number of bytes to convert to longs
+   */
+  void expandBytesToLongs(int64_t* buffer, uint64_t numValues) {
+    for(size_t i=numValues - 1; i < numValues; --i) {
+      buffer[i] = reinterpret_cast<char *>(buffer)[i];
+    }
+  }
+
+  class BooleanColumnReader: public ColumnReader {
+  private:
+    std::unique_ptr<orc::ByteRleDecoder> rle;
+
+  public:
+    BooleanColumnReader(const Type& type, StripeStreams& stipe);
+    ~BooleanColumnReader();
+
+    uint64_t skip(uint64_t numValues) override;
+
+    void next(ColumnVectorBatch& rowBatch,
+              uint64_t numValues,
+              char* notNull) override;
+  };
+
+  BooleanColumnReader::BooleanColumnReader(const Type& type,
+                                           StripeStreams& stripe
+                                           ): ColumnReader(type, stripe){
+    rle = createBooleanRleDecoder(stripe.getStream(columnId,
+                                                   proto::Stream_Kind_DATA,
+                                                   true));
+  }
+
+  BooleanColumnReader::~BooleanColumnReader() {
+    // PASS
+  }
+
+  uint64_t BooleanColumnReader::skip(uint64_t numValues) {
+    numValues = ColumnReader::skip(numValues);
+    rle->skip(numValues);
+    return numValues;
+  }
+
+  void BooleanColumnReader::next(ColumnVectorBatch& rowBatch,
+                                 uint64_t numValues,
+                                 char *notNull) {
+    ColumnReader::next(rowBatch, numValues, notNull);
+    // Since the byte rle places the output in a char* instead of long*,
+    // 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);
+    expandBytesToLongs(ptr, numValues);
+  }
+
+  class ByteColumnReader: public ColumnReader {
+  private:
+    std::unique_ptr<orc::ByteRleDecoder> rle;
+
+  public:
+    ByteColumnReader(const Type& type, StripeStreams& stipe);
+    ~ByteColumnReader();
+
+    uint64_t skip(uint64_t numValues) override;
+
+    void next(ColumnVectorBatch& rowBatch,
+              uint64_t numValues,
+              char* notNull) override;
+  };
+
+  ByteColumnReader::ByteColumnReader(const Type& type,
+                                           StripeStreams& stripe
+                                           ): ColumnReader(type, stripe){
+    rle = createByteRleDecoder(stripe.getStream(columnId,
+                                                proto::Stream_Kind_DATA,
+                                                true));
+  }
+
+  ByteColumnReader::~ByteColumnReader() {
+    // PASS
+  }
+
+  uint64_t ByteColumnReader::skip(uint64_t numValues) {
+    numValues = ColumnReader::skip(numValues);
+    rle->skip(numValues);
+    return numValues;
+  }
+
+  void ByteColumnReader::next(ColumnVectorBatch& rowBatch,
+                              uint64_t numValues,
+                              char *notNull) {
+    ColumnReader::next(rowBatch, numValues, notNull);
+    // Since the byte rle places the output in a char* instead of long*,
+    // 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);
+    expandBytesToLongs(ptr, numValues);
+  }
+
+  class IntegerColumnReader: public ColumnReader {
+  protected:
+    std::unique_ptr<orc::RleDecoder> rle;
+
+  public:
+    IntegerColumnReader(const Type& type, StripeStreams& stripe);
+    ~IntegerColumnReader();
+
+    uint64_t skip(uint64_t numValues) override;
+
+    void next(ColumnVectorBatch& rowBatch,
+              uint64_t numValues,
+              char* notNull) override;
+  };
+
+  IntegerColumnReader::IntegerColumnReader(const Type& type,
+                                           StripeStreams& stripe
+                                           ): ColumnReader(type, stripe) {
+    RleVersion vers = convertRleVersion(stripe.getEncoding(columnId).kind());
+    rle = createRleDecoder(stripe.getStream(columnId,
+                                            proto::Stream_Kind_DATA,
+                                            true),
+                           true, vers, memoryPool);
+  }
+
+  IntegerColumnReader::~IntegerColumnReader() {
+    // PASS
+  }
+
+  uint64_t IntegerColumnReader::skip(uint64_t numValues) {
+    numValues = ColumnReader::skip(numValues);
+    rle->skip(numValues);
+    return numValues;
+  }
+
+  void IntegerColumnReader::next(ColumnVectorBatch& rowBatch,
+                                 uint64_t numValues,
+                                 char *notNull) {
+    ColumnReader::next(rowBatch, numValues, notNull);
+    rle->next(dynamic_cast<LongVectorBatch&>(rowBatch).data.data(),
+              numValues, rowBatch.hasNulls ? rowBatch.notNull.data() : 0);
+  }
+
+  class TimestampColumnReader: public IntegerColumnReader {
+  private:
+    std::unique_ptr<orc::RleDecoder> nanoRle;
+    DataBuffer<int64_t> nanoBuffer;
+
+  public:
+    TimestampColumnReader(const Type& type, StripeStreams& stripe);
+    ~TimestampColumnReader();
+
+    uint64_t skip(uint64_t numValues) override;
+
+    void next(ColumnVectorBatch& rowBatch,
+              uint64_t numValues,
+              char* notNull) override;
+  };
+
+
+  TimestampColumnReader::TimestampColumnReader(const Type& type,
+                                               StripeStreams& stripe
+                                               ): IntegerColumnReader(type,
+                                                                      stripe),
+                                                  nanoBuffer(memoryPool, 1024){
+    RleVersion vers = convertRleVersion(stripe.getEncoding(columnId).kind());
+    nanoRle = createRleDecoder(stripe.getStream(columnId,
+                                                proto::Stream_Kind_SECONDARY,
+                                                true),
+                               false, vers, memoryPool);
+  }
+
+  TimestampColumnReader::~TimestampColumnReader() {
+    // PASS
+  }
+
+  uint64_t TimestampColumnReader::skip(uint64_t numValues) {
+    numValues = IntegerColumnReader::skip(numValues);
+    nanoRle->skip(numValues);
+    return numValues;
+  }
+
+  void TimestampColumnReader::next(ColumnVectorBatch& rowBatch,
+                                 uint64_t numValues,
+                                 char *notNull) {
+    ColumnReader::next(rowBatch, numValues, notNull);
+    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : nullptr;
+    int64_t* pStamp = dynamic_cast<LongVectorBatch&>(rowBatch).data.data();
+
+    // make sure that nanoBuffer is large enough
+    if (numValues > nanoBuffer.size()) {
+      nanoBuffer.resize(numValues);
+    }
+
+    rle->next(pStamp, numValues, notNull);
+    nanoRle->next(nanoBuffer.data(), numValues, notNull);
+
+    // Construct the values
+    for(uint64_t i=0; i < numValues; i++) {
+      if (notNull == nullptr || notNull[i]) {
+        int64_t nanosec =  nanoBuffer[i] >> 3;
+        uint64_t zeros = nanoBuffer[i] & 0x7;
+        if (zeros != 0) {
+          for(uint64_t j = 0; j <= zeros; ++j) {
+            nanosec *= 10;
+          }
+        }
+        pStamp[i] =  pStamp[i] * 1000000000 + 1420070400000000000;
+        if (pStamp[i] >= 0) {
+          pStamp[i] += nanosec;
+        } else {
+          pStamp[i] -= nanosec;
+        }
+      }
+    }
+  }
+
+  class DoubleColumnReader: public ColumnReader {
+  public:
+    DoubleColumnReader(const Type& type, StripeStreams& stripe);
+    ~DoubleColumnReader();
+
+    uint64_t skip(uint64_t numValues) override;
+
+    void next(ColumnVectorBatch& rowBatch,
+              uint64_t numValues,
+              char* notNull) override;
+
+  private:
+    std::unique_ptr<SeekableInputStream> inputStream;
+    TypeKind columnKind;
+    const uint64_t bytesPerValue ;
+    const char *bufferPointer;
+    const char *bufferEnd;
+
+    unsigned char readByte() {
+      if (bufferPointer == bufferEnd) {
+        int length;
+        if (!inputStream->Next
+            (reinterpret_cast<const void**>(&bufferPointer), &length)) {
+          throw ParseError("bad read in DoubleColumnReader::next()");
+        }
+        bufferEnd = bufferPointer + length;
+      }
+      return static_cast<unsigned char>(*(bufferPointer++));
+    }
+
+    double readDouble() {
+      int64_t bits = 0;
+      for (uint64_t i=0; i < 8; i++) {
+        bits |= static_cast<int64_t>(readByte()) << (i*8);
+      }
+      double *result = reinterpret_cast<double*>(&bits);
+      return *result;
+    }
+
+    double readFloat() {
+      int32_t bits = 0;
+      for (uint64_t i=0; i < 4; i++) {
+        bits |= readByte() << (i*8);
+      }
+      float *result = reinterpret_cast<float*>(&bits);
+      return *result;
+    }
+  };
+
+  DoubleColumnReader::DoubleColumnReader(const Type& type,
+                                         StripeStreams& stripe
+                                         ): ColumnReader(type, stripe),
+                                            inputStream
+                                               (stripe.getStream
+                                                (columnId,
+                                                 proto::Stream_Kind_DATA,
+                                                 true)),
+                                            columnKind(type.getKind()),
+                                            bytesPerValue((type.getKind() ==
+                                                           FLOAT) ? 4 : 8),
+                                            bufferPointer(NULL),
+                                            bufferEnd(NULL) {
+    // PASS
+  }
+
+  DoubleColumnReader::~DoubleColumnReader() {
+    // PASS
+  }
+
+  uint64_t DoubleColumnReader::skip(uint64_t numValues) {
+    numValues = ColumnReader::skip(numValues);
+
+    if (static_cast<size_t>(bufferEnd - bufferPointer) >=
+        bytesPerValue * numValues) {
+      bufferPointer+= bytesPerValue*numValues;
+    } else {
+      inputStream->Skip(static_cast<int>(bytesPerValue*numValues -
+                                         static_cast<size_t>(bufferEnd -
+                                                             bufferPointer)));
+      bufferEnd = NULL;
+      bufferPointer = NULL;
+    }
+
+    return numValues;
+  }
+
+  void DoubleColumnReader::next(ColumnVectorBatch& rowBatch,
+                                uint64_t numValues,
+                                char *notNull) {
+    ColumnReader::next(rowBatch, numValues, notNull);
+    // update the notNull from the parent class
+    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : 0;
+    double* outArray = dynamic_cast<DoubleVectorBatch&>(rowBatch).data.data();
+
+    if (columnKind == FLOAT) {
+      if (notNull) {
+        for(size_t i=0; i < numValues; ++i) {
+          if (notNull[i]) {
+            outArray[i] = readFloat();
+          }
+        }
+      } else {
+        for(size_t i=0; i < numValues; ++i) {
+          outArray[i] = readFloat();
+        }
+      }
+    } else {
+      if (notNull) {
+        for(size_t i=0; i < numValues; ++i) {
+          if (notNull[i]) {
+            outArray[i] = readDouble();
+          }
+        }
+      } else {
+        for(size_t i=0; i < numValues; ++i) {
+          outArray[i] = readDouble();
+        }
+      }
+    }
+  }
+
+  void readFully(char* buffer, int64_t bufferSize, SeekableInputStream* stream) {
+    int64_t posn = 0;
+    while (posn < bufferSize) {
+      const void* chunk;
+      int length;
+      if (!stream->Next(&chunk, &length)) {
+        throw ParseError("bad read in readFully");
+      }
+      memcpy(buffer + posn, chunk, static_cast<size_t>(length));
+      posn += length;
+    }
+  }
+
+  class StringDictionaryColumnReader: public ColumnReader {
+  private:
+    DataBuffer<char> dictionaryBlob;
+    DataBuffer<int64_t> dictionaryOffset;
+    std::unique_ptr<RleDecoder> rle;
+    uint64_t dictionaryCount;
+
+  public:
+    StringDictionaryColumnReader(const Type& type, StripeStreams& stipe);
+    ~StringDictionaryColumnReader();
+
+    uint64_t skip(uint64_t numValues) override;
+
+    void next(ColumnVectorBatch& rowBatch,
+              uint64_t numValues,
+              char *notNull) override;
+  };
+
+  StringDictionaryColumnReader::StringDictionaryColumnReader
+             (const Type& type,
+              StripeStreams& stripe
+              ): ColumnReader(type, stripe),
+                 dictionaryBlob(stripe.getMemoryPool()),
+                 dictionaryOffset(stripe.getMemoryPool()) {
+    RleVersion rleVersion = convertRleVersion(stripe.getEncoding(columnId)
+                                                .kind());
+    dictionaryCount = stripe.getEncoding(columnId).dictionarysize();
+    rle = createRleDecoder(stripe.getStream(columnId,
+                                            proto::Stream_Kind_DATA,
+                                            true),
+                           false, rleVersion, memoryPool);
+    std::unique_ptr<RleDecoder> lengthDecoder =
+      createRleDecoder(stripe.getStream(columnId,
+                                        proto::Stream_Kind_LENGTH,
+                                        false),
+                       false, rleVersion, memoryPool);
+    dictionaryOffset.resize(dictionaryCount+1);
+    int64_t* lengthArray = dictionaryOffset.data();
+    lengthDecoder->next(lengthArray + 1, dictionaryCount, 0);
+    lengthArray[0] = 0;
+    for(uint64_t i=1; i < dictionaryCount + 1; ++i) {
+      lengthArray[i] += lengthArray[i-1];
+    }
+    int64_t blobSize = lengthArray[dictionaryCount];
+    dictionaryBlob.resize(static_cast<uint64_t>(blobSize));
+    std::unique_ptr<SeekableInputStream> blobStream =
+      stripe.getStream(columnId, proto::Stream_Kind_DICTIONARY_DATA, false);
+    readFully(dictionaryBlob.data(), blobSize, blobStream.get());
+  }
+
+  StringDictionaryColumnReader::~StringDictionaryColumnReader() {
+    // PASS
+  }
+
+  uint64_t StringDictionaryColumnReader::skip(uint64_t numValues) {
+    numValues = ColumnReader::skip(numValues);
+    rle->skip(numValues);
+    return numValues;
+  }
+
+  void StringDictionaryColumnReader::next(ColumnVectorBatch& rowBatch,
+                                          uint64_t numValues,
+                                          char *notNull) {
+    ColumnReader::next(rowBatch, numValues, notNull);
+    // update the notNull from the parent class
+    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : 0;
+    StringVectorBatch& byteBatch = dynamic_cast<StringVectorBatch&>(rowBatch);
+    char *blob = dictionaryBlob.data();
+    int64_t *dictionaryOffsets = dictionaryOffset.data();
+    char **outputStarts = byteBatch.data.data();
+    int64_t *outputLengths = byteBatch.length.data();
+    rle->next(outputLengths, numValues, notNull);
+    if (notNull) {
+      for(uint64_t i=0; i < numValues; ++i) {
+        if (notNull[i]) {
+          int64_t entry = outputLengths[i];
+          outputStarts[i] = blob + dictionaryOffsets[entry];
+          outputLengths[i] = dictionaryOffsets[entry+1] -
+            dictionaryOffsets[entry];
+        }
+      }
+    } else {
+      for(uint64_t i=0; i < numValues; ++i) {
+        int64_t entry = outputLengths[i];
+        outputStarts[i] = blob + dictionaryOffsets[entry];
+        outputLengths[i] = dictionaryOffsets[entry+1] -
+          dictionaryOffsets[entry];
+      }
+    }
+  }
+
+  class StringDirectColumnReader: public ColumnReader {
+  private:
+    DataBuffer<char> blobBuffer;
+    std::unique_ptr<RleDecoder> lengthRle;
+    std::unique_ptr<SeekableInputStream> blobStream;
+    const char *lastBuffer;
+    size_t lastBufferLength;
+
+    /**
+     * Compute the total length of the values.
+     * @param lengths the array of lengths
+     * @param notNull the array of notNull flags
+     * @param numValues the lengths of the arrays
+     * @return the total number of bytes for the non-null values
+     */
+    size_t computeSize(const int64_t *lengths, const char *notNull,
+                       uint64_t numValues);
+
+  public:
+    StringDirectColumnReader(const Type& type, StripeStreams& stipe);
+    ~StringDirectColumnReader();
+
+    uint64_t skip(uint64_t numValues) override;
+
+    void next(ColumnVectorBatch& rowBatch,
+              uint64_t numValues,
+              char *notNull) override;
+  };
+
+  StringDirectColumnReader::StringDirectColumnReader
+                 (const Type& type,
+                  StripeStreams& stripe
+                  ): ColumnReader(type, stripe),
+                     blobBuffer(stripe.getMemoryPool()) {
+    RleVersion rleVersion = convertRleVersion(stripe.getEncoding(columnId)
+                                                .kind());
+    lengthRle = createRleDecoder(stripe.getStream(columnId,
+                                                  proto::Stream_Kind_LENGTH,
+                                                  true),
+                                 false, rleVersion, memoryPool);
+    blobStream = stripe.getStream(columnId, proto::Stream_Kind_DATA, true);
+    lastBuffer = 0;
+    lastBufferLength = 0;
+  }
+
+  StringDirectColumnReader::~StringDirectColumnReader() {
+    // PASS
+  }
+
+  uint64_t StringDirectColumnReader::skip(uint64_t numValues) {
+    const size_t BUFFER_SIZE = 1024;
+    numValues = ColumnReader::skip(numValues);
+    int64_t buffer[BUFFER_SIZE];
+    uint64_t done = 0;
+    size_t totalBytes = 0;
+    // read the lengths, so we know haw many bytes to skip
+    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);
+      done += step;
+    }
+    if (totalBytes <= lastBufferLength) {
+      // subtract the needed bytes from the ones left over
+      lastBufferLength -= totalBytes;
+      lastBuffer += totalBytes;
+    } else {
+      // move the stream forward after accounting for the buffered bytes
+      totalBytes -= lastBufferLength;
+      blobStream->Skip(static_cast<int>(totalBytes));
+      lastBufferLength = 0;
+      lastBuffer = 0;
+    }
+    return numValues;
+  }
+
+  size_t StringDirectColumnReader::computeSize(const int64_t* lengths,
+                                               const char* notNull,
+                                               uint64_t numValues) {
+    size_t totalLength = 0;
+    if (notNull) {
+      for(size_t i=0; i < numValues; ++i) {
+        if (notNull[i]) {
+          totalLength += static_cast<size_t>(lengths[i]);
+        }
+      }
+    } else {
+      for(size_t i=0; i < numValues; ++i) {
+        totalLength += static_cast<size_t>(lengths[i]);
+      }
+    }
+    return totalLength;
+  }
+
+  void StringDirectColumnReader::next(ColumnVectorBatch& rowBatch,
+                                      uint64_t numValues,
+                                      char *notNull) {
+    ColumnReader::next(rowBatch, numValues, notNull);
+    // update the notNull from the parent class
+    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : 0;
+    StringVectorBatch& byteBatch = dynamic_cast<StringVectorBatch&>(rowBatch);
+    char **startPtr = byteBatch.data.data();
+    int64_t *lengthPtr = byteBatch.length.data();
+
+    // read the length vector
+    lengthRle->next(lengthPtr, numValues, notNull);
+
+    // figure out the total length of data we need from the blob stream
+    const size_t totalLength = computeSize(lengthPtr, notNull, numValues);
+
+    // Load data from the blob stream into our buffer until we have enough
+    // to get the rest directly out of the stream's buffer.
+    size_t bytesBuffered = 0;
+    blobBuffer.resize(totalLength);
+    char *ptr= blobBuffer.data();
+    while (bytesBuffered + lastBufferLength < totalLength) {
+      blobBuffer.resize(bytesBuffered + lastBufferLength);
+      memcpy(ptr + bytesBuffered, lastBuffer, lastBufferLength);
+      bytesBuffered += lastBufferLength;
+      const void* readBuffer;
+      int readLength;
+      if (!blobStream->Next(&readBuffer, &readLength)) {
+        throw ParseError("failed to read in StringDirectColumnReader.next");
+      }
+      lastBuffer = static_cast<const char*>(readBuffer);
+      lastBufferLength = static_cast<size_t>(readLength);
+    }
+
+    // Set up the start pointers for the ones that will come out of the buffer.
+    size_t filledSlots = 0;
+    size_t usedBytes = 0;
+    ptr = blobBuffer.data();
+    if (notNull) {
+      while (filledSlots < numValues &&
+             (usedBytes + static_cast<size_t>(lengthPtr[filledSlots]) <=
+              bytesBuffered)) {
+        if (notNull[filledSlots]) {
+          startPtr[filledSlots] = ptr + usedBytes;
+          usedBytes += static_cast<size_t>(lengthPtr[filledSlots]);
+        }
+        filledSlots += 1;
+      }
+    } else {
+      while (filledSlots < numValues &&
+             (usedBytes + static_cast<size_t>(lengthPtr[filledSlots]) <=
+              bytesBuffered)) {
+        startPtr[filledSlots] = ptr + usedBytes;
+        usedBytes += static_cast<size_t>(lengthPtr[filledSlots]);
+        filledSlots += 1;
+      }
+    }
+
+    // do we need to complete the last value in the blob buffer?
+    if (usedBytes < bytesBuffered) {
+      size_t moreBytes = static_cast<size_t>(lengthPtr[filledSlots]) -
+        (bytesBuffered - usedBytes);
+      blobBuffer.resize(bytesBuffered + moreBytes);
+      ptr = blobBuffer.data();
+      memcpy(ptr + bytesBuffered, lastBuffer, moreBytes);
+      lastBuffer += moreBytes;
+      lastBufferLength -= moreBytes;
+      startPtr[filledSlots++] = ptr + usedBytes;
+    }
+
+    // Finally, set up any remaining entries into the stream buffer
+    if (notNull) {
+      while (filledSlots < numValues) {
+        if (notNull[filledSlots]) {
+          startPtr[filledSlots] = const_cast<char*>(lastBuffer);
+          lastBuffer += lengthPtr[filledSlots];
+          lastBufferLength -= static_cast<size_t>(lengthPtr[filledSlots]);
+        }
+        filledSlots += 1;
+      }
+    } else {
+      while (filledSlots < numValues) {
+        startPtr[filledSlots] = const_cast<char*>(lastBuffer);
+        lastBuffer += lengthPtr[filledSlots];
+        lastBufferLength -= static_cast<size_t>(lengthPtr[filledSlots]);
+        filledSlots += 1;
+      }
+    }
+  }
+
+  class StructColumnReader: public ColumnReader {
+  private:
+    std::vector<ColumnReader*> children;
+
+  public:
+    StructColumnReader(const Type& type, StripeStreams& stipe);
+    ~StructColumnReader();
+
+    uint64_t skip(uint64_t numValues) override;
+
+    void next(ColumnVectorBatch& rowBatch,
+              uint64_t numValues,
+              char *notNull) override;
+  };
+
+  StructColumnReader::StructColumnReader(const Type& type,
+                                         StripeStreams& stripe
+                                         ): ColumnReader(type, stripe) {
+    // count the number of selected sub-columns
+    const std::vector<bool> selectedColumns = stripe.getSelectedColumns();
+    switch (static_cast<int64_t>(stripe.getEncoding(columnId).kind())) {
+    case proto::ColumnEncoding_Kind_DIRECT:
+      for(unsigned int i=0; i < type.getSubtypeCount(); ++i) {
+        const Type& child = type.getSubtype(i);
+        if (selectedColumns[static_cast<uint64_t>(child.getColumnId())]) {
+          children.push_back(buildReader(child, stripe).release());
+        }
+      }
+      break;
+    case proto::ColumnEncoding_Kind_DIRECT_V2:
+    case proto::ColumnEncoding_Kind_DICTIONARY:
+    case proto::ColumnEncoding_Kind_DICTIONARY_V2:
+    default:
+      throw ParseError("Unknown encoding for StructColumnReader");
+    }
+  }
+
+  StructColumnReader::~StructColumnReader() {
+    for (size_t i=0; i<children.size(); i++) {
+      delete children[i];
+    }
+  }
+
+  uint64_t StructColumnReader::skip(uint64_t numValues) {
+    numValues = ColumnReader::skip(numValues);
+    for(std::vector<ColumnReader*>::iterator ptr=children.begin(); ptr != children.end(); ++ptr) {
+      (*ptr)->skip(numValues);
+    }
+    return numValues;
+  }
+
+  void StructColumnReader::next(ColumnVectorBatch& rowBatch,
+                                uint64_t numValues,
+                                char *notNull) {
+    ColumnReader::next(rowBatch, numValues, notNull);
+    uint64_t i=0;
+    notNull = rowBatch.hasNulls? rowBatch.notNull.data() : 0;
+    for(std::vector<ColumnReader*>::iterator ptr=children.begin();
+        ptr != children.end(); ++ptr, ++i) {
+      (*ptr)->next(*(dynamic_cast<StructVectorBatch&>(rowBatch).fields[i]),
+                   numValues, notNull);
+    }
+  }
+
+  class ListColumnReader: public ColumnReader {
+  private:
+    std::unique_ptr<ColumnReader> child;
+    std::unique_ptr<RleDecoder> rle;
+
+  public:
+    ListColumnReader(const Type& type, StripeStreams& stipe);
+    ~ListColumnReader();
+
+    uint64_t skip(uint64_t numValues) override;
+
+    void next(ColumnVectorBatch& rowBatch,
+              uint64_t numValues,
+              char *notNull) override;
+  };
+
+  ListColumnReader::ListColumnReader(const Type& type,
+                                     StripeStreams& stripe
+                                     ): ColumnReader(type, stripe) {
+    // count the number of selected sub-columns
+    const std::vector<bool> selectedColumns = stripe.getSelectedColumns();
+    RleVersion vers = convertRleVersion(stripe.getEncoding(columnId).kind());
+    rle = createRleDecoder(stripe.getStream(columnId,
+                                            proto::Stream_Kind_LENGTH,
+                                            true),
+                           false, vers, memoryPool);
+    const Type& childType = type.getSubtype(0);
+    if (selectedColumns[static_cast<uint64_t>(childType.getColumnId())]) {
+      child = buildReader(childType, stripe);
+    }
+  }
+
+  ListColumnReader::~ListColumnReader() {
+    // PASS
+  }
+
+  uint64_t ListColumnReader::skip(uint64_t numValues) {
+    numValues = ColumnReader::skip(numValues);
+    ColumnReader *childReader = child.get();
+    if (childReader) {
+      const uint64_t BUFFER_SIZE = 1024;
+      int64_t buffer[BUFFER_SIZE];
+      uint64_t childrenElements = 0;
+      uint64_t lengthsRead = 0;
+      while (lengthsRead < numValues) {
+        uint64_t chunk = std::min(numValues - lengthsRead, BUFFER_SIZE);
+        rle->next(buffer, chunk, 0);
+        for(size_t i=0; i < chunk; ++i) {
+          childrenElements += static_cast<size_t>(buffer[i]);
+        }
+        lengthsRead += chunk;
+      }
+      childReader->skip(childrenElements);
+    } else {
+      rle->skip(numValues);
+    }
+    return numValues;
+  }
+
+  void ListColumnReader::next(ColumnVectorBatch& rowBatch,
+                              uint64_t numValues,
+                              char *notNull) {
+    ColumnReader::next(rowBatch, numValues, notNull);
+    ListVectorBatch &listBatch = dynamic_cast<ListVectorBatch&>(rowBatch);
+    int64_t* offsets = listBatch.offsets.data();
+    notNull = listBatch.hasNulls ? listBatch.notNull.data() : 0;
+    rle->next(offsets, numValues, notNull);
+    uint64_t totalChildren = 0;
+    if (notNull) {
+      for(size_t i=0; i < numValues; ++i) {
+        if (notNull[i]) {
+          uint64_t tmp = static_cast<uint64_t>(offsets[i]);
+          offsets[i] = static_cast<int64_t>(totalChildren);
+          totalChildren += tmp;
+        } else {
+          offsets[i] = static_cast<int64_t>(totalChildren);
+        }
+      }
+    } else {
+      for(size_t i=0; i < numValues; ++i) {
+        uint64_t tmp = static_cast<uint64_t>(offsets[i]);
+        offsets[i] = static_cast<int64_t>(totalChildren);
+        totalChildren += tmp;
+      }
+    }
+    offsets[numValues] = static_cast<int64_t>(totalChildren);
+    ColumnReader *childReader = child.get();
+    if (childReader) {
+      childReader->next(*(listBatch.elements.get()), totalChildren, 0);
+    }
+  }
+
+  class MapColumnReader: public ColumnReader {
+  private:
+    std::unique_ptr<ColumnReader> keyReader;
+    std::unique_ptr<ColumnReader> elementReader;
+    std::unique_ptr<RleDecoder> rle;
+
+  public:
+    MapColumnReader(const Type& type, StripeStreams& stipe);
+    ~MapColumnReader();
+
+    uint64_t skip(uint64_t numValues) override;
+
+    void next(ColumnVectorBatch& rowBatch,
+              uint64_t numValues,
+              char *notNull) override;
+  };
+
+  MapColumnReader::MapColumnReader(const Type& type,
+                                   StripeStreams& stripe
+                                   ): ColumnReader(type, stripe) {
+    // Determine if the key and/or value columns are selected
+    const std::vector<bool> selectedColumns = stripe.getSelectedColumns();
+    RleVersion vers = convertRleVersion(stripe.getEncoding(columnId).kind());
+    rle = createRleDecoder(stripe.getStream(columnId,
+                                            proto::Stream_Kind_LENGTH,
+                                            true),
+                           false, vers, memoryPool);
+    const Type& keyType = type.getSubtype(0);
+    if (selectedColumns[static_cast<uint64_t>(keyType.getColumnId())]) {
+      keyReader = buildReader(keyType, stripe);
+    }
+    const Type& elementType = type.getSubtype(1);
+    if (selectedColumns[static_cast<uint64_t>(elementType.getColumnId())]) {
+      elementReader = buildReader(elementType, stripe);
+    }
+  }
+
+  MapColumnReader::~MapColumnReader() {
+    // PASS
+  }
+
+  uint64_t MapColumnReader::skip(uint64_t numValues) {
+    numValues = ColumnReader::skip(numValues);
+    ColumnReader *rawKeyReader = keyReader.get();
+    ColumnReader *rawElementReader = elementReader.get();
+    if (rawKeyReader || rawElementReader) {
+      const uint64_t BUFFER_SIZE = 1024;
+      int64_t buffer[BUFFER_SIZE];
+      uint64_t childrenElements = 0;
+      uint64_t lengthsRead = 0;
+      while (lengthsRead < numValues) {
+        uint64_t chunk = std::min(numValues - lengthsRead, BUFFER_SIZE);
+        rle->next(buffer, chunk, 0);
+        for(size_t i=0; i < chunk; ++i) {
+          childrenElements += static_cast<size_t>(buffer[i]);
+        }
+        lengthsRead += chunk;
+      }
+      if (rawKeyReader) {
+        rawKeyReader->skip(childrenElements);
+      }
+      if (rawElementReader) {
+        rawElementReader->skip(childrenElements);
+      }
+    } else {
+      rle->skip(numValues);
+    }
+    return numValues;
+  }
+
+  void MapColumnReader::next(ColumnVectorBatch& rowBatch,
+                             uint64_t numValues,
+                             char *notNull) {
+    ColumnReader::next(rowBatch, numValues, notNull);
+    MapVectorBatch &mapBatch = dynamic_cast<MapVectorBatch&>(rowBatch);
+    int64_t* offsets = mapBatch.offsets.data();
+    notNull = mapBatch.hasNulls ? mapBatch.notNull.data() : 0;
+    rle->next(offsets, numValues, notNull);
+    uint64_t totalChildren = 0;
+    if (notNull) {
+      for(size_t i=0; i < numValues; ++i) {
+        if (notNull[i]) {
+          uint64_t tmp = static_cast<uint64_t>(offsets[i]);
+          offsets[i] = static_cast<int64_t>(totalChildren);
+          totalChildren += tmp;
+        } else {
+          offsets[i] = static_cast<int64_t>(totalChildren);
+        }
+      }
+    } else {
+      for(size_t i=0; i < numValues; ++i) {
+        uint64_t tmp = static_cast<uint64_t>(offsets[i]);
+        offsets[i] = static_cast<int64_t>(totalChildren);
+        totalChildren += tmp;
+      }
+    }
+    offsets[numValues] = static_cast<int64_t>(totalChildren);
+    ColumnReader *rawKeyReader = keyReader.get();
+    if (rawKeyReader) {
+      rawKeyReader->next(*(mapBatch.keys.get()), totalChildren, 0);
+    }
+    ColumnReader *rawElementReader = elementReader.get();
+    if (rawElementReader) {
+      rawElementReader->next(*(mapBatch.elements.get()), totalChildren, 0);
+    }
+  }
+
+  class UnionColumnReader: public ColumnReader {
+  private:
+    std::unique_ptr<ByteRleDecoder> rle;
+    std::vector<ColumnReader*> childrenReader;
+    std::vector<int64_t> childrenCounts;
+    uint64_t numChildren;
+
+  public:
+    UnionColumnReader(const Type& type, StripeStreams& stipe);
+    ~UnionColumnReader();
+
+    uint64_t skip(uint64_t numValues) override;
+
+    void next(ColumnVectorBatch& rowBatch,
+              uint64_t numValues,
+              char *notNull) override;
+  };
+
+  UnionColumnReader::UnionColumnReader(const Type& type,
+                                       StripeStreams& stripe
+                                       ): ColumnReader(type, stripe) {
+    numChildren = type.getSubtypeCount();
+    childrenReader.resize(numChildren);
+    childrenCounts.resize(numChildren);
+
+    rle = createByteRleDecoder(stripe.getStream(columnId,
+                                                proto::Stream_Kind_DATA,
+                                                true));
+    // figure out which types are selected
+    const std::vector<bool> selectedColumns = stripe.getSelectedColumns();
+    for(unsigned int i=0; i < numChildren; ++i) {
+      const Type &child = type.getSubtype(i);
+      if (selectedColumns[static_cast<size_t>(child.getColumnId())]) {
+        childrenReader[i] = buildReader(child, stripe).release();
+      }
+    }
+  }
+
+  UnionColumnReader::~UnionColumnReader() {
+    for(std::vector<ColumnReader*>::iterator itr = childrenReader.begin();
+        itr != childrenReader.end(); ++itr) {
+      delete *itr;
+    }
+  }
+
+  uint64_t UnionColumnReader::skip(uint64_t numValues) {
+    numValues = ColumnReader::skip(numValues);
+    const uint64_t BUFFER_SIZE = 1024;
+    char buffer[BUFFER_SIZE];
+    uint64_t lengthsRead = 0;
+    int64_t *counts = childrenCounts.data();
+    memset(counts, 0, sizeof(int64_t) * numChildren);
+    while (lengthsRead < numValues) {
+      uint64_t chunk = std::min(numValues - lengthsRead, BUFFER_SIZE);
+      rle->next(buffer, chunk, 0);
+      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) {
+        childrenReader[i]->skip(static_cast<uint64_t>(counts[i]));
+      }
+    }
+    return numValues;
+  }
+
+  void UnionColumnReader::next(ColumnVectorBatch& rowBatch,
+                               uint64_t numValues,
+                               char *notNull) {
+    ColumnReader::next(rowBatch, numValues, notNull);
+    UnionVectorBatch &unionBatch = dynamic_cast<UnionVectorBatch&>(rowBatch);
+    uint64_t* offsets = unionBatch.offsets.data();
+    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;
+    rle->next(reinterpret_cast<char *>(tags), numValues, notNull);
+    // set the offsets for each row
+    if (notNull) {
+      for(size_t i=0; i < numValues; ++i) {
+        if (notNull[i]) {
+          offsets[i] =
+            static_cast<uint64_t>(counts[static_cast<size_t>(tags[i])]++);
+        }
+      }
+    } else {
+      for(size_t i=0; i < numValues; ++i) {
+        offsets[i] =
+          static_cast<uint64_t>(counts[static_cast<size_t>(tags[i])]++);
+      }
+    }
+    // read the right number of each child column
+    for(size_t i=0; i < numChildren; ++i) {
+      if (childrenReader[i] != nullptr) {
+        childrenReader[i]->next(*(unionBatch.children[i]),
+                                static_cast<uint64_t>(counts[i]), nullptr);
+      }
+    }
+  }
+
+  /**
+   * Destructively convert the number from zigzag encoding to the
+   * natural signed representation.
+   */
+  void unZigZagInt128(Int128& value) {
+    bool needsNegate = value.getLowBits() & 1;
+    value >>= 1;
+    if (needsNegate) {
+      value.negate();
+      value -= 1;
+    }
+  }
+
+  class Decimal64ColumnReader: public ColumnReader {
+  public:
+    static const uint32_t MAX_PRECISION_64 = 18;
+    static const uint32_t MAX_PRECISION_128 = 38;
+    static const int64_t POWERS_OF_TEN[MAX_PRECISION_64 + 1];
+
+  protected:
+    std::unique_ptr<SeekableInputStream> valueStream;
+    int32_t precision;
+    int32_t scale;
+    const char* buffer;
+    const char* bufferEnd;
+
+    std::unique_ptr<RleDecoder> scaleDecoder;
+
+    /**
+     * Read the valueStream for more bytes.
+     */
+    void readBuffer() {
+      while (buffer == bufferEnd) {
+        int length;
+        if (!valueStream->Next(reinterpret_cast<const void**>(&buffer),
+                               &length)) {
+          throw ParseError("Read past end of stream in Decimal64ColumnReader "+
+                           valueStream->getName());
+        }
+        bufferEnd = buffer + length;
+      }
+    }
+
+    void readInt64(int64_t& value, int32_t currentScale) {
+      value = 0;
+      size_t offset = 0;
+      while (true) {
+        readBuffer();
+        unsigned char ch = static_cast<unsigned char>(*(buffer++));
+        value |= static_cast<uint64_t>(ch & 0x7f) << offset;
+        offset += 7;
+        if (!(ch & 0x80)) {
+          break;
+        }
+      }
+      value = unZigZag(static_cast<uint64_t>(value));
+      if (scale > currentScale) {
+        value *= POWERS_OF_TEN[scale - currentScale];
+      } else if (scale < currentScale) {
+        value /= POWERS_OF_TEN[currentScale - scale];
+      }
+    }
+
+  public:
+    Decimal64ColumnReader(const Type& type, StripeStreams& stipe);
+    ~Decimal64ColumnReader();
+
+    uint64_t skip(uint64_t numValues) override;
+
+    void next(ColumnVectorBatch& rowBatch,
+              uint64_t numValues,
+              char *notNull) override;
+  };
+  const uint32_t Decimal64ColumnReader::MAX_PRECISION_64;
+  const uint32_t Decimal64ColumnReader::MAX_PRECISION_128;
+  const int64_t Decimal64ColumnReader::POWERS_OF_TEN[MAX_PRECISION_64 + 1]=
+    {1,
+     10,
+     100,
+     1000,
+     10000,
+     100000,
+     1000000,
+     10000000,
+     100000000,
+     1000000000,
+     10000000000,
+     100000000000,
+     1000000000000,
+     10000000000000,
+     100000000000000,
+     1000000000000000,
+     10000000000000000,
+     100000000000000000,
+     1000000000000000000};
+
+  Decimal64ColumnReader::Decimal64ColumnReader(const Type& type,
+                                               StripeStreams& stripe
+                                               ): ColumnReader(type, stripe) {
+    scale = static_cast<int32_t>(type.getScale());
+    precision = static_cast<int32_t>(type.getPrecision());
+    valueStream = stripe.getStream(columnId, proto::Stream_Kind_DATA, true);
+    buffer = nullptr;
+    bufferEnd = nullptr;
+    RleVersion vers = convertRleVersion(stripe.getEncoding(columnId).kind());
+    scaleDecoder = createRleDecoder(stripe.getStream
+                                    (columnId,
+                                     proto::Stream_Kind_SECONDARY,
+                                     true),
+                                    true, vers, memoryPool);
+  }
+
+  Decimal64ColumnReader::~Decimal64ColumnReader() {
+    // PASS
+  }
+
+  uint64_t Decimal64ColumnReader::skip(uint64_t numValues) {
+    numValues = ColumnReader::skip(numValues);
+    uint64_t skipped = 0;
+    while (skipped < numValues) {
+      readBuffer();
+      if (!(0x80 & *(buffer++))) {
+        skipped += 1;
+      }
+    }
+    scaleDecoder->skip(numValues);
+    return numValues;
+  }
+
+  void Decimal64ColumnReader::next(ColumnVectorBatch& rowBatch,
+                                   uint64_t numValues,
+                                   char *notNull) {
+    ColumnReader::next(rowBatch, numValues, notNull);
+    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : 0;
+    Decimal64VectorBatch &batch =
+      dynamic_cast<Decimal64VectorBatch&>(rowBatch);
+    int64_t* values = batch.values.data();
+    // read the next group of scales
+    int64_t* scaleBuffer = batch.readScales.data();
+    scaleDecoder->next(scaleBuffer, numValues, notNull);
+    batch.precision = precision;
+    batch.scale = scale;
+    if (notNull) {
+      for(size_t i=0; i < numValues; ++i) {
+        if (notNull[i]) {
+          readInt64(values[i], static_cast<int32_t>(scaleBuffer[i]));
+        }
+      }
+    } else {
+      for(size_t i=0; i < numValues; ++i) {
+        readInt64(values[i], static_cast<int32_t>(scaleBuffer[i]));
+      }
+    }
+  }
+
+  void scaleInt128(Int128& value, uint32_t scale, uint32_t currentScale) {
+    if (scale > currentScale) {
+      while(scale > currentScale) {
+        uint32_t scaleAdjust =
+          std::min(Decimal64ColumnReader::MAX_PRECISION_64,
+                   scale - currentScale);
+        value *= Decimal64ColumnReader::POWERS_OF_TEN[scaleAdjust];
+        currentScale += scaleAdjust;
+      }
+    } else if (scale < currentScale) {
+      Int128 remainder;
+      while(currentScale > scale) {
+        uint32_t scaleAdjust =
+          std::min(Decimal64ColumnReader::MAX_PRECISION_64,
+                   currentScale - scale);
+        value = value.divide(Decimal64ColumnReader::POWERS_OF_TEN[scaleAdjust],
+                             remainder);
+        currentScale -= scaleAdjust;
+      }
+    }
+  }
+
+  class Decimal128ColumnReader: public Decimal64ColumnReader {
+  public:
+    Decimal128ColumnReader(const Type& type, StripeStreams& stipe);
+    ~Decimal128ColumnReader();
+
+    void next(ColumnVectorBatch& rowBatch,
+              uint64_t numValues,
+              char *notNull) override;
+
+  private:
+    void readInt128(Int128& value, int32_t currentScale) {
+      value = 0;
+      Int128 work;
+      uint32_t offset = 0;
+      while (true) {
+        readBuffer();
+        unsigned char ch = static_cast<unsigned char>(*(buffer++));
+        work = ch & 0x7f;
+        work <<= offset;
+        value |=  work;
+        offset += 7;
+        if (!(ch & 0x80)) {
+          break;
+        }
+      }
+      unZigZagInt128(value);
+      scaleInt128(value, static_cast<uint32_t>(scale), 
+                  static_cast<uint32_t>(currentScale));
+    }
+  };
+
+  Decimal128ColumnReader::Decimal128ColumnReader
+                (const Type& type,
+                 StripeStreams& stripe
+                 ): Decimal64ColumnReader(type, stripe) {
+    // PASS
+  }
+
+  Decimal128ColumnReader::~Decimal128ColumnReader() {
+    // PASS
+  }
+
+  void Decimal128ColumnReader::next(ColumnVectorBatch& rowBatch,
+                                   uint64_t numValues,
+                                   char *notNull) {
+    ColumnReader::next(rowBatch, numValues, notNull);
+    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : 0;
+    Decimal128VectorBatch &batch =
+      dynamic_cast<Decimal128VectorBatch&>(rowBatch);
+    Int128* values = batch.values.data();
+    // read the next group of scales
+    int64_t* scaleBuffer = batch.readScales.data();
+    scaleDecoder->next(scaleBuffer, numValues, notNull);
+    batch.precision = precision;
+    batch.scale = scale;
+    if (notNull) {
+      for(size_t i=0; i < numValues; ++i) {
+        if (notNull[i]) {
+          readInt128(values[i], static_cast<int32_t>(scaleBuffer[i]));
+        }
+      }
+    } else {
+      for(size_t i=0; i < numValues; ++i) {
+        readInt128(values[i], static_cast<int32_t>(scaleBuffer[i]));
+      }
+    }
+  }
+
+  class DecimalHive11ColumnReader: public Decimal64ColumnReader {
+  private:
+    bool throwOnOverflow;
+    std::ostream* errorStream;
+
+    /**
+     * Read an Int128 from the stream and correct it to the desired scale.
+     */
+    bool readInt128(Int128& value, int32_t currentScale) {
+      // -/+ 99999999999999999999999999999999999999
+      static const Int128 MIN_VALUE(-0x4b3b4ca85a86c47b, 0xf675ddc000000001);
+      static const Int128 MAX_VALUE( 0x4b3b4ca85a86c47a, 0x098a223fffffffff);
+
+      value = 0;
+      Int128 work;
+      uint32_t offset = 0;
+      bool result = true;
+      while (true) {
+        readBuffer();
+        unsigned char ch = static_cast<unsigned char>(*(buffer++));
+        work = ch & 0x7f;
+        // If we have read more than 128 bits, we flag the error, but keep
+        // reading bytes so the stream isn't thrown off.
+        if (offset > 128 || (offset == 126 && work > 3)) {
+          result = false;
+        }
+        work <<= offset;
+        value |=  work;
+        offset += 7;
+        if (!(ch & 0x80)) {
+          break;
+        }
+      }
+
+      if (!result) {
+        return result;
+      }
+      unZigZagInt128(value);
+      scaleInt128(value, static_cast<uint32_t>(scale),
+                  static_cast<uint32_t>(currentScale));
+      return value >= MIN_VALUE && value <= MAX_VALUE;
+    }
+
+  public:
+    DecimalHive11ColumnReader(const Type& type, StripeStreams& stipe);
+    ~DecimalHive11ColumnReader();
+
+    void next(ColumnVectorBatch& rowBatch,
+              uint64_t numValues,
+              char *notNull) override;
+  };
+
+  DecimalHive11ColumnReader::DecimalHive11ColumnReader
+                    (const Type& type,
+                     StripeStreams& stripe
+                     ): Decimal64ColumnReader(type, stripe) {
+    const ReaderOptions options = stripe.getReaderOptions();
+    scale = options.getForcedScaleOnHive11Decimal();
+    throwOnOverflow = options.getThrowOnHive11DecimalOverflow();
+    errorStream = options.getErrorStream();
+  }
+
+  DecimalHive11ColumnReader::~DecimalHive11ColumnReader() {
+    // PASS
+  }
+
+  void DecimalHive11ColumnReader::next(ColumnVectorBatch& rowBatch,
+                                       uint64_t numValues,
+                                       char *notNull) {
+    ColumnReader::next(rowBatch, numValues, notNull);
+    notNull = rowBatch.hasNulls ? rowBatch.notNull.data() : 0;
+    Decimal128VectorBatch &batch =
+      dynamic_cast<Decimal128VectorBatch&>(rowBatch);
+    Int128* values = batch.values.data();
+    // read the next group of scales
+    int64_t* scaleBuffer = batch.readScales.data();
+
+    scaleDecoder->next(scaleBuffer, numValues, notNull);
+
+    batch.precision = precision;
+    batch.scale = scale;
+    if (notNull) {
+      for(size_t i=0; i < numValues; ++i) {
+        if (notNull[i]) {
+          if (!readInt128(values[i],
+                          static_cast<int32_t>(scaleBuffer[i]))) {
+            if (throwOnOverflow) {
+              throw ParseError("Hive 0.11 decimal was more than 38 digits.");
+            } else {
+              *errorStream << "Warning: "
+                           << "Hive 0.11 decimal with more than 38 digits "
+                           << "replaced by NULL.\n";
+              notNull[i] = false;
+            }
+          }
+        }
+      }
+    } else {
+      for(size_t i=0; i < numValues; ++i) {
+        if (!readInt128(values[i],
+                        static_cast<int32_t>(scaleBuffer[i]))) {
+          if (throwOnOverflow) {
+            throw ParseError("Hive 0.11 decimal was more than 38 digits.");
+          } else {
+            *errorStream << "Warning: "
+                         << "Hive 0.11 decimal with more than 38 digits "
+                         << "replaced by NULL.\n";
+            batch.hasNulls = true;
+            batch.notNull[i] = false;
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Create a reader for the given stripe.
+   */
+  std::unique_ptr<ColumnReader> buildReader(const Type& type,
+                                            StripeStreams& stripe) {
+    switch (static_cast<int64_t>(type.getKind())) {
+    case DATE:
+    case INT:
+    case LONG:
+    case SHORT:
+      return std::unique_ptr<ColumnReader>(
+          new IntegerColumnReader(type, stripe));
+    case BINARY:
+    case CHAR:
+    case STRING:
+    case VARCHAR:
+      switch (static_cast<int64_t>(stripe.getEncoding(type.getColumnId()).kind())){
+      case proto::ColumnEncoding_Kind_DICTIONARY:
+      case proto::ColumnEncoding_Kind_DICTIONARY_V2:
+        return std::unique_ptr<ColumnReader>(
+            new StringDictionaryColumnReader(type, stripe));
+      case proto::ColumnEncoding_Kind_DIRECT:
+      case proto::ColumnEncoding_Kind_DIRECT_V2:
+        return std::unique_ptr<ColumnReader>(
+            new StringDirectColumnReader(type, stripe));
+      default:
+        throw NotImplementedYet("buildReader unhandled string encoding");
+      }
+
+    case BOOLEAN:
+      return std::unique_ptr<ColumnReader>(
+          new BooleanColumnReader(type, stripe));
+
+    case BYTE:
+      return std::unique_ptr<ColumnReader>(
+          new ByteColumnReader(type, stripe));
+
+    case LIST:
+      return std::unique_ptr<ColumnReader>(
+          new ListColumnReader(type, stripe));
+
+    case MAP:
+      return std::unique_ptr<ColumnReader>(
+          new MapColumnReader(type, stripe));
+
+    case UNION:
+      return std::unique_ptr<ColumnReader>(
+          new UnionColumnReader(type, stripe));
+
+    case STRUCT:
+      return std::unique_ptr<ColumnReader>(
+          new StructColumnReader(type, stripe));
+
+    case FLOAT:
+    case DOUBLE:
+      return std::unique_ptr<ColumnReader>(
+          new DoubleColumnReader(type, stripe));
+
+    case TIMESTAMP:
+      return std::unique_ptr<ColumnReader>
+        (new TimestampColumnReader(type, stripe));
+
+    case DECIMAL:
+      // is this a Hive 0.11 or 0.12 file?
+      if (type.getPrecision() == 0) {
+        return std::unique_ptr<ColumnReader>
+          (new DecimalHive11ColumnReader(type, stripe));
+
+      // can we represent the values using int64_t?
+      } else if (type.getPrecision() <=
+                 Decimal64ColumnReader::MAX_PRECISION_64) {
+        return std::unique_ptr<ColumnReader>
+          (new Decimal64ColumnReader(type, stripe));
+
+      // otherwise we use the Int128 implementation
+      } else {
+        return std::unique_ptr<ColumnReader>
+          (new Decimal128ColumnReader(type, stripe));
+      }
+
+    default:
+      throw NotImplementedYet("buildReader unhandled type");
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/ColumnReader.hh
----------------------------------------------------------------------
diff --git a/c++/src/ColumnReader.hh b/c++/src/ColumnReader.hh
new file mode 100644
index 0000000..b90c942
--- /dev/null
+++ b/c++/src/ColumnReader.hh
@@ -0,0 +1,109 @@
+/**
+ * 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 ORC_COLUMN_READER_HH
+#define ORC_COLUMN_READER_HH
+
+#include "orc/Vector.hh"
+#include "ByteRLE.hh"
+#include "Compression.hh"
+#include "wrap/orc-proto-wrapper.hh"
+
+namespace orc {
+
+  class StripeStreams {
+  public:
+    virtual ~StripeStreams();
+
+    /**
+     * Get the reader options.
+     */
+    virtual const ReaderOptions& getReaderOptions() const = 0;
+
+    /**
+     * Get the array of booleans for which columns are selected.
+     * @return the address of an array which contains true at the index of
+     *    each columnId is selected.
+     */
+    virtual const std::vector<bool> getSelectedColumns() const = 0;
+
+    /**
+     * Get the encoding for the given column for this stripe.
+     */
+    virtual proto::ColumnEncoding getEncoding(int64_t columnId) const = 0;
+
+    /**
+     * Get the stream for the given column/kind in this stripe.
+     * @param columnId the id of the column
+     * @param kind the kind of the stream
+     * @param shouldStream should the reading page the stream in
+     * @return the new stream
+     */
+    virtual std::unique_ptr<SeekableInputStream>
+                    getStream(int64_t columnId,
+                              proto::Stream_Kind kind,
+                              bool shouldStream) const = 0;
+
+    /**
+     * Get the memory pool for this reader.
+     */
+    virtual MemoryPool& getMemoryPool() const = 0;
+  };
+
+  /**
+   * The interface for reading ORC data types.
+   */
+  class ColumnReader {
+  protected:
+    std::unique_ptr<ByteRleDecoder> notNullDecoder;
+    int64_t columnId;
+    MemoryPool& memoryPool;
+
+  public:
+    ColumnReader(const Type& type, StripeStreams& stipe);
+
+    virtual ~ColumnReader();
+
+    /**
+     * Skip number of specified rows.
+     * @param numValues the number of values to skip
+     * @return the number of non-null values skipped
+     */
+    virtual uint64_t skip(uint64_t numValues);
+
+    /**
+     * Read the next group of values into this rowBatch.
+     * @param rowBatch the memory to read into.
+     * @param numValues the number of values to read
+     * @param notNull if null, all values are not null. Otherwise, it is
+     *           a mask (with at least numValues bytes) for which values to
+     *           set.
+     */
+    virtual void next(ColumnVectorBatch& rowBatch,
+                      uint64_t numValues,
+                      char* notNull);
+  };
+
+  /**
+   * Create a reader for the given stripe.
+   */
+  std::unique_ptr<ColumnReader> buildReader(const Type& type,
+                                            StripeStreams& stripe);
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/Compression.cc
----------------------------------------------------------------------
diff --git a/c++/src/Compression.cc b/c++/src/Compression.cc
new file mode 100644
index 0000000..81b2c1c
--- /dev/null
+++ b/c++/src/Compression.cc
@@ -0,0 +1,750 @@
+/**
+ * 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 "Compression.hh"
+#include "Exceptions.hh"
+
+#include <algorithm>
+#include <iomanip>
+#include <iostream>
+#include <sstream>
+
+#include "zlib.h"
+
+#include "wrap/snappy-wrapper.h"
+
+namespace orc {
+
+  void printBuffer(std::ostream& out,
+                   const char *buffer,
+                   uint64_t length) {
+    const uint64_t width = 24;
+    out << std::hex;
+    for(uint64_t line = 0; line < (length + width - 1) / width; ++line) {
+      out << std::setfill('0') << std::setw(7) << (line * width);
+      for(uint64_t byte = 0;
+          byte < width && line * width + byte < length; ++byte) {
+        out << " " << std::setfill('0') << std::setw(2)
+                  << static_cast<uint64_t>(0xff & buffer[line * width +
+                                                             byte]);
+      }
+      out << "\n";
+    }
+    out << std::dec;
+  }
+
+  PositionProvider::PositionProvider(const std::list<uint64_t>& posns) {
+    position = posns.begin();
+  }
+
+  uint64_t PositionProvider::next() {
+    uint64_t result = *position;
+    ++position;
+    return result;
+  }
+
+  SeekableInputStream::~SeekableInputStream() {
+    // PASS
+  }
+
+  SeekableArrayInputStream::~SeekableArrayInputStream() {
+    // PASS
+  }
+
+  SeekableArrayInputStream::SeekableArrayInputStream
+               (const unsigned char* values,
+                uint64_t size,
+                int64_t blkSize
+                ): data(reinterpret_cast<const char*>(values)) {
+    length = size;
+    position = 0;
+    blockSize = blkSize == -1 ? length : static_cast<uint64_t>(blkSize);
+  }
+
+  SeekableArrayInputStream::SeekableArrayInputStream(const char* values,
+                                                     uint64_t size,
+                                                     int64_t blkSize
+                                                     ): data(values) {
+    length = size;
+    position = 0;
+    blockSize = blkSize == -1 ? length : static_cast<uint64_t>(blkSize);
+  }
+
+  bool SeekableArrayInputStream::Next(const void** buffer, int*size) {
+    uint64_t currentSize = std::min(length - position, blockSize);
+    if (currentSize > 0) {
+      *buffer = data + position;
+      *size = static_cast<int>(currentSize);
+      position += currentSize;
+      return true;
+    }
+    *size = 0;
+    return false;
+  }
+
+  void SeekableArrayInputStream::BackUp(int count) {
+    if (count >= 0) {
+      uint64_t unsignedCount = static_cast<uint64_t>(count);
+      if (unsignedCount <= blockSize && unsignedCount <= position) {
+        position -= unsignedCount;
+      } else {
+        throw std::logic_error("Can't backup that much!");
+      }
+    }
+  }
+
+  bool SeekableArrayInputStream::Skip(int count) {
+    if (count >= 0) {
+      uint64_t unsignedCount = static_cast<uint64_t>(count);
+      if (unsignedCount + position <= length) {
+        position += unsignedCount;
+        return true;
+      } else {
+        position = length;
+      }
+    }
+    return false;
+  }
+
+  google::protobuf::int64 SeekableArrayInputStream::ByteCount() const {
+    return static_cast<google::protobuf::int64>(position);
+  }
+
+  void SeekableArrayInputStream::seek(PositionProvider& seekPosition) {
+    position = seekPosition.next();
+  }
+
+  std::string SeekableArrayInputStream::getName() const {
+    std::ostringstream result;
+    result << "SeekableArrayInputStream " << position << " of " << length;
+    return result.str();
+  }
+
+  static uint64_t computeBlock(int64_t request, uint64_t length) {
+    return std::min(length,
+                    static_cast<uint64_t>(request < 0 ?
+                                          256 * 1024 : request));
+  }
+
+  SeekableFileInputStream::SeekableFileInputStream(InputStream* stream,
+                                                   uint64_t offset,
+                                                   uint64_t byteCount,
+                                                   MemoryPool& _pool,
+                                                   int64_t _blockSize
+                                                   ):pool(_pool),
+                                                     input(stream),
+                                                     start(offset),
+                                                     length(byteCount),
+                                                     blockSize(computeBlock
+                                                               (_blockSize,
+                                                                length)) {
+
+    position = 0;
+    buffer.reset(new DataBuffer<char>(pool));
+    pushBack = 0;
+  }
+
+  SeekableFileInputStream::~SeekableFileInputStream() {
+    // PASS
+  }
+
+  bool SeekableFileInputStream::Next(const void** data, int*size) {
+    uint64_t bytesRead;
+    if (pushBack != 0) {
+      *data = buffer->data() + (buffer->size() - pushBack);
+      bytesRead = pushBack;
+    } else {
+      bytesRead = std::min(length - position, blockSize);
+      buffer->resize(bytesRead);
+      if (bytesRead > 0) {
+        input->read(buffer->data(), bytesRead, start+position);
+        *data = static_cast<void*>(buffer->data());
+      }
+    }
+    position += bytesRead;
+    pushBack = 0;
+    *size = static_cast<int>(bytesRead);
+    return bytesRead != 0;
+  }
+
+  void SeekableFileInputStream::BackUp(int signedCount) {
+    if (signedCount < 0) {
+      throw std::logic_error("can't backup negative distances");
+    }
+    uint64_t count = static_cast<uint64_t>(signedCount);
+    if (pushBack > 0) {
+      throw std::logic_error("can't backup unless we just called Next");
+    }
+    if (count > blockSize || count > position) {
+      throw std::logic_error("can't backup that far");
+    }
+    pushBack = static_cast<uint64_t>(count);
+    position -= pushBack;
+  }
+
+  bool SeekableFileInputStream::Skip(int signedCount) {
+    if (signedCount < 0) {
+      return false;
+    }
+    uint64_t count = static_cast<uint64_t>(signedCount);
+    position = std::min(position + count, length);
+    pushBack = 0;
+    return position < length;
+  }
+
+  int64_t SeekableFileInputStream::ByteCount() const {
+    return static_cast<int64_t>(position);
+  }
+
+  void SeekableFileInputStream::seek(PositionProvider& location) {
+    position = location.next();
+    if (position > length) {
+      position = length;
+      throw std::logic_error("seek too far");
+    }
+    pushBack = 0;
+  }
+
+  std::string SeekableFileInputStream::getName() const {
+    std::ostringstream result;
+    result << input->getName() << " from " << start << " for "
+           << length;
+    return result.str();
+  }
+
+  enum DecompressState { DECOMPRESS_HEADER,
+                         DECOMPRESS_START,
+                         DECOMPRESS_CONTINUE,
+                         DECOMPRESS_ORIGINAL,
+                         DECOMPRESS_EOF};
+
+  class ZlibDecompressionStream: public SeekableInputStream {
+  public:
+    ZlibDecompressionStream(std::unique_ptr<SeekableInputStream> inStream,
+                            size_t blockSize,
+                            MemoryPool& pool);
+    virtual ~ZlibDecompressionStream();
+    virtual bool Next(const void** data, int*size) override;
+    virtual void BackUp(int count) override;
+    virtual bool Skip(int count) override;
+    virtual int64_t ByteCount() const override;
+    virtual void seek(PositionProvider& position) override;
+    virtual std::string getName() const override;
+
+  private:
+    void readBuffer(bool failOnEof) {
+      int length;
+      if (!input->Next(reinterpret_cast<const void**>(&inputBuffer),
+                       &length)) {
+        if (failOnEof) {
+          throw ParseError("Read past EOF in "
+                           "ZlibDecompressionStream::readBuffer");
+        }
+        state = DECOMPRESS_EOF;
+        inputBuffer = nullptr;
+        inputBufferEnd = nullptr;
+      } else {
+        inputBufferEnd = inputBuffer + length;
+      }
+    }
+
+    uint32_t readByte(bool failOnEof) {
+      if (inputBuffer == inputBufferEnd) {
+        readBuffer(failOnEof);
+        if (state == DECOMPRESS_EOF) {
+          return 0;
+        }
+      }
+      return static_cast<unsigned char>(*(inputBuffer++));
+    }
+
+    void readHeader() {
+      uint32_t header = readByte(false);
+      if (state != DECOMPRESS_EOF) {
+        header |= readByte(true) << 8;
+        header |= readByte(true) << 16;
+        if (header & 1) {
+          state = DECOMPRESS_ORIGINAL;
+        } else {
+          state = DECOMPRESS_START;
+        }
+        remainingLength = header >> 1;
+      } else {
+        remainingLength = 0;
+      }
+    }
+
+    MemoryPool& pool;
+    const size_t blockSize;
+    std::unique_ptr<SeekableInputStream> input;
+    z_stream zstream;
+    DataBuffer<char> buffer;
+
+    // the current state
+    DecompressState state;
+
+    // the start of the current buffer
+    // This pointer is not owned by us. It is either owned by zstream or
+    // the underlying stream.
+    const char* outputBuffer;
+    // the size of the current buffer
+    size_t outputBufferLength;
+    // the size of the current chunk
+    size_t remainingLength;
+
+    // the last buffer returned from the input
+    const char *inputBuffer;
+    const char *inputBufferEnd;
+
+    // roughly the number of bytes returned
+    off_t bytesReturned;
+  };
+
+DIAGNOSTIC_PUSH
+DIAGNOSTIC_IGNORE("-Wold-style-cast")
+
+  ZlibDecompressionStream::ZlibDecompressionStream
+                   (std::unique_ptr<SeekableInputStream> inStream,
+                    size_t _blockSize,
+                    MemoryPool& _pool
+                    ): pool(_pool),
+                       blockSize(_blockSize),
+                       buffer(pool, _blockSize) {
+    input.reset(inStream.release());
+    zstream.next_in = Z_NULL;
+    zstream.avail_in = 0;
+    zstream.zalloc = Z_NULL;
+    zstream.zfree = Z_NULL;
+    zstream.opaque = Z_NULL;
+    zstream.next_out = reinterpret_cast<Bytef*>(buffer.data());
+    zstream.avail_out = static_cast<uInt>(blockSize);
+    int64_t result = inflateInit2(&zstream, -15);
+    switch (result) {
+    case Z_OK:
+      break;
+    case Z_MEM_ERROR:
+      throw std::logic_error("Memory error from inflateInit2");
+    case Z_VERSION_ERROR:
+      throw std::logic_error("Version error from inflateInit2");
+    case Z_STREAM_ERROR:
+      throw std::logic_error("Stream error from inflateInit2");
+    default:
+      throw std::logic_error("Unknown error from inflateInit2");
+    }
+    outputBuffer = nullptr;
+    outputBufferLength = 0;
+    remainingLength = 0;
+    state = DECOMPRESS_HEADER;
+    inputBuffer = nullptr;
+    inputBufferEnd = nullptr;
+    bytesReturned = 0;
+  }
+
+DIAGNOSTIC_POP
+
+  ZlibDecompressionStream::~ZlibDecompressionStream() {
+    int64_t result = inflateEnd(&zstream);
+    if (result != Z_OK) {
+      // really can't throw in destructors
+      std::cout << "Error in ~ZlibDecompressionStream() " << result << "\n";
+    }
+  }
+
+  bool ZlibDecompressionStream::Next(const void** data, int*size) {
+    // if the user pushed back, return them the partial buffer
+    if (outputBufferLength) {
+      *data = outputBuffer;
+      *size = static_cast<int>(outputBufferLength);
+      outputBuffer += outputBufferLength;
+      outputBufferLength = 0;
+      return true;
+    }
+    if (state == DECOMPRESS_HEADER || remainingLength == 0) {
+      readHeader();
+    }
+    if (state == DECOMPRESS_EOF) {
+      return false;
+    }
+    if (inputBuffer == inputBufferEnd) {
+      readBuffer(true);
+    }
+    size_t availSize =
+      std::min(static_cast<size_t>(inputBufferEnd - inputBuffer),
+               remainingLength);
+    if (state == DECOMPRESS_ORIGINAL) {
+      *data = inputBuffer;
+      *size = static_cast<int>(availSize);
+      outputBuffer = inputBuffer + availSize;
+      outputBufferLength = 0;
+    } else if (state == DECOMPRESS_START) {
+      zstream.next_in =
+        reinterpret_cast<Bytef*>(const_cast<char*>(inputBuffer));
+      zstream.avail_in = static_cast<uInt>(availSize);
+      outputBuffer = buffer.data();
+      zstream.next_out =
+        reinterpret_cast<Bytef*>(const_cast<char*>(outputBuffer));
+      zstream.avail_out = static_cast<uInt>(blockSize);
+      if (inflateReset(&zstream) != Z_OK) {
+        throw std::logic_error("Bad inflateReset in "
+                               "ZlibDecompressionStream::Next");
+      }
+      int64_t result;
+      do {
+        result = inflate(&zstream, availSize == remainingLength ? Z_FINISH :
+                         Z_SYNC_FLUSH);
+        switch (result) {
+        case Z_OK:
+          remainingLength -= availSize;
+          inputBuffer += availSize;
+          readBuffer(true);
+          availSize =
+            std::min(static_cast<size_t>(inputBufferEnd - inputBuffer),
+                     remainingLength);
+          zstream.next_in =
+            reinterpret_cast<Bytef*>(const_cast<char*>(inputBuffer));
+          zstream.avail_in = static_cast<uInt>(availSize);
+          break;
+        case Z_STREAM_END:
+          break;
+        case Z_BUF_ERROR:
+          throw std::logic_error("Buffer error in "
+                                 "ZlibDecompressionStream::Next");
+        case Z_DATA_ERROR:
+          throw std::logic_error("Data error in "
+                                 "ZlibDecompressionStream::Next");
+        case Z_STREAM_ERROR:
+          throw std::logic_error("Stream error in "
+                                 "ZlibDecompressionStream::Next");
+        default:
+          throw std::logic_error("Unknown error in "
+                                 "ZlibDecompressionStream::Next");
+        }
+      } while (result != Z_STREAM_END);
+      *size = static_cast<int>(blockSize - zstream.avail_out);
+      *data = outputBuffer;
+      outputBufferLength = 0;
+      outputBuffer += *size;
+    } else {
+      throw std::logic_error("Unknown compression state in "
+                             "ZlibDecompressionStream::Next");
+    }
+    inputBuffer += availSize;
+    remainingLength -= availSize;
+    bytesReturned += *size;
+    return true;
+  }
+
+  void ZlibDecompressionStream::BackUp(int count) {
+    if (outputBuffer == nullptr || outputBufferLength != 0) {
+      throw std::logic_error("Backup without previous Next in "
+                             "ZlibDecompressionStream");
+    }
+    outputBuffer -= static_cast<size_t>(count);
+    outputBufferLength = static_cast<size_t>(count);
+    bytesReturned -= count;
+  }
+
+  bool ZlibDecompressionStream::Skip(int count) {
+    bytesReturned += count;
+    // this is a stupid implementation for now.
+    // should skip entire blocks without decompressing
+    while (count > 0) {
+      const void *ptr;
+      int len;
+      if (!Next(&ptr, &len)) {
+        return false;
+      }
+      if (len > count) {
+        BackUp(len - count);
+        count = 0;
+      } else {
+        count -= len;
+      }
+    }
+    return true;
+  }
+
+  int64_t ZlibDecompressionStream::ByteCount() const {
+    return bytesReturned;
+  }
+
+  void ZlibDecompressionStream::seek(PositionProvider& position) {
+    input->seek(position);
+    bytesReturned = input->ByteCount();
+    if (!Skip(static_cast<int>(position.next()))) {
+      throw ParseError("Bad skip in ZlibDecompressionStream::seek");
+    }
+  }
+
+  std::string ZlibDecompressionStream::getName() const {
+    std::ostringstream result;
+    result << "zlib(" << input->getName() << ")";
+    return result.str();
+  }
+
+  class SnappyDecompressionStream: public SeekableInputStream {
+  public:
+    SnappyDecompressionStream(std::unique_ptr<SeekableInputStream> inStream,
+                              size_t blockSize,
+                              MemoryPool& pool);
+
+    virtual ~SnappyDecompressionStream() {}
+    virtual bool Next(const void** data, int*size) override;
+    virtual void BackUp(int count) override;
+    virtual bool Skip(int count) override;
+    virtual int64_t ByteCount() const override;
+    virtual void seek(PositionProvider& position) override;
+    virtual std::string getName() const override;
+
+  private:
+    void readBuffer(bool failOnEof) {
+      int length;
+      if (!input->Next(reinterpret_cast<const void**>(&inputBufferPtr),
+                       &length)) {
+        if (failOnEof) {
+          throw ParseError("SnappyDecompressionStream read past EOF");
+        }
+        state = DECOMPRESS_EOF;
+        inputBufferPtr = nullptr;
+        inputBufferPtrEnd = nullptr;
+      } else {
+        inputBufferPtrEnd = inputBufferPtr + length;
+      }
+    }
+
+    uint32_t readByte(bool failOnEof) {
+      if (inputBufferPtr == inputBufferPtrEnd) {
+        readBuffer(failOnEof);
+        if (state == DECOMPRESS_EOF) {
+          return 0;
+        }
+      }
+      return static_cast<unsigned char>(*(inputBufferPtr++));
+    }
+
+    void readHeader() {
+      uint32_t header = readByte(false);
+      if (state != DECOMPRESS_EOF) {
+        header |= readByte(true) << 8;
+        header |= readByte(true) << 16;
+        if (header & 1) {
+          state = DECOMPRESS_ORIGINAL;
+        } else {
+          state = DECOMPRESS_START;
+        }
+        remainingLength = header >> 1;
+      } else {
+        remainingLength = 0;
+      }
+    }
+
+    std::unique_ptr<SeekableInputStream> input;
+    MemoryPool& pool;
+
+    // may need to stitch together multiple input buffers;
+    // to give snappy a contiguous block
+    DataBuffer<char> inputBuffer;
+
+    // uncompressed output
+    DataBuffer<char> outputBuffer;
+
+    // the current state
+    DecompressState state;
+
+    // the start of the current output buffer
+    const char* outputBufferPtr;
+    // the size of the current output buffer
+    size_t outputBufferLength;
+
+    // the size of the current chunk
+    size_t remainingLength;
+
+    // the last buffer returned from the input
+    const char *inputBufferPtr;
+    const char *inputBufferPtrEnd;
+
+    // bytes returned by this stream
+    off_t bytesReturned;
+  };
+
+  SnappyDecompressionStream::SnappyDecompressionStream
+                   (std::unique_ptr<SeekableInputStream> inStream,
+                    size_t bufferSize,
+                    MemoryPool& _pool
+                    ) : pool(_pool),
+                        inputBuffer(pool, bufferSize),
+                        outputBuffer(pool, bufferSize),
+                        state(DECOMPRESS_HEADER),
+                        outputBufferPtr(0),
+                        outputBufferLength(0),
+                        remainingLength(0),
+                        inputBufferPtr(0),
+                        inputBufferPtrEnd(0),
+                        bytesReturned(0) {
+    input.reset(inStream.release());
+  }
+
+  bool SnappyDecompressionStream::Next(const void** data, int*size) {
+    // if the user pushed back, return them the partial buffer
+    if (outputBufferLength) {
+      *data = outputBufferPtr;
+      *size = static_cast<int>(outputBufferLength);
+      outputBufferPtr += outputBufferLength;
+      bytesReturned += outputBufferLength;
+      outputBufferLength = 0;
+      return true;
+    }
+    if (state == DECOMPRESS_HEADER || remainingLength == 0) {
+      readHeader();
+    }
+    if (state == DECOMPRESS_EOF) {
+      return false;
+    }
+    if (inputBufferPtr == inputBufferPtrEnd) {
+      readBuffer(true);
+    }
+
+    size_t availSize =
+      std::min(static_cast<size_t>(inputBufferPtrEnd - inputBufferPtr),
+               remainingLength);
+    if (state == DECOMPRESS_ORIGINAL) {
+      *data = inputBufferPtr;
+      *size = static_cast<int>(availSize);
+      outputBufferPtr = inputBufferPtr + availSize;
+      outputBufferLength = 0;
+      inputBufferPtr += availSize;
+      remainingLength -= availSize;
+    } else if (state == DECOMPRESS_START) {
+      // Get contiguous bytes of compressed block.
+      const char *compressed = inputBufferPtr;
+      if (remainingLength == availSize) {
+          inputBufferPtr += availSize;
+      } else {
+        // Did not read enough from input.
+        if (inputBuffer.capacity() < remainingLength) {
+          inputBuffer.resize(remainingLength);
+        }
+        ::memcpy(inputBuffer.data(), inputBufferPtr, availSize);
+        inputBufferPtr += availSize;
+        compressed = inputBuffer.data();
+
+        for (size_t pos = availSize; pos < remainingLength; ) {
+          readBuffer(true);
+          size_t avail =
+              std::min(static_cast<size_t>(inputBufferPtrEnd - inputBufferPtr),
+                       remainingLength - pos);
+          ::memcpy(inputBuffer.data() + pos, inputBufferPtr, avail);
+          pos += avail;
+          inputBufferPtr += avail;
+        }
+      }
+
+      if (!snappy::GetUncompressedLength(compressed, remainingLength,
+                                         &outputBufferLength)) {
+        throw ParseError("SnappyDecompressionStream choked on corrupt input");
+      }
+
+      if (outputBufferLength > outputBuffer.capacity()) {
+        throw std::logic_error("uncompressed length exceeds block size");
+      }
+
+      if (!snappy::RawUncompress(compressed, remainingLength,
+                                 outputBuffer.data())) {
+        throw ParseError("SnappyDecompressionStream choked on corrupt input");
+      }
+
+      remainingLength = 0;
+      state = DECOMPRESS_HEADER;
+      *data = outputBuffer.data();
+      *size = static_cast<int>(outputBufferLength);
+      outputBufferPtr = outputBuffer.data() + outputBufferLength;
+      outputBufferLength = 0;
+    }
+
+    bytesReturned += *size;
+    return true;
+  }
+
+  void SnappyDecompressionStream::BackUp(int count) {
+    if (outputBufferPtr == nullptr || outputBufferLength != 0) {
+      throw std::logic_error("Backup without previous Next in "
+                             "SnappyDecompressionStream");
+    }
+    outputBufferPtr -= static_cast<size_t>(count);
+    outputBufferLength = static_cast<size_t>(count);
+    bytesReturned -= count;
+  }
+
+  bool SnappyDecompressionStream::Skip(int count) {
+    bytesReturned += count;
+    // this is a stupid implementation for now.
+    // should skip entire blocks without decompressing
+    while (count > 0) {
+      const void *ptr;
+      int len;
+      if (!Next(&ptr, &len)) {
+        return false;
+      }
+      if (len > count) {
+        BackUp(len - count);
+        count = 0;
+      } else {
+        count -= len;
+      }
+    }
+    return true;
+  }
+
+  int64_t SnappyDecompressionStream::ByteCount() const {
+    return bytesReturned;
+  }
+
+  void SnappyDecompressionStream::seek(PositionProvider& position) {
+    input->seek(position);
+    if (!Skip(static_cast<int>(position.next()))) {
+      throw ParseError("Bad skip in SnappyDecompressionStream::seek");
+    }
+  }
+
+  std::string SnappyDecompressionStream::getName() const {
+    std::ostringstream result;
+    result << "snappy(" << input->getName() << ")";
+    return result.str();
+  }
+
+  std::unique_ptr<SeekableInputStream>
+     createDecompressor(CompressionKind kind,
+                        std::unique_ptr<SeekableInputStream> input,
+                        uint64_t blockSize,
+                        MemoryPool& pool) {
+    switch (static_cast<int64_t>(kind)) {
+    case CompressionKind_NONE:
+      return std::move(input);
+    case CompressionKind_ZLIB:
+      return std::unique_ptr<SeekableInputStream>
+        (new ZlibDecompressionStream(std::move(input), blockSize, pool));
+    case CompressionKind_SNAPPY:
+      return std::unique_ptr<SeekableInputStream>
+        (new SnappyDecompressionStream(std::move(input), blockSize, pool));
+    case CompressionKind_LZO:
+    default:
+      throw NotImplementedYet("compression codec");
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/Compression.hh
----------------------------------------------------------------------
diff --git a/c++/src/Compression.hh b/c++/src/Compression.hh
new file mode 100644
index 0000000..2c02584
--- /dev/null
+++ b/c++/src/Compression.hh
@@ -0,0 +1,130 @@
+/**
+ * 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 ORC_COMPRESSION_HH
+#define ORC_COMPRESSION_HH
+
+#include "orc/OrcFile.hh"
+
+#include "Adaptor.hh"
+#include "wrap/zero-copy-stream-wrapper.h"
+
+#include <list>
+#include <vector>
+#include <fstream>
+#include <iostream>
+#include <sstream>
+#include <memory>
+
+namespace orc {
+
+  void printBuffer(std::ostream& out,
+                   const char *buffer,
+                   uint64_t length);
+
+  class PositionProvider {
+  private:
+    std::list<uint64_t>::const_iterator position;
+  public:
+    PositionProvider(const std::list<uint64_t>& positions);
+    uint64_t next();
+  };
+
+  /**
+   * A subclass of Google's ZeroCopyInputStream that supports seek.
+   * By extending Google's class, we get the ability to pass it directly
+   * to the protobuf readers.
+   */
+  class SeekableInputStream: public google::protobuf::io::ZeroCopyInputStream {
+  public:
+    virtual ~SeekableInputStream();
+    virtual void seek(PositionProvider& position) = 0;
+    virtual std::string getName() const = 0;
+  };
+
+  /**
+   * Create a seekable input stream based on a memory range.
+   */
+  class SeekableArrayInputStream: public SeekableInputStream {
+  private:
+    const char* data;
+    uint64_t length;
+    uint64_t position;
+    uint64_t blockSize;
+
+  public:
+    SeekableArrayInputStream(const unsigned char* list,
+                             uint64_t length,
+                             int64_t block_size = -1);
+    SeekableArrayInputStream(const char* list,
+                             uint64_t length,
+                             int64_t block_size = -1);
+    virtual ~SeekableArrayInputStream();
+    virtual bool Next(const void** data, int*size) override;
+    virtual void BackUp(int count) override;
+    virtual bool Skip(int count) override;
+    virtual google::protobuf::int64 ByteCount() const override;
+    virtual void seek(PositionProvider& position) override;
+    virtual std::string getName() const override;
+  };
+
+  /**
+   * Create a seekable input stream based on an input stream.
+   */
+  class SeekableFileInputStream: public SeekableInputStream {
+  private:
+    MemoryPool& pool;
+    InputStream* const input;
+    const uint64_t start;
+    const uint64_t length;
+    const uint64_t blockSize;
+    std::unique_ptr<DataBuffer<char> > buffer;
+    uint64_t position;
+    uint64_t pushBack;
+
+  public:
+    SeekableFileInputStream(InputStream* input,
+                            uint64_t offset,
+                            uint64_t byteCount,
+                            MemoryPool& pool,
+                            int64_t blockSize = -1);
+    virtual ~SeekableFileInputStream();
+
+    virtual bool Next(const void** data, int*size) override;
+    virtual void BackUp(int count) override;
+    virtual bool Skip(int count) override;
+    virtual int64_t ByteCount() const override;
+    virtual void seek(PositionProvider& position) override;
+    virtual std::string getName() const override;
+  };
+
+  /**
+   * Create a decompressor for the given compression kind.
+   * @param kind the compression type to implement
+   * @param input the input stream that is the underlying source
+   * @param bufferSize the maximum size of the buffer
+   * @param pool the memory pool
+   */
+  std::unique_ptr<SeekableInputStream>
+     createDecompressor(CompressionKind kind,
+                        std::unique_ptr<SeekableInputStream> input,
+                        uint64_t bufferSize,
+                        MemoryPool& pool);
+}
+
+#endif


[23/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
ORC-23. Simplify directory structure.

This closes #1


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

Branch: refs/heads/master
Commit: 7f55b45370529b6a68a1b62068352336901609b8
Parents: 486433f
Author: Owen O'Malley <om...@apache.org>
Authored: Thu Jul 2 14:19:40 2015 -0700
Committer: Owen O'Malley <om...@apache.org>
Committed: Mon Jul 6 14:51:55 2015 -0700

----------------------------------------------------------------------
 CMakeLists.txt                          |    2 +-
 c++/src/Adaptor.hh.in                   |   92 +
 c++/src/ByteRLE.cc                      |  352 +++
 c++/src/ByteRLE.hh                      |   71 +
 c++/src/C09Adapter.cc                   |   28 +
 c++/src/CMakeLists.txt                  |   34 +-
 c++/src/ColumnPrinter.cc                |  725 +++++
 c++/src/ColumnReader.cc                 | 1558 ++++++++++
 c++/src/ColumnReader.hh                 |  109 +
 c++/src/Compression.cc                  |  750 +++++
 c++/src/Compression.hh                  |  130 +
 c++/src/Exceptions.cc                   |   59 +
 c++/src/Exceptions.hh                   |   50 +
 c++/src/Int128.cc                       |  438 +++
 c++/src/MemoryPool.cc                   |  232 ++
 c++/src/OrcFile.cc                      |  102 +
 c++/src/RLE.cc                          |   47 +
 c++/src/RLE.hh                          |   78 +
 c++/src/RLEv1.cc                        |  190 ++
 c++/src/RLEv1.hh                        |   70 +
 c++/src/RLEv2.cc                        |  482 +++
 c++/src/RLEv2.hh                        |  175 ++
 c++/src/Reader.cc                       | 1903 ++++++++++++
 c++/src/TypeImpl.cc                     |  369 +++
 c++/src/TypeImpl.hh                     |  101 +
 c++/src/Vector.cc                       |  306 ++
 c++/src/orc/Adaptor.hh.in               |   92 -
 c++/src/orc/ByteRLE.cc                  |  352 ---
 c++/src/orc/ByteRLE.hh                  |   71 -
 c++/src/orc/C09Adapter.cc               |   28 -
 c++/src/orc/ColumnPrinter.cc            |  724 -----
 c++/src/orc/ColumnReader.cc             | 1557 ----------
 c++/src/orc/ColumnReader.hh             |  109 -
 c++/src/orc/Compression.cc              |  751 -----
 c++/src/orc/Compression.hh              |  129 -
 c++/src/orc/Exceptions.cc               |   59 -
 c++/src/orc/Exceptions.hh               |   50 -
 c++/src/orc/Int128.cc                   |  438 ---
 c++/src/orc/MemoryPool.cc               |  231 --
 c++/src/orc/OrcFile.cc                  |  101 -
 c++/src/orc/RLE.cc                      |   47 -
 c++/src/orc/RLE.hh                      |   78 -
 c++/src/orc/RLEv1.cc                    |  190 --
 c++/src/orc/RLEv1.hh                    |   70 -
 c++/src/orc/RLEv2.cc                    |  482 ---
 c++/src/orc/RLEv2.hh                    |  175 --
 c++/src/orc/Reader.cc                   | 1902 ------------
 c++/src/orc/TypeImpl.cc                 |  369 ---
 c++/src/orc/TypeImpl.hh                 |  101 -
 c++/src/orc/Vector.cc                   |  305 --
 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/snappy-wrapper.h           |    2 +-
 c++/src/wrap/zero-copy-stream-wrapper.h |    2 +-
 c++/test/CMakeLists.txt                 |   14 +-
 c++/test/OrcTest.hh                     |   25 +
 c++/test/TestByteRle.cc                 | 1385 +++++++++
 c++/test/TestColumnPrinter.cc           |  588 ++++
 c++/test/TestColumnReader.cc            | 4308 +++++++++++++++++++++++++
 c++/test/TestCompression.cc             |  647 ++++
 c++/test/TestDriver.cc                  |   33 +
 c++/test/TestInt128.cc                  |  587 ++++
 c++/test/TestRle.cc                     | 2639 ++++++++++++++++
 c++/test/orc/OrcTest.hh                 |   25 -
 c++/test/orc/TestByteRle.cc             | 1385 ---------
 c++/test/orc/TestColumnPrinter.cc       |  588 ----
 c++/test/orc/TestColumnReader.cc        | 4309 --------------------------
 c++/test/orc/TestCompression.cc         |  647 ----
 c++/test/orc/TestDriver.cc              |   33 -
 c++/test/orc/TestInt128.cc              |  587 ----
 c++/test/orc/TestRle.cc                 | 2639 ----------------
 tools-c++/CMakeLists.txt                |   14 -
 tools-c++/src/CMakeLists.txt            |   64 -
 tools-c++/src/FileContents.cc           |   62 -
 tools-c++/src/FileMetadata.cc           |  181 --
 tools-c++/src/FileScan.cc               |   56 -
 tools-c++/src/FileStatistics.cc         |   75 -
 tools-c++/test/CMakeLists.txt           |   40 -
 tools-c++/test/TestReader.cc            | 2950 ------------------
 tools-c++/test/ToolTest.cc              |   43 -
 tools-c++/test/ToolTest.hh              |   21 -
 tools-c++/test/gzip.cc                  |  115 -
 tools-c++/test/gzip.hh                  |   52 -
 tools/CMakeLists.txt                    |   14 +
 tools/src/CMakeLists.txt                |   64 +
 tools/src/FileContents.cc               |   63 +
 tools/src/FileMetadata.cc               |  181 ++
 tools/src/FileScan.cc                   |   57 +
 tools/src/FileStatistics.cc             |   75 +
 tools/test/CMakeLists.txt               |   40 +
 tools/test/TestReader.cc                | 2950 ++++++++++++++++++
 tools/test/ToolTest.cc                  |   43 +
 tools/test/ToolTest.hh                  |   21 +
 tools/test/gzip.cc                      |  115 +
 tools/test/gzip.hh                      |   52 +
 98 files changed, 22336 insertions(+), 22329 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index a25551e..9981c65 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -95,7 +95,7 @@ enable_testing()
 set (EXAMPLE_DIRECTORY ${CMAKE_SOURCE_DIR}/examples)
 
 add_subdirectory(c++)
-add_subdirectory(tools-c++)
+add_subdirectory(tools)
 
 # Add another target called test-out that prints the results on failure
 if (CMAKE_CONFIGURATION_TYPES)

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/Adaptor.hh.in
----------------------------------------------------------------------
diff --git a/c++/src/Adaptor.hh.in b/c++/src/Adaptor.hh.in
new file mode 100644
index 0000000..5b3a677
--- /dev/null
+++ b/c++/src/Adaptor.hh.in
@@ -0,0 +1,92 @@
+/**
+ * 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 ADAPTER_HH
+#define ADAPTER_HH
+
+#cmakedefine INT64_IS_LL
+#cmakedefine HAS_PREAD
+#cmakedefine HAS_STOLL
+#cmakedefine HAS_DIAGNOSTIC_PUSH
+#cmakedefine HAS_PRE_1970
+#cmakedefine HAS_POST_2038
+
+#include "orc/orc-config.hh"
+#include <string>
+
+#ifndef HAS_STOLL
+  // A poor man's stoll that converts str to a long long int base 10
+  namespace std {
+    int64_t stoll(std::string str);
+  }
+#endif
+
+#ifndef HAS_PREAD
+  ssize_t pread(int fd, void *buf, size_t count, off_t offset);
+#endif
+
+#ifdef INT64_IS_LL
+  #define INT64_FORMAT_STRING "ll"
+#else
+  #define INT64_FORMAT_STRING "l"
+#endif
+
+#ifndef ORC_CXX_HAS_NOEXCEPT
+  #define noexcept ORC_NOEXCEPT
+#endif
+
+#ifndef ORC_CXX_HAS_OVERRIDE
+  #define override ORC_OVERRIDE
+#endif
+
+#ifdef HAS_DIAGNOSTIC_PUSH
+  #ifdef __clang__
+    #define DIAGNOSTIC_PUSH _Pragma("clang diagnostic push")
+    #define DIAGNOSTIC_POP _Pragma("clang diagnostic pop")
+  #elif defined(__GNUC__)
+    #define DIAGNOSTIC_PUSH _Pragma("GCC diagnostic push")
+    #define DIAGNOSTIC_POP _Pragma("GCC diagnostic pop")
+  #else
+    #error("Unknown compiler")
+  #endif
+#else
+  #define DIAGNOSTIC_PUSH
+  #define DIAGNOSTIC_POP
+#endif
+
+#define PRAGMA(TXT) _Pragma(#TXT)
+
+#ifdef __clang__
+  #define DIAGNOSTIC_IGNORE(XXX) PRAGMA(clang diagnostic ignored XXX)
+#elif defined(__GNUC__)
+  #define DIAGNOSTIC_IGNORE(XXX) PRAGMA(GCC diagnostic ignored XXX)
+#else
+  #define DIAGNOSTIC_IGNORE(XXX)
+#endif
+
+#ifndef ORC_CXX_HAS_UNIQUE_PTR
+  #define unique_ptr auto_ptr
+#endif
+
+#ifndef UINT32_MAX
+  #define UINT32_MAX 0xffffffff
+#endif
+
+#define GTEST_LANG_CXX11 0
+
+#endif /* ADAPTER_HH */

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/ByteRLE.cc
----------------------------------------------------------------------
diff --git a/c++/src/ByteRLE.cc b/c++/src/ByteRLE.cc
new file mode 100644
index 0000000..2c0032c
--- /dev/null
+++ b/c++/src/ByteRLE.cc
@@ -0,0 +1,352 @@
+/**
+ * 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 <algorithm>
+#include <iostream>
+#include <string.h>
+#include <utility>
+
+#include "ByteRLE.hh"
+#include "Exceptions.hh"
+
+namespace orc {
+
+  const size_t MINIMUM_REPEAT = 3;
+
+  ByteRleDecoder::~ByteRleDecoder() {
+    // PASS
+  }
+
+  class ByteRleDecoderImpl: public ByteRleDecoder {
+  public:
+    ByteRleDecoderImpl(std::unique_ptr<SeekableInputStream> input);
+
+    virtual ~ByteRleDecoderImpl();
+
+    /**
+     * Seek to a particular spot.
+     */
+    virtual void seek(PositionProvider&);
+
+    /**
+     * Seek over a given number of values.
+     */
+    virtual void skip(uint64_t numValues);
+
+    /**
+     * Read a number of values into the batch.
+     */
+    virtual void next(char* data, uint64_t numValues, char* notNull);
+
+  protected:
+    inline void nextBuffer();
+    inline signed char readByte();
+    inline void readHeader();
+
+    std::unique_ptr<SeekableInputStream> inputStream;
+    size_t remainingValues;
+    char value;
+    const char* bufferStart;
+    const char* bufferEnd;
+    bool repeating;
+  };
+
+  void ByteRleDecoderImpl::nextBuffer() {
+    int bufferLength;
+    const void* bufferPointer;
+    bool result = inputStream->Next(&bufferPointer, &bufferLength);
+    if (!result) {
+      throw ParseError("bad read in nextBuffer");
+    }
+    bufferStart = static_cast<const char*>(bufferPointer);
+    bufferEnd = bufferStart + bufferLength;
+  }
+
+  signed char ByteRleDecoderImpl::readByte() {
+    if (bufferStart == bufferEnd) {
+      nextBuffer();
+    }
+    return *(bufferStart++);
+  }
+
+  void ByteRleDecoderImpl::readHeader() {
+    signed char ch = readByte();
+    if (ch < 0) {
+      remainingValues = static_cast<size_t>(-ch);
+      repeating = false;
+    } else {
+      remainingValues = static_cast<size_t>(ch) + MINIMUM_REPEAT;
+      repeating = true;
+      value = readByte();
+    }
+  }
+
+  ByteRleDecoderImpl::ByteRleDecoderImpl(std::unique_ptr<SeekableInputStream>
+                                         input) {
+    inputStream = std::move(input);
+    repeating = false;
+    remainingValues = 0;
+    value = 0;
+    bufferStart = 0;
+    bufferEnd = 0;
+  }
+
+  ByteRleDecoderImpl::~ByteRleDecoderImpl() {
+    // PASS
+  }
+
+  void ByteRleDecoderImpl::seek(PositionProvider& location) {
+    // move the input stream
+    inputStream->seek(location);
+    // force a re-read from the stream
+    bufferEnd = bufferStart;
+    // read a new header
+    readHeader();
+    // skip ahead the given number of records
+    skip(location.next());
+  }
+
+  void ByteRleDecoderImpl::skip(uint64_t numValues) {
+    while (numValues > 0) {
+      if (remainingValues == 0) {
+        readHeader();
+      }
+      size_t count = std::min(static_cast<size_t>(numValues), remainingValues);
+      remainingValues -= count;
+      numValues -= count;
+      // for literals we need to skip over count bytes, which may involve
+      // reading from the underlying stream
+      if (!repeating) {
+        size_t consumedBytes = count;
+        while (consumedBytes > 0) {
+          if (bufferStart == bufferEnd) {
+            nextBuffer();
+          }
+          size_t skipSize = std::min(static_cast<size_t>(consumedBytes),
+                                     static_cast<size_t>(bufferEnd -
+                                                         bufferStart));
+          bufferStart += skipSize;
+          consumedBytes -= skipSize;
+        }
+      }
+    }
+  }
+
+  void ByteRleDecoderImpl::next(char* data, uint64_t numValues,
+                                char* notNull) {
+    uint64_t position = 0;
+    // skip over null values
+    while (notNull && position < numValues && !notNull[position]) {
+      position += 1;
+    }
+    while (position < numValues) {
+      // if we are out of values, read more
+      if (remainingValues == 0) {
+        readHeader();
+      }
+      // how many do we read out of this block?
+      size_t count = std::min(static_cast<size_t>(numValues - position),
+                              remainingValues);
+      uint64_t consumed = 0;
+      if (repeating) {
+        if (notNull) {
+          for(uint64_t i=0; i < count; ++i) {
+            if (notNull[position + i]) {
+              data[position + i] = value;
+              consumed += 1;
+            }
+          }
+        } else {
+          memset(data + position, value, count);
+          consumed = count;
+        }
+      } else {
+        if (notNull) {
+          for(uint64_t i=0; i < count; ++i) {
+            if (notNull[position + i]) {
+              data[position + i] = readByte();
+              consumed += 1;
+            }
+          }
+        } else {
+          uint64_t i = 0;
+          while (i < count) {
+            if (bufferStart == bufferEnd) {
+              nextBuffer();
+            }
+            uint64_t copyBytes =
+              std::min(static_cast<uint64_t>(count - i),
+                       static_cast<uint64_t>(bufferEnd - bufferStart));
+            memcpy(data + position + i, bufferStart, copyBytes);
+            bufferStart += copyBytes;
+            i += copyBytes;
+          }
+          consumed = count;
+        }
+      }
+      remainingValues -= consumed;
+      position += count;
+      // skip over any null values
+      while (notNull && position < numValues && !notNull[position]) {
+        position += 1;
+      }
+    }
+  }
+
+  std::unique_ptr<ByteRleDecoder> createByteRleDecoder
+                                 (std::unique_ptr<SeekableInputStream> input) {
+    return std::unique_ptr<ByteRleDecoder>(new ByteRleDecoderImpl
+                                           (std::move(input)));
+  }
+
+  class BooleanRleDecoderImpl: public ByteRleDecoderImpl {
+  public:
+    BooleanRleDecoderImpl(std::unique_ptr<SeekableInputStream> input);
+
+    virtual ~BooleanRleDecoderImpl();
+
+    /**
+     * Seek to a particular spot.
+     */
+    virtual void seek(PositionProvider&);
+
+    /**
+     * Seek over a given number of values.
+     */
+    virtual void skip(uint64_t numValues);
+
+    /**
+     * Read a number of values into the batch.
+     */
+    virtual void next(char* data, uint64_t numValues, char* notNull);
+
+  protected:
+    size_t remainingBits;
+    char lastByte;
+  };
+
+  BooleanRleDecoderImpl::BooleanRleDecoderImpl
+                                (std::unique_ptr<SeekableInputStream> input
+                                 ): ByteRleDecoderImpl(std::move(input)) {
+    remainingBits = 0;
+    lastByte = 0;
+  }
+
+  BooleanRleDecoderImpl::~BooleanRleDecoderImpl() {
+    // PASS
+  }
+
+  void BooleanRleDecoderImpl::seek(PositionProvider& location) {
+    ByteRleDecoderImpl::seek(location);
+    uint64_t consumed = location.next();
+    if (consumed > 8) {
+      throw ParseError("bad position");
+    }
+    if (consumed != 0) {
+      remainingBits = 8 - consumed;
+      ByteRleDecoderImpl::next(&lastByte, 1, 0);
+    }
+  }
+
+  void BooleanRleDecoderImpl::skip(uint64_t numValues) {
+    if (numValues <= remainingBits) {
+      remainingBits -= numValues;
+    } else {
+      numValues -= remainingBits;
+      uint64_t bytesSkipped = numValues / 8;
+      ByteRleDecoderImpl::skip(bytesSkipped);
+      ByteRleDecoderImpl::next(&lastByte, 1, 0);
+      remainingBits = 8 - (numValues % 8);
+    }
+  }
+
+  void BooleanRleDecoderImpl::next(char* data, uint64_t numValues,
+                                   char* notNull) {
+    // next spot to fill in
+    uint64_t position = 0;
+
+    // use up any remaining bits
+    if (notNull) {
+      while(remainingBits > 0 && position < numValues) {
+        if (notNull[position]) {
+          remainingBits -= 1;
+          data[position] = (static_cast<unsigned char>(lastByte) >>
+                            remainingBits) & 0x1;
+        } else {
+          data[position] = 0;
+        }
+        position += 1;
+      }
+    } else {
+      while(remainingBits > 0 && position < numValues) {
+        remainingBits -= 1;
+        data[position++] = (static_cast<unsigned char>(lastByte) >>
+                            remainingBits) & 0x1;
+      }
+    }
+
+    // count the number of nonNulls remaining
+    uint64_t nonNulls = numValues - position;
+    if (notNull) {
+      for(uint64_t i=position; i < numValues; ++i) {
+        if (!notNull[i]) {
+          nonNulls -= 1;
+        }
+      }
+    }
+
+    // fill in the remaining values
+    if (nonNulls == 0) {
+      while (position < numValues) {
+        data[position++] = 0;
+      }
+    } else if (position < numValues) {
+      // read the new bytes into the array
+      uint64_t bytesRead = (nonNulls + 7) / 8;
+      ByteRleDecoderImpl::next(data + position, bytesRead, 0);
+      lastByte = data[position + bytesRead - 1];
+      remainingBits = bytesRead * 8 - nonNulls;
+      // expand the array backwards so that we don't clobber the data
+      uint64_t bitsLeft = bytesRead * 8 - remainingBits;
+      if (notNull) {
+        for(int64_t i=static_cast<int64_t>(numValues) - 1;
+            i >= static_cast<int64_t>(position); --i) {
+          if (notNull[i]) {
+            uint64_t shiftPosn = (-bitsLeft) % 8;
+            data[i] = (data[position + (bitsLeft - 1) / 8] >> shiftPosn) & 0x1;
+            bitsLeft -= 1;
+          } else {
+            data[i] = 0;
+          }
+        }
+      } else {
+        for(int64_t i=static_cast<int64_t>(numValues) - 1;
+            i >= static_cast<int64_t>(position); --i, --bitsLeft) {
+          uint64_t shiftPosn = (-bitsLeft) % 8;
+          data[i] = (data[position + (bitsLeft - 1) / 8] >> shiftPosn) & 0x1;
+        }
+      }
+    }
+  }
+
+  std::unique_ptr<ByteRleDecoder> createBooleanRleDecoder
+                                 (std::unique_ptr<SeekableInputStream> input) {
+    BooleanRleDecoderImpl* decoder = new BooleanRleDecoderImpl(std::move(input)) ;
+    return std::unique_ptr<ByteRleDecoder>(reinterpret_cast<ByteRleDecoder*>(decoder));
+  }
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/ByteRLE.hh
----------------------------------------------------------------------
diff --git a/c++/src/ByteRLE.hh b/c++/src/ByteRLE.hh
new file mode 100644
index 0000000..6762cb5
--- /dev/null
+++ b/c++/src/ByteRLE.hh
@@ -0,0 +1,71 @@
+/**
+ * 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 ORC_BYTE_RLE_HH
+#define ORC_BYTE_RLE_HH
+
+#include <memory>
+
+#include "Compression.hh"
+
+namespace orc {
+
+  class ByteRleDecoder {
+  public:
+    virtual ~ByteRleDecoder();
+
+    /**
+     * Seek to a particular spot.
+     */
+    virtual void seek(PositionProvider&) = 0;
+
+    /**
+     * Seek over a given number of values.
+     */
+    virtual void skip(uint64_t numValues) = 0;
+
+    /**
+     * Read a number of values into the batch.
+     * @param data the array to read into
+     * @param numValues the number of values to read
+     * @param notNull If the pointer is null, all values are read. If the
+     *    pointer is not null, positions that are false are skipped.
+     */
+    virtual void next(char* data, uint64_t numValues, char* notNull) = 0;
+  };
+
+  /**
+   * Create a byte RLE decoder.
+   * @param input the input stream to read from
+   */
+  std::unique_ptr<ByteRleDecoder> createByteRleDecoder
+                                 (std::unique_ptr<SeekableInputStream> input);
+
+  /**
+   * Create a boolean RLE decoder.
+   *
+   * Unlike the other RLE decoders, the boolean decoder sets the data to 0
+   * if the value is masked by notNull. This is required for the notNull stream
+   * processing to properly apply multiple masks from nested types.
+   * @param input the input stream to read from
+   */
+  std::unique_ptr<ByteRleDecoder> createBooleanRleDecoder
+                                 (std::unique_ptr<SeekableInputStream> input);
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/C09Adapter.cc
----------------------------------------------------------------------
diff --git a/c++/src/C09Adapter.cc b/c++/src/C09Adapter.cc
new file mode 100644
index 0000000..8afc752
--- /dev/null
+++ b/c++/src/C09Adapter.cc
@@ -0,0 +1,28 @@
+/**
+* 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 "orc/C09Adapter.hh"
+#include <sstream>
+
+int64_t std::stoll(std::string str) {
+  int64_t val = 0;
+  stringstream ss ;
+  ss << str ;
+  ss >> val ;
+  return val;
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/c++/src/CMakeLists.txt b/c++/src/CMakeLists.txt
index b1ed10a..b7b5260 100644
--- a/c++/src/CMakeLists.txt
+++ b/c++/src/CMakeLists.txt
@@ -82,8 +82,8 @@ CHECK_CXX_SOURCE_RUNS("
 )
 
 configure_file (
-  "orc/Adaptor.hh.in"
-  "${CMAKE_CURRENT_BINARY_DIR}/orc/Adaptor.hh"
+  "Adaptor.hh.in"
+  "${CMAKE_CURRENT_BINARY_DIR}/Adaptor.hh"
   )
 
 include_directories (
@@ -104,23 +104,23 @@ add_custom_command(OUTPUT orc_proto.pb.h orc_proto.pb.cc
 )
 
 add_library (orc STATIC
-  "${CMAKE_CURRENT_BINARY_DIR}/orc/Adaptor.hh"
+  "${CMAKE_CURRENT_BINARY_DIR}/Adaptor.hh"
   orc_proto.pb.h
   wrap/orc-proto-wrapper.cc
-  orc/ByteRLE.cc
-  orc/ColumnPrinter.cc
-  orc/ColumnReader.cc
-  orc/Compression.cc
-  orc/Exceptions.cc
-  orc/Int128.cc
-  orc/MemoryPool.cc
-  orc/OrcFile.cc
-  orc/Reader.cc
-  orc/RLEv1.cc
-  orc/RLEv2.cc
-  orc/RLE.cc
-  orc/TypeImpl.cc
-  orc/Vector.cc
+  ByteRLE.cc
+  ColumnPrinter.cc
+  ColumnReader.cc
+  Compression.cc
+  Exceptions.cc
+  Int128.cc
+  MemoryPool.cc
+  OrcFile.cc
+  Reader.cc
+  RLEv1.cc
+  RLEv2.cc
+  RLE.cc
+  TypeImpl.cc
+  Vector.cc
   )
 
 install(TARGETS orc DESTINATION lib)

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/ColumnPrinter.cc
----------------------------------------------------------------------
diff --git a/c++/src/ColumnPrinter.cc b/c++/src/ColumnPrinter.cc
new file mode 100644
index 0000000..190441c
--- /dev/null
+++ b/c++/src/ColumnPrinter.cc
@@ -0,0 +1,725 @@
+/**
+ * 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 "orc/ColumnPrinter.hh"
+#include "orc/orc-config.hh"
+
+#include "Adaptor.hh"
+
+#include <limits>
+#include <sstream>
+#include <stdexcept>
+#include <time.h>
+#include <typeinfo>
+
+#ifdef __clang__
+  #pragma clang diagnostic ignored "-Wformat-security"
+#endif
+
+namespace orc {
+
+  class BooleanColumnPrinter: public ColumnPrinter {
+  private:
+    const int64_t* data;
+  public:
+    BooleanColumnPrinter(std::string&, const Type&);
+    ~BooleanColumnPrinter() {}
+    void printRow(uint64_t rowId) override;
+    void reset(const ColumnVectorBatch& batch) override;
+  };
+
+  class LongColumnPrinter: public ColumnPrinter {
+  private:
+    const int64_t* data;
+  public:
+    LongColumnPrinter(std::string&, const Type&);
+    ~LongColumnPrinter() {}
+    void printRow(uint64_t rowId) override;
+    void reset(const ColumnVectorBatch& batch) override;
+  };
+
+  class DoubleColumnPrinter: public ColumnPrinter {
+  private:
+    const double* data;
+    const bool isFloat;
+
+  public:
+    DoubleColumnPrinter(std::string&, const Type&);
+    virtual ~DoubleColumnPrinter() {}
+    void printRow(uint64_t rowId) override;
+    void reset(const ColumnVectorBatch& batch) override;
+  };
+
+  class TimestampColumnPrinter: public ColumnPrinter {
+  private:
+    const int64_t* data;
+    time_t epoch;
+
+  public:
+    TimestampColumnPrinter(std::string&, const Type&);
+    ~TimestampColumnPrinter() {}
+    void printRow(uint64_t rowId) override;
+    void reset(const ColumnVectorBatch& batch) override;
+  };
+
+  class DateColumnPrinter: public ColumnPrinter {
+  private:
+    const int64_t* data;
+
+  public:
+    DateColumnPrinter(std::string&, const Type& type);
+    ~DateColumnPrinter() {}
+    void printRow(uint64_t rowId) override;
+    void reset(const ColumnVectorBatch& batch) override;
+  };
+
+  class Decimal64ColumnPrinter: public ColumnPrinter {
+  private:
+    const int64_t* data;
+    int32_t scale;
+  public:
+    Decimal64ColumnPrinter(std::string&, const Type& type);
+    ~Decimal64ColumnPrinter() {}
+    void printRow(uint64_t rowId) override;
+    void reset(const ColumnVectorBatch& batch) override;
+  };
+
+  class Decimal128ColumnPrinter: public ColumnPrinter {
+  private:
+    const Int128* data;
+    int32_t scale;
+  public:
+    Decimal128ColumnPrinter(std::string&, const Type& type);
+    ~Decimal128ColumnPrinter() {}
+    void printRow(uint64_t rowId) override;
+    void reset(const ColumnVectorBatch& batch) override;
+  };
+
+  class StringColumnPrinter: public ColumnPrinter {
+  private:
+    const char* const * start;
+    const int64_t* length;
+  public:
+    StringColumnPrinter(std::string&, const Type& type);
+    virtual ~StringColumnPrinter() {}
+    void printRow(uint64_t rowId) override;
+    void reset(const ColumnVectorBatch& batch) override;
+  };
+
+  class BinaryColumnPrinter: public ColumnPrinter {
+  private:
+    const char* const * start;
+    const int64_t* length;
+  public:
+    BinaryColumnPrinter(std::string&, const Type& type);
+    virtual ~BinaryColumnPrinter() {}
+    void printRow(uint64_t rowId) override;
+    void reset(const ColumnVectorBatch& batch) override;
+  };
+
+  class ListColumnPrinter: public ColumnPrinter {
+  private:
+    const int64_t* offsets;
+    std::unique_ptr<ColumnPrinter> elementPrinter;
+
+  public:
+    ListColumnPrinter(std::string&, const Type& type);
+    virtual ~ListColumnPrinter() {}
+    void printRow(uint64_t rowId) override;
+    void reset(const ColumnVectorBatch& batch) override;
+  };
+
+  class MapColumnPrinter: public ColumnPrinter {
+  private:
+    const int64_t* offsets;
+    std::unique_ptr<ColumnPrinter> keyPrinter;
+    std::unique_ptr<ColumnPrinter> elementPrinter;
+
+  public:
+    MapColumnPrinter(std::string&, const Type& type);
+    virtual ~MapColumnPrinter() {}
+    void printRow(uint64_t rowId) override;
+    void reset(const ColumnVectorBatch& batch) override;
+  };
+
+  class UnionColumnPrinter: public ColumnPrinter {
+  private:
+    const unsigned char *tags;
+    const uint64_t* offsets;
+    std::vector<ColumnPrinter*> fieldPrinter;
+
+  public:
+    UnionColumnPrinter(std::string&, const Type& type);
+    virtual ~UnionColumnPrinter();
+    void printRow(uint64_t rowId) override;
+    void reset(const ColumnVectorBatch& batch) override;
+  };
+
+  class StructColumnPrinter: public ColumnPrinter {
+  private:
+    std::vector<ColumnPrinter*> fieldPrinter;
+  public:
+    StructColumnPrinter(std::string&, const Type& type);
+    virtual ~StructColumnPrinter();
+    void printRow(uint64_t rowId) override;
+    void reset(const ColumnVectorBatch& batch) override;
+  };
+
+  void writeChar(std::string& file, char ch) {
+    file += ch;
+  }
+
+  void writeString(std::string& file, const char *ptr) {
+    size_t len = strlen(ptr);
+    file.append(ptr, len);
+  }
+
+  ColumnPrinter::ColumnPrinter(std::string& _buffer, const Type& _type
+                               ): buffer(_buffer),
+                                  type(_type) {
+    notNull = nullptr;
+    hasNulls = false;
+  }
+
+  ColumnPrinter::~ColumnPrinter() {
+    // PASS
+  }
+
+  void ColumnPrinter::reset(const ColumnVectorBatch& batch) {
+    hasNulls = batch.hasNulls;
+    if (hasNulls) {
+      notNull = batch.notNull.data();
+    } else {
+      notNull = nullptr ;
+    }
+  }
+
+  std::unique_ptr<ColumnPrinter> createColumnPrinter(std::string& buffer,
+                                                     const Type& type) {
+    ColumnPrinter *result;
+    switch(static_cast<int64_t>(type.getKind())) {
+    case BOOLEAN:
+      result = new BooleanColumnPrinter(buffer, type);
+      break;
+
+    case BYTE:
+    case SHORT:
+    case INT:
+    case LONG:
+      result = new LongColumnPrinter(buffer, type);
+      break;
+
+    case FLOAT:
+    case DOUBLE:
+      result = new DoubleColumnPrinter(buffer, type);
+      break;
+
+    case STRING:
+    case VARCHAR :
+    case CHAR:
+      result = new StringColumnPrinter(buffer, type);
+      break;
+
+    case BINARY:
+      result = new BinaryColumnPrinter(buffer, type);
+      break;
+
+    case TIMESTAMP:
+      result = new TimestampColumnPrinter(buffer, type);
+      break;
+
+    case LIST:
+      result = new ListColumnPrinter(buffer, type);
+      break;
+
+    case MAP:
+      result = new MapColumnPrinter(buffer, type);
+      break;
+
+    case STRUCT:
+      result = new StructColumnPrinter(buffer, type);
+      break;
+
+    case DECIMAL:
+      if (type.getPrecision() == 0 || type.getPrecision() > 18) {
+        result = new Decimal128ColumnPrinter(buffer, type);
+      } else {
+        result = new Decimal64ColumnPrinter(buffer, type);
+      }
+      break;
+
+    case DATE:
+      result = new DateColumnPrinter(buffer, type);
+      break;
+
+    case UNION:
+      result = new UnionColumnPrinter(buffer, type);
+      break;
+
+    default:
+      throw std::logic_error("unknown batch type");
+    }
+    return std::unique_ptr<ColumnPrinter>(result);
+  }
+
+  LongColumnPrinter::LongColumnPrinter(std::string& buffer,
+                                       const Type& type
+                                       ): ColumnPrinter(buffer, type) {
+    // pass
+  }
+
+  void LongColumnPrinter::reset(const  ColumnVectorBatch& batch) {
+    ColumnPrinter::reset(batch);
+    data = dynamic_cast<const LongVectorBatch&>(batch).data.data();
+  }
+
+  void LongColumnPrinter::printRow(uint64_t rowId) {
+    if (hasNulls && !notNull[rowId]) {
+      writeString(buffer, "null");
+    } else {
+      char numBuffer[64];
+      snprintf(numBuffer, sizeof(numBuffer), "%" INT64_FORMAT_STRING "d",
+               static_cast<int64_t >(data[rowId]));
+      writeString(buffer, numBuffer);
+    }
+  }
+
+  DoubleColumnPrinter::DoubleColumnPrinter(std::string& buffer,
+                                           const Type& type
+                                           ): ColumnPrinter(buffer, type),
+                                              isFloat(type.getKind() == FLOAT){
+    // PASS
+  }
+
+  void DoubleColumnPrinter::reset(const  ColumnVectorBatch& batch) {
+    ColumnPrinter::reset(batch);
+    data = dynamic_cast<const DoubleVectorBatch&>(batch).data.data();
+  }
+
+  void DoubleColumnPrinter::printRow(uint64_t rowId) {
+    if (hasNulls && !notNull[rowId]) {
+      writeString(buffer, "null");
+    } else {
+      char numBuffer[64];
+      snprintf(numBuffer, sizeof(numBuffer), isFloat ? "%.7g" : "%.14g",
+               data[rowId]);
+      writeString(buffer, numBuffer);
+    }
+  }
+
+  Decimal64ColumnPrinter::Decimal64ColumnPrinter(std::string& buffer,
+                                                 const  Type& type
+                                                 ): ColumnPrinter(buffer,
+                                                                  type) {
+    // PASS
+  }
+
+  void Decimal64ColumnPrinter::reset(const  ColumnVectorBatch& batch) {
+    ColumnPrinter::reset(batch);
+    data = dynamic_cast<const Decimal64VectorBatch&>(batch).values.data();
+    scale = dynamic_cast<const Decimal64VectorBatch&>(batch).scale;
+  }
+
+  std::string toDecimalString(int64_t value, int32_t scale) {
+    std::stringstream buffer;
+    if (scale == 0) {
+      buffer << value;
+      return buffer.str();
+    }
+    std::string sign = "";
+    if (value < 0) {
+      sign = "-";
+      value = -value;
+    }
+    buffer << value;
+    std::string str = buffer.str();
+    int32_t len = static_cast<int32_t>(str.length());
+    if (len > scale) {
+      return sign + str.substr(0, static_cast<size_t>(len - scale)) + "." +
+        str.substr(static_cast<size_t>(len - scale),
+                   static_cast<size_t>(scale));
+    } else if (len == scale) {
+      return sign + "0." + str;
+    } else {
+      std::string result = sign + "0.";
+      for(int32_t i=0; i < scale - len; ++i) {
+        result += "0";
+      }
+      return result + str;
+    }
+  }
+
+  void Decimal64ColumnPrinter::printRow(uint64_t rowId) {
+    if (hasNulls && !notNull[rowId]) {
+      writeString(buffer, "null");
+    } else {
+      writeString(buffer, toDecimalString(data[rowId], scale).c_str());
+    }
+  }
+
+  Decimal128ColumnPrinter::Decimal128ColumnPrinter(std::string& buffer,
+                                                   const Type& type
+                                                   ): ColumnPrinter(buffer,
+                                                                    type) {
+     // PASS
+   }
+
+   void Decimal128ColumnPrinter::reset(const  ColumnVectorBatch& batch) {
+     ColumnPrinter::reset(batch);
+     data = dynamic_cast<const Decimal128VectorBatch&>(batch).values.data();
+     scale =dynamic_cast<const Decimal128VectorBatch&>(batch).scale;
+   }
+
+   void Decimal128ColumnPrinter::printRow(uint64_t rowId) {
+     if (hasNulls && !notNull[rowId]) {
+       writeString(buffer, "null");
+     } else {
+       writeString(buffer, data[rowId].toDecimalString(scale).c_str());
+     }
+   }
+
+  StringColumnPrinter::StringColumnPrinter(std::string& buffer,
+                                           const Type& type
+                                           ): ColumnPrinter(buffer, type) {
+    // PASS
+  }
+
+  void StringColumnPrinter::reset(const ColumnVectorBatch& batch) {
+    ColumnPrinter::reset(batch);
+    start = dynamic_cast<const StringVectorBatch&>(batch).data.data();
+    length = dynamic_cast<const StringVectorBatch&>(batch).length.data();
+  }
+
+  void StringColumnPrinter::printRow(uint64_t rowId) {
+    if (hasNulls && !notNull[rowId]) {
+      writeString(buffer, "null");
+    } else {
+      writeChar(buffer, '"');
+      for(int64_t i=0; i < length[rowId]; ++i) {
+        char ch = static_cast<char>(start[rowId][i]);
+        switch (ch) {
+        case '\\':
+          writeString(buffer, "\\\\");
+          break;
+        case '\b':
+          writeString(buffer, "\\b");
+          break;
+        case '\f':
+          writeString(buffer, "\\f");
+          break;
+        case '\n':
+          writeString(buffer, "\\n");
+          break;
+        case '\r':
+          writeString(buffer, "\\r");
+          break;
+        case '\t':
+          writeString(buffer, "\\t");
+          break;
+        case '"':
+          writeString(buffer, "\\\"");
+          break;
+        default:
+          writeChar(buffer, ch);
+          break;
+        }
+      }
+      writeChar(buffer, '"');
+    }
+  }
+
+  ListColumnPrinter::ListColumnPrinter(std::string& buffer,
+                                       const Type& type
+                                       ): ColumnPrinter(buffer, type) {
+    elementPrinter = createColumnPrinter(buffer, type.getSubtype(0));
+  }
+
+  void ListColumnPrinter::reset(const  ColumnVectorBatch& batch) {
+    ColumnPrinter::reset(batch);
+    offsets = dynamic_cast<const ListVectorBatch&>(batch).offsets.data();
+    elementPrinter->reset(*dynamic_cast<const ListVectorBatch&>(batch).
+                          elements);
+  }
+
+  void ListColumnPrinter::printRow(uint64_t rowId) {
+    if (hasNulls && !notNull[rowId]) {
+      writeString(buffer, "null");
+    } else {
+      writeChar(buffer, '[');
+      for(int64_t i=offsets[rowId]; i < offsets[rowId+1]; ++i) {
+        if (i != offsets[rowId]) {
+          writeString(buffer, ", ");
+        }
+        elementPrinter->printRow(static_cast<uint64_t>(i));
+      }
+      writeChar(buffer, ']');
+    }
+  }
+
+  MapColumnPrinter::MapColumnPrinter(std::string& buffer,
+                                     const Type& type
+                                     ): ColumnPrinter(buffer, type) {
+    keyPrinter = createColumnPrinter(buffer, type.getSubtype(0));
+    elementPrinter = createColumnPrinter(buffer, type.getSubtype(1));
+  }
+
+  void MapColumnPrinter::reset(const  ColumnVectorBatch& batch) {
+    ColumnPrinter::reset(batch);
+    const MapVectorBatch& myBatch = dynamic_cast<const MapVectorBatch&>(batch);
+    offsets = myBatch.offsets.data();
+    keyPrinter->reset(*myBatch.keys);
+    elementPrinter->reset(*myBatch.elements);
+  }
+
+  void MapColumnPrinter::printRow(uint64_t rowId) {
+    if (hasNulls && !notNull[rowId]) {
+      writeString(buffer, "null");
+    } else {
+      writeChar(buffer, '[');
+      for(int64_t i=offsets[rowId]; i < offsets[rowId+1]; ++i) {
+        if (i != offsets[rowId]) {
+          writeString(buffer, ", ");
+        }
+        writeString(buffer, "{\"key\": ");
+        keyPrinter->printRow(static_cast<uint64_t>(i));
+        writeString(buffer, ", \"value\": ");
+        elementPrinter->printRow(static_cast<uint64_t>(i));
+        writeChar(buffer, '}');
+      }
+      writeChar(buffer, ']');
+    }
+  }
+
+  UnionColumnPrinter::UnionColumnPrinter(std::string& buffer,
+                                           const Type& type
+                                           ): ColumnPrinter(buffer, type) {
+    for(unsigned int i=0; i < type.getSubtypeCount(); ++i) {
+      fieldPrinter.push_back(createColumnPrinter(buffer, type.getSubtype(i))
+                             .release());
+    }
+  }
+
+  UnionColumnPrinter::~UnionColumnPrinter() {
+    for (size_t i = 0; i < fieldPrinter.size(); i++) {
+      delete fieldPrinter[i];
+    }
+  }
+
+  void UnionColumnPrinter::reset(const ColumnVectorBatch& batch) {
+    ColumnPrinter::reset(batch);
+    const UnionVectorBatch& unionBatch =
+      dynamic_cast<const UnionVectorBatch&>(batch);
+    tags = unionBatch.tags.data();
+    offsets = unionBatch.offsets.data();
+    for(size_t i=0; i < fieldPrinter.size(); ++i) {
+      fieldPrinter[i]->reset(*(unionBatch.children[i]));
+    }
+  }
+
+  void UnionColumnPrinter::printRow(uint64_t rowId) {
+    if (hasNulls && !notNull[rowId]) {
+      writeString(buffer, "null");
+    } else {
+      writeString(buffer, "{\"tag\": ");
+      char numBuffer[64];
+      snprintf(numBuffer, sizeof(numBuffer), "%" INT64_FORMAT_STRING "d",
+               static_cast<int64_t>(tags[rowId]));
+      writeString(buffer, numBuffer);
+      writeString(buffer, ", \"value\": ");
+      fieldPrinter[tags[rowId]]->printRow(offsets[rowId]);
+      writeChar(buffer, '}');
+    }
+  }
+
+  StructColumnPrinter::StructColumnPrinter(std::string& buffer,
+                                           const Type& type
+                                           ): ColumnPrinter(buffer, type) {
+    for(unsigned int i=0; i < type.getSubtypeCount(); ++i) {
+      fieldPrinter.push_back(createColumnPrinter(buffer, type.getSubtype(i))
+                             .release());
+    }
+  }
+
+  StructColumnPrinter::~StructColumnPrinter() {
+    for (size_t i = 0; i < fieldPrinter.size(); i++) {
+      delete fieldPrinter[i];
+    }
+  }
+
+  void StructColumnPrinter::reset(const ColumnVectorBatch& batch) {
+    ColumnPrinter::reset(batch);
+    const StructVectorBatch& structBatch =
+      dynamic_cast<const StructVectorBatch&>(batch);
+    for(size_t i=0; i < fieldPrinter.size(); ++i) {
+      fieldPrinter[i]->reset(*(structBatch.fields[i]));
+    }
+  }
+
+  void StructColumnPrinter::printRow(uint64_t rowId) {
+    if (hasNulls && !notNull[rowId]) {
+      writeString(buffer, "null");
+    } else {
+      writeChar(buffer, '{');
+      for(unsigned int i=0; i < fieldPrinter.size(); ++i) {
+        if (i != 0) {
+          writeString(buffer, ", ");
+        }
+        writeChar(buffer, '"');
+        writeString(buffer, type.getFieldName(i).c_str());
+        writeString(buffer, "\": ");
+        fieldPrinter[i]->printRow(rowId);
+      }
+      writeChar(buffer, '}');
+    }
+  }
+
+  DateColumnPrinter::DateColumnPrinter(std::string& buffer,
+                                       const Type& type
+                                       ): ColumnPrinter(buffer, type) {
+    // PASS
+  }
+
+  void DateColumnPrinter::printRow(uint64_t rowId) {
+    if (hasNulls && !notNull[rowId]) {
+      writeString(buffer, "null");
+    } else {
+      const time_t timeValue = data[rowId] * 24 * 60 * 60;
+      struct tm tmValue;
+      gmtime_r(&timeValue, &tmValue);
+      char timeBuffer[11];
+      strftime(timeBuffer, sizeof(timeBuffer), "%Y-%m-%d", &tmValue);
+      writeChar(buffer, '"');
+      writeString(buffer, timeBuffer);
+      writeChar(buffer, '"');
+    }
+  }
+
+  void DateColumnPrinter::reset(const ColumnVectorBatch& batch) {
+    ColumnPrinter::reset(batch);
+    data = dynamic_cast<const LongVectorBatch&>(batch).data.data();
+  }
+
+  BooleanColumnPrinter::BooleanColumnPrinter(std::string& buffer,
+                                             const Type& type
+                                             ): ColumnPrinter(buffer, type) {
+    // PASS
+  }
+
+  void BooleanColumnPrinter::printRow(uint64_t rowId) {
+    if (hasNulls && !notNull[rowId]) {
+      writeString(buffer, "null");
+    } else {
+      writeString(buffer, (data[rowId] ? "true" : "false"));
+    }
+  }
+
+  void BooleanColumnPrinter::reset(const ColumnVectorBatch& batch) {
+    ColumnPrinter::reset(batch);
+    data = dynamic_cast<const LongVectorBatch&>(batch).data.data();
+  }
+
+  BinaryColumnPrinter::BinaryColumnPrinter(std::string& buffer,
+                                           const Type& type
+                                           ): ColumnPrinter(buffer, type) {
+    // PASS
+  }
+
+  void BinaryColumnPrinter::printRow(uint64_t rowId) {
+    if (hasNulls && !notNull[rowId]) {
+      writeString(buffer, "null");
+    } else {
+      writeChar(buffer, '[');
+      for(int64_t i=0; i < length[rowId]; ++i) {
+        if (i != 0) {
+          writeString(buffer, ", ");
+        }
+        char numBuffer[64];
+        snprintf(numBuffer, sizeof(numBuffer), "%d",
+                 (static_cast<const int>(start[rowId][i]) & 0xff));
+        writeString(buffer, numBuffer);
+      }
+      writeChar(buffer, ']');
+    }
+  }
+
+  void BinaryColumnPrinter::reset(const ColumnVectorBatch& batch) {
+    ColumnPrinter::reset(batch);
+    start = dynamic_cast<const StringVectorBatch&>(batch).data.data();
+    length = dynamic_cast<const StringVectorBatch&>(batch).length.data();
+  }
+
+  TimestampColumnPrinter::TimestampColumnPrinter(std::string& buffer,
+                                                 const Type& type
+                                                 ): ColumnPrinter(buffer,
+                                                                  type) {
+    struct tm epochTm;
+    epochTm.tm_sec = 0;
+    epochTm.tm_min = 0;
+    epochTm.tm_hour = 0;
+    epochTm.tm_mday = 1;
+    epochTm.tm_mon = 0;
+    epochTm.tm_year = 70;
+    epochTm.tm_isdst = 0;
+    epoch = mktime(&epochTm);
+  }
+
+  void TimestampColumnPrinter::printRow(uint64_t rowId) {
+    const int64_t NANOS_PER_SECOND = 1000000000;
+    const int64_t NANO_DIGITS = 9;
+    if (hasNulls && !notNull[rowId]) {
+      writeString(buffer, "null");
+    } else {
+      int64_t nanos = data[rowId] % NANOS_PER_SECOND;
+      time_t seconds =
+        static_cast<time_t>(data[rowId] / NANOS_PER_SECOND) + epoch;
+      // make sure the nanos are positive
+      if (nanos < 0) {
+        seconds -= 1;
+        nanos = -nanos;
+      }
+      struct tm tmValue;
+      localtime_r(&seconds, &tmValue);
+      char timeBuffer[20];
+      strftime(timeBuffer, sizeof(timeBuffer), "%Y-%m-%d %H:%M:%S", &tmValue);
+      writeChar(buffer, '"');
+      writeString(buffer, timeBuffer);
+      writeChar(buffer, '.');
+      // remove trailing zeros off the back of the nanos value.
+      int64_t zeroDigits = 0;
+      if (nanos == 0) {
+        zeroDigits = 8;
+      } else {
+        while (nanos % 10 == 0) {
+          nanos /= 10;
+          zeroDigits += 1;
+        }
+      }
+      char numBuffer[64];
+      snprintf(numBuffer, sizeof(numBuffer),
+               "%0*" INT64_FORMAT_STRING "d\"",
+               static_cast<int>(NANO_DIGITS - zeroDigits),
+               static_cast<int64_t >(nanos));
+      writeString(buffer, numBuffer);
+    }
+  }
+
+  void TimestampColumnPrinter::reset(const ColumnVectorBatch& batch) {
+    ColumnPrinter::reset(batch);
+    data = dynamic_cast<const LongVectorBatch&>(batch).data.data();
+  }
+}


[09/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/test/orc/TestColumnPrinter.cc
----------------------------------------------------------------------
diff --git a/c++/test/orc/TestColumnPrinter.cc b/c++/test/orc/TestColumnPrinter.cc
deleted file mode 100644
index 652711a..0000000
--- a/c++/test/orc/TestColumnPrinter.cc
+++ /dev/null
@@ -1,588 +0,0 @@
-/**
- * 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 "orc/ColumnPrinter.hh"
-#include "orc/Exceptions.hh"
-
-#include "wrap/gtest-wrapper.h"
-
-namespace orc {
-
-  TEST(TestColumnPrinter, BooleanColumnPrinter) {
-    std::string line;
-    std::unique_ptr<Type> type = createPrimitiveType(BOOLEAN);
-    std::unique_ptr<ColumnPrinter> printer =
-      createColumnPrinter(line, *type);
-    LongVectorBatch batch(1024, *getDefaultPool());
-    const char *expected[] = {"true", "false", "true"};
-    batch.numElements = 3;
-    batch.hasNulls = false;
-    batch.data[0] = 1;
-    batch.data[1] = 0;
-    batch.data[2] = 1;
-    printer->reset(batch);
-    for(uint64_t i=0; i < 3; ++i) {
-      line.clear();
-      printer->printRow(i);
-      EXPECT_EQ(expected[i], line);
-    }
-    const char *expected2[] = {"null", "null", "true", "false"};
-    batch.numElements = 4;
-    batch.data[3] = false;
-    batch.hasNulls = true;
-    batch.notNull[0] = false;
-    batch.notNull[1] = false;
-    batch.notNull[2] = true;
-    batch.notNull[3] = true;
-    printer->reset(batch);
-    for(uint64_t i=0; i < 4; ++i) {
-      line.clear();
-      printer->printRow(i);
-      EXPECT_EQ(expected2[i], line);
-    }
-  }
-
-  TEST(TestColumnPrinter, LongColumnPrinter) {
-    std::string line;
-    std::unique_ptr<Type> type = createPrimitiveType(LONG);
-    std::unique_ptr<ColumnPrinter> printer = createColumnPrinter(line, *type);
-    LongVectorBatch batch(1024, *getDefaultPool());
-    batch.numElements = 2;
-    batch.hasNulls = false;
-    batch.data[0] = 9223372036854775807LL;
-    batch.data[1] = -9223372036854775807LL;
-    printer->reset(batch);
-    const char *expected[] = {"9223372036854775807", "-9223372036854775807"};
-    for(uint64_t i=0; i < 2; ++i) {
-      line.clear();
-      printer->printRow(i);
-      EXPECT_EQ(expected[i], line);
-    }
-    batch.numElements = 3;
-    batch.hasNulls = true;
-    batch.data[0] = 127;
-    batch.data[1] = -127;
-    batch.data[2] = 123;
-    batch.notNull[0] = true;
-    batch.notNull[1] = true;
-    batch.notNull[2] = false;
-    printer->reset(batch);
-    const char *expected2[] = {"127", "-127", "null"};
-    for(uint64_t i=0; i < 3; ++i) {
-      line.clear();
-      printer->printRow(i);
-      EXPECT_EQ(expected2[i], line);
-    }
-  }
-
-  TEST(TestColumnPrinter, DoubleColumnPrinter) {
-    std::string line;
-    std::unique_ptr<Type> type = createPrimitiveType(DOUBLE);
-    std::unique_ptr<ColumnPrinter> printer =
-      createColumnPrinter(line, *type);
-    DoubleVectorBatch batch(1024, *getDefaultPool());
-    batch.numElements = 2;
-    batch.hasNulls = false;
-    batch.data[0] = 1234.5;
-    batch.data[1] = -1234.5;
-    printer->reset(batch);
-    const char *expected[] = {"1234.5", "-1234.5"};
-    for(uint64_t i=0; i < 2; ++i) {
-      line.clear();
-      printer->printRow(i);
-      EXPECT_EQ(expected[i], line);
-    }
-    batch.numElements = 3;
-    batch.hasNulls = true;
-    batch.data[0] = 9999.125;
-    batch.data[1] = -9999.125;
-    batch.data[2] = 100000;
-    batch.notNull[0] = true;
-    batch.notNull[1] = true;
-    batch.notNull[2] = false;
-    printer->reset(batch);
-    const char *expected2[] = {"9999.125", "-9999.125", "null"};
-    for(uint64_t i=0; i < 3; ++i) {
-      line.clear();
-      printer->printRow(i);
-      EXPECT_EQ(expected2[i], line);
-    }
-  }
-
-  TEST(TestColumnPrinter, TimestampColumnPrinter) {
-    std::string line;
-    std::unique_ptr<Type> type = createPrimitiveType(TIMESTAMP);
-    std::unique_ptr<ColumnPrinter> printer = createColumnPrinter(line, *type);
-    LongVectorBatch batch(1024, *getDefaultPool());
-    batch.numElements = 12;
-    batch.hasNulls = false;
-    batch.data[0]  = 1420070400000000000;
-    batch.data[1]  =  963270000000000000;
-    batch.data[2]  = 1426168859000000000;
-    batch.data[3]  = 1426168859000000001;
-    batch.data[4]  = 1426168859000000010;
-    batch.data[5]  = 1426168859000000100;
-    batch.data[6]  = 1426168859000001000;
-    batch.data[7]  = 1426168859000010000;
-    batch.data[8]  = 1426168859000100000;
-    batch.data[9]  = 1426168859001000000;
-    batch.data[10] = 1426168859010000000;
-    batch.data[11] = 1426168859100000000;
-    const char *expected[] = {"\"2015-01-01 00:00:00.0\"",
-                              "\"2000-07-11 00:00:00.0\"",
-                              "\"2015-03-12 15:00:59.0\"",
-                              "\"2015-03-12 15:00:59.000000001\"",
-                              "\"2015-03-12 15:00:59.00000001\"",
-                              "\"2015-03-12 15:00:59.0000001\"",
-                              "\"2015-03-12 15:00:59.000001\"",
-                              "\"2015-03-12 15:00:59.00001\"",
-                              "\"2015-03-12 15:00:59.0001\"",
-                              "\"2015-03-12 15:00:59.001\"",
-                              "\"2015-03-12 15:00:59.01\"",
-                              "\"2015-03-12 15:00:59.1\""};
-    printer->reset(batch);
-    for(uint64_t i=0; i < batch.numElements; ++i) {
-      line.clear();
-      printer->printRow(i);
-      EXPECT_EQ(expected[i], line) << "for i = " << i;
-    }
-    batch.hasNulls = true;
-    for(size_t i = 0; i < batch.numElements; ++i) {
-      batch.notNull[i] = i % 2;
-    }
-    printer->reset(batch);
-    for(uint64_t i=0; i < batch.numElements; ++i) {
-      line.clear();
-      printer->printRow(i);
-      if (i % 2) {
-        EXPECT_EQ(expected[i], line) << "for i = " << i;
-      } else {
-        EXPECT_EQ("null", line) << "for i = " << i;
-      }
-    }
-  }
-
-  TEST(TestColumnPrinter, DateColumnPrinter) {
-    std::string line;
-    std::unique_ptr<Type> type = createPrimitiveType(DATE);
-    std::unique_ptr<ColumnPrinter> printer =
-      createColumnPrinter(line, *type);
-    LongVectorBatch batch(1024, *getDefaultPool());
-    batch.numElements = 10;
-    batch.hasNulls = false;
-    batch.data[0]  = 0;
-    batch.data[1]  = 11738;
-    batch.data[2]  = -165;
-    batch.data[3]  = -33165;
-    batch.data[4]  = 10489;
-    batch.data[5]  = -5171;
-    batch.data[6]  = 11016;
-    batch.data[7]  = 5763;
-    batch.data[8]  = 16729;
-    batch.data[9]  = 12275;
-    const char *expected[] = {"\"1970-01-01\"",
-                              "\"2002-02-20\"",
-                              "\"1969-07-20\"",
-                              "\"1879-03-14\"",
-                              "\"1998-09-20\"",
-                              "\"1955-11-05\"",
-                              "\"2000-02-29\"",
-                              "\"1985-10-12\"",
-                              "\"2015-10-21\"",
-                              "\"2003-08-11\""};
-    printer->reset(batch);
-    for(uint64_t i=0; i < batch.numElements; ++i) {
-      line.clear();
-      printer->printRow(i);
-      EXPECT_EQ(expected[i], line) << "for i = " << i;
-    }
-    batch.hasNulls = true;
-    for(size_t i = 0; i < batch.numElements; ++i) {
-      batch.notNull[i] = i % 2;
-    }
-    printer->reset(batch);
-    for(uint64_t i=0; i < batch.numElements; ++i) {
-      line.clear();
-      printer->printRow(i);
-      if (i % 2) {
-        EXPECT_EQ(expected[i], line) << "for i = " << i;
-      } else {
-        EXPECT_EQ("null", line) << "for i = " << i;
-      }
-    }
-  }
-
-  TEST(TestColumnPrinter, Decimal64ColumnPrinter) {
-    std::string line;
-    std::unique_ptr<Type> type = createDecimalType(16, 5);
-    std::unique_ptr<ColumnPrinter> printer =
-      createColumnPrinter(line, *type);
-    Decimal64VectorBatch batch(1024, *getDefaultPool());
-    batch.numElements = 10;
-    batch.hasNulls = false;
-    batch.scale = 5;
-    batch.values[0] =  0;
-    batch.values[1] =  1;
-    batch.values[2] = -10;
-    batch.values[3] =  100;
-    batch.values[4] =  1000;
-    batch.values[5] =  10000;
-    batch.values[6] =  100000;
-    batch.values[7] =  1000000;
-    batch.values[8] = -10000000;
-    batch.values[9] =  100000000;
-    const char *expected[] = { "0.00000",
-                               "0.00001",
-                               "-0.00010",
-                               "0.00100",
-                               "0.01000",
-                               "0.10000",
-                               "1.00000",
-                               "10.00000",
-                               "-100.00000",
-                               "1000.00000"};
-    printer->reset(batch);
-    for(uint64_t i=0; i < batch.numElements; ++i) {
-      line.clear();
-      printer->printRow(i);
-      EXPECT_EQ(expected[i], line) << "for i = " << i;
-    }
-    batch.hasNulls = true;
-    for(size_t i = 0; i < batch.numElements; ++i) {
-      batch.notNull[i] = i % 2;
-    }
-    printer->reset(batch);
-    for(uint64_t i=0; i < batch.numElements; ++i) {
-      line.clear();
-      printer->printRow(i);
-      if (i % 2) {
-        EXPECT_EQ(expected[i], line) << "for i = " << i;
-      } else {
-        EXPECT_EQ("null", line) << "for i = " << i;
-      }
-    }
-  }
-
-  TEST(TestColumnPrinter, Decimal128ColumnPrinter) {
-    std::string line;
-    std::unique_ptr<Type> type = createDecimalType(30, 5);
-    std::unique_ptr<ColumnPrinter> printer =
-      createColumnPrinter(line, *type);
-    Decimal128VectorBatch batch(1024, *getDefaultPool());
-    batch.numElements = 10;
-    batch.hasNulls = false;
-    batch.scale = 5;
-    batch.values[0] =  0;
-    batch.values[1] =  1;
-    batch.values[2] = -10;
-    batch.values[3] =  100;
-    batch.values[4] =  1000;
-    batch.values[5] =  10000;
-    batch.values[6] =  100000;
-    batch.values[7] =  1000000;
-    batch.values[8] = -10000000;
-    batch.values[9] =  100000000;
-    const char *expected[] = { "0.00000",
-                               "0.00001",
-                               "-0.00010",
-                               "0.00100",
-                               "0.01000",
-                               "0.10000",
-                               "1.00000",
-                               "10.00000",
-                               "-100.00000",
-                               "1000.00000"};
-    printer->reset(batch);
-    for(uint64_t i=0; i < batch.numElements; ++i) {
-      line.clear();
-      printer->printRow(i);
-      EXPECT_EQ(expected[i], line) << "for i = " << i;
-    }
-    batch.hasNulls = true;
-    for(size_t i = 0; i < batch.numElements; ++i) {
-      batch.notNull[i] = i % 2;
-    }
-    printer->reset(batch);
-    for(uint64_t i=0; i < batch.numElements; ++i) {
-      line.clear();
-      printer->printRow(i);
-      if (i % 2) {
-        EXPECT_EQ(expected[i], line) << "for i = " << i;
-      } else {
-        EXPECT_EQ("null", line) << "for i = " << i;
-      }
-    }
-  }
-
-  TEST(TestColumnPrinter, StringColumnPrinter) {
-    std::string line;
-    std::unique_ptr<Type> type = createPrimitiveType(STRING);
-    std::unique_ptr<ColumnPrinter> printer =
-      createColumnPrinter(line, *type);
-    StringVectorBatch batch(1024, *getDefaultPool());
-    const char *blob= "thisisatest\b\f\n\r\t\\\"'";
-    batch.numElements = 5;
-    batch.hasNulls = false;
-    batch.data[0] = const_cast<char *>(blob);
-    batch.length[0] = 4;
-    batch.length[1] = 2;
-    batch.length[2] = 1;
-    batch.length[3] = 4;
-    batch.length[4] = 8;
-    for(size_t i=1; i < 5; ++i) {
-      batch.data[i] = batch.data.data()[i - 1] +
-        static_cast<size_t>(batch.length[i - 1]);
-    }
-    const char *expected[] = {"\"this\"", "\"is\"", "\"a\"", "\"test\"",
-                              "\"\\b\\f\\n\\r\\t\\\\\\\"'\""};
-    printer->reset(batch);
-    for(uint64_t i=0; i < batch.numElements; ++i) {
-      line.clear();
-      printer->printRow(i);
-      EXPECT_EQ(expected[i], line) << "for i = " << i;
-    }
-    batch.hasNulls = true;
-    for(size_t i = 0; i < batch.numElements; ++i) {
-      batch.notNull[i] = i % 2;
-    }
-    printer->reset(batch);
-    for(uint64_t i=0; i < batch.numElements; ++i) {
-      line.clear();
-      printer->printRow(i);
-      if (i % 2) {
-        EXPECT_EQ(expected[i], line) << "for i = " << i;
-      } else {
-        EXPECT_EQ("null", line) << "for i = " << i;
-      }
-    }
-  }
-
-  TEST(TestColumnPrinter, BinaryColumnPrinter) {
-    std::string line;
-    std::unique_ptr<Type> type = createPrimitiveType(BINARY);
-    std::unique_ptr<ColumnPrinter> printer =
-      createColumnPrinter(line, *type);
-    StringVectorBatch batch(1024, *getDefaultPool());
-    char blob[45];
-    for(size_t i=0; i < sizeof(blob); ++i) {
-      blob[i] = static_cast<char>(i);
-    }
-    batch.numElements = 10;
-    batch.hasNulls = false;
-    batch.data[0] = blob;
-    batch.length[0] = 0;
-    for(size_t i=1; i < batch.numElements; ++i) {
-      batch.length[i] = static_cast<int64_t>(i);
-      batch.data[i] = batch.data.data()[i - 1] + batch.length[i - 1];
-    }
-    printer->reset(batch);
-    const char* expected[] = {"[]",
-                              "[0]",
-                              "[1, 2]",
-                              "[3, 4, 5]",
-                              "[6, 7, 8, 9]",
-                              "[10, 11, 12, 13, 14]",
-                              "[15, 16, 17, 18, 19, 20]",
-                              "[21, 22, 23, 24, 25, 26, 27]",
-                              "[28, 29, 30, 31, 32, 33, 34, 35]",
-                              "[36, 37, 38, 39, 40, 41, 42, 43, 44]"};
-    for(uint64_t i=0; i < batch.numElements; ++i) {
-      line.clear();
-      printer->printRow(i);
-      EXPECT_EQ(expected[i], line) << "for i = " << i;
-    }
-    batch.hasNulls = true;
-    for(size_t i = 0; i < batch.numElements; ++i) {
-      batch.notNull[i] = i % 2;
-    }
-    printer->reset(batch);
-    for(uint64_t i=0; i < batch.numElements; ++i) {
-      line.clear();
-      printer->printRow(i);
-      if (i % 2) {
-        EXPECT_EQ(expected[i], line) << "for i = " << i;
-      } else {
-        EXPECT_EQ("null", line) << "for i = " << i;
-      }
-    }
-  }
-
-  TEST(TestColumnPrinter, ListColumnPrinter) {
-    std::string line;
-    std::unique_ptr<Type> type = createListType(createPrimitiveType(LONG));
-    std::unique_ptr<ColumnPrinter> printer =
-      createColumnPrinter(line, *type);
-    ListVectorBatch batch(1024, *getDefaultPool());
-    LongVectorBatch* longBatch = new LongVectorBatch(1024, *getDefaultPool());
-    batch.elements = std::unique_ptr<ColumnVectorBatch>(longBatch);
-    batch.numElements = 10;
-    batch.hasNulls = false;
-    batch.offsets[0] = 0;
-    for(size_t i = 1; i <= batch.numElements; ++i) {
-      batch.offsets[i] = batch.offsets[i-1] + static_cast<int64_t>(i-1);
-    }
-    longBatch->numElements = 45;
-    longBatch->hasNulls = false;
-    for(size_t i = 0; i < longBatch->numElements; ++i) {
-      longBatch->data[i] = static_cast<int64_t>(i);
-    }
-    const char* expected[] = {"[]",
-                              "[0]",
-                              "[1, 2]",
-                              "[3, 4, 5]",
-                              "[6, 7, 8, 9]",
-                              "[10, 11, 12, 13, 14]",
-                              "[15, 16, 17, 18, 19, 20]",
-                              "[21, 22, 23, 24, 25, 26, 27]",
-                              "[28, 29, 30, 31, 32, 33, 34, 35]",
-                              "[36, 37, 38, 39, 40, 41, 42, 43, 44]"};
-    printer->reset(batch);
-    for(uint64_t i=0; i < batch.numElements; ++i) {
-      line.clear();
-      printer->printRow(i);
-      EXPECT_EQ(expected[i], line) << "for i = " << i;
-    }
-    batch.hasNulls = true;
-    for(size_t i = 0; i < batch.numElements; ++i) {
-      batch.notNull[i] = i % 2;
-    }
-    printer->reset(batch);
-    for(uint64_t i=0; i < batch.numElements; ++i) {
-      line.clear();
-      printer->printRow(i);
-      if (i % 2) {
-        EXPECT_EQ(expected[i], line) << "for i = " << i;
-      } else {
-        EXPECT_EQ("null", line) << "for i = " << i;
-      }
-    }
-  }
-
-  TEST(TestColumnPrinter, MapColumnPrinter) {
-    std::string line;
-    std::unique_ptr<Type> type = createMapType(createPrimitiveType(LONG),
-                                               createPrimitiveType(LONG));
-    std::unique_ptr<ColumnPrinter> printer =
-      createColumnPrinter(line, *type);
-    MapVectorBatch batch(1024, *getDefaultPool());
-    LongVectorBatch* keyBatch = new LongVectorBatch(1024, *getDefaultPool());
-    LongVectorBatch* valueBatch = new LongVectorBatch(1024, *getDefaultPool());
-    batch.keys = std::unique_ptr<ColumnVectorBatch>(keyBatch);
-    batch.elements = std::unique_ptr<ColumnVectorBatch>(valueBatch);
-    batch.numElements = 4;
-    batch.hasNulls = false;
-    batch.offsets[0] = 0;
-    for(size_t i = 1; i <= batch.numElements; ++i) {
-      batch.offsets[i] = batch.offsets[i-1] + static_cast<int64_t>(i-1);
-    }
-    keyBatch->numElements = 6;
-    keyBatch->hasNulls = false;
-    valueBatch->numElements = 6;
-    valueBatch->hasNulls = false;
-    for(size_t i = 0; i < keyBatch->numElements; ++i) {
-      keyBatch->data[i] = static_cast<int64_t>(i);
-      valueBatch->data[i] = static_cast<int64_t>(2 * i);
-    }
-    const char* expected[] = {"[]",
-                              "[{\"key\": 0, \"value\": 0}]",
-                              ("[{\"key\": 1, \"value\": 2},"
-                               " {\"key\": 2, \"value\": 4}]"),
-                              ("[{\"key\": 3, \"value\": 6},"
-                               " {\"key\": 4, \"value\": 8},"
-                               " {\"key\": 5, \"value\": 10}]")};
-    printer->reset(batch);
-    for(uint64_t i=0; i < batch.numElements; ++i) {
-      line.clear();
-      printer->printRow(i);
-      EXPECT_EQ(expected[i], line) << "for i = " << i;
-    }
-    batch.hasNulls = true;
-    for(size_t i = 0; i < batch.numElements; ++i) {
-      batch.notNull[i] = i % 2;
-    }
-    printer->reset(batch);
-    for(uint64_t i=0; i < batch.numElements; ++i) {
-      line.clear();
-      printer->printRow(i);
-      if (i % 2) {
-        EXPECT_EQ(expected[i], line) << "for i = " << i;
-      } else {
-        EXPECT_EQ("null", line) << "for i = " << i;
-      }
-    }
-  }
-
-  TEST(TestColumnPrinter, StructColumnPrinter) {
-    std::string line;
-    std::vector<std::string> fieldNames;
-    std::vector<Type*> subtypes;
-    fieldNames.push_back("first");
-    fieldNames.push_back("second");
-    subtypes.push_back(createPrimitiveType(LONG).release());
-    subtypes.push_back(createPrimitiveType(LONG).release());
-    std::unique_ptr<Type> type = createStructType(subtypes, fieldNames);
-    std::unique_ptr<ColumnPrinter> printer =
-      createColumnPrinter(line, *type);
-    StructVectorBatch batch(1024, *getDefaultPool());
-    LongVectorBatch* firstBatch = new LongVectorBatch(1024, *getDefaultPool());
-    LongVectorBatch* secondBatch =
-      new LongVectorBatch(1024, *getDefaultPool());
-    batch.fields.push_back(firstBatch);
-    batch.fields.push_back(secondBatch);
-    batch.numElements = 10;
-    batch.hasNulls = false;
-    firstBatch->numElements = 10;
-    firstBatch->hasNulls = false;
-    secondBatch->numElements = 10;
-    secondBatch->hasNulls = false;
-    for(size_t i = 0; i < firstBatch->numElements; ++i) {
-      firstBatch->data[i] = static_cast<int64_t>(i);
-      secondBatch->data[i] = static_cast<int64_t>(2 * i);
-    }
-    const char* expected[] = {"{\"first\": 0, \"second\": 0}",
-                              "{\"first\": 1, \"second\": 2}",
-                              "{\"first\": 2, \"second\": 4}",
-                              "{\"first\": 3, \"second\": 6}",
-                              "{\"first\": 4, \"second\": 8}",
-                              "{\"first\": 5, \"second\": 10}",
-                              "{\"first\": 6, \"second\": 12}",
-                              "{\"first\": 7, \"second\": 14}",
-                              "{\"first\": 8, \"second\": 16}",
-                              "{\"first\": 9, \"second\": 18}"};
-    printer->reset(batch);
-    for(uint64_t i=0; i < batch.numElements; ++i) {
-      line.clear();
-      printer->printRow(i);
-      EXPECT_EQ(expected[i], line) << "for i = " << i;
-    }
-    batch.hasNulls = true;
-    for(size_t i = 0; i < batch.numElements; ++i) {
-      batch.notNull[i] = i % 2;
-    }
-    printer->reset(batch);
-    for(uint64_t i=0; i < batch.numElements; ++i) {
-      line.clear();
-      printer->printRow(i);
-      if (i % 2) {
-        EXPECT_EQ(expected[i], line) << "for i = " << i;
-      } else {
-        EXPECT_EQ("null", line) << "for i = " << i;
-      }
-    }
-  }
-}  // namespace orc


[05/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools-c++/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/tools-c++/CMakeLists.txt b/tools-c++/CMakeLists.txt
deleted file mode 100644
index 9606d3b..0000000
--- a/tools-c++/CMakeLists.txt
+++ /dev/null
@@ -1,14 +0,0 @@
-# Licensed 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.
-
-add_subdirectory(src)
-add_subdirectory(test)

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools-c++/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/tools-c++/src/CMakeLists.txt b/tools-c++/src/CMakeLists.txt
deleted file mode 100644
index bdcbba2..0000000
--- a/tools-c++/src/CMakeLists.txt
+++ /dev/null
@@ -1,64 +0,0 @@
-# Licensed 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_directories (
-  ${PROJECT_SOURCE_DIR}/c++/include
-  ${PROJECT_SOURCE_DIR}/c++/src
-  ${PROJECT_BINARY_DIR}/c++/include
-  ${PROJECT_BINARY_DIR}/c++/src
-  ${PROTOBUF_INCLUDE_DIRS}
-  )
-
-set (CMAKE_CXX_FLAGS "-O0 ${CMAKE_CXX_FLAGS} -g ${CXX11_FLAGS} ${WARN_FLAGS}")
-
-add_executable (file-contents
-  FileContents.cc
-  )
-
-target_link_libraries (file-contents
-  orc
-  ${PROTOBUF_LIBRARIES}
-  )
-
-add_executable (file-scan
-  FileScan.cc
-  )
-
-target_link_libraries (file-scan
-  orc
-  ${PROTOBUF_LIBRARIES}
-  )
-
-add_executable (file-metadata
-  FileMetadata.cc
-  )
-
-target_link_libraries (file-metadata
-  orc
-  ${PROTOBUF_LIBRARIES}
-  )
-  
- add_executable (file-statistics
-  FileStatistics.cc
-  )
-
-target_link_libraries (file-statistics
-  orc
-  ${PROTOBUF_LIBRARIES}
-  ) 
-
-install(TARGETS
-   file-contents
-   file-metadata
-   file-scan
-   file-statistics
-   DESTINATION bin)

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools-c++/src/FileContents.cc
----------------------------------------------------------------------
diff --git a/tools-c++/src/FileContents.cc b/tools-c++/src/FileContents.cc
deleted file mode 100644
index 377c15d..0000000
--- a/tools-c++/src/FileContents.cc
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * 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 "orc/ColumnPrinter.hh"
-#include "orc/Exceptions.hh"
-
-#include <memory>
-#include <string>
-#include <iostream>
-#include <string>
-
-int main(int argc, char* argv[]) {
-  if (argc < 2) {
-    std::cout << "Usage: file-contents <filename>\n";
-    return 1;
-  }
-  orc::ReaderOptions opts;
-  std::list<int64_t> cols;
-  cols.push_back(0);
-  opts.include(cols);
-
-  std::unique_ptr<orc::Reader> reader;
-  try{
-    reader = orc::createReader(orc::readLocalFile(std::string(argv[1])), opts);
-  } catch (orc::ParseError e) {
-    std::cout << "Error reading file " << argv[1] << "! "
-              << e.what() << std::endl;
-    return -1;
-  }
-
-  std::unique_ptr<orc::ColumnVectorBatch> batch = reader->createRowBatch(1000);
-  std::string line;
-  std::unique_ptr<orc::ColumnPrinter> printer =
-    createColumnPrinter(line, reader->getType());
-
-  while (reader->next(*batch)) {
-    printer->reset(*batch);
-    for(unsigned long i=0; i < batch->numElements; ++i) {
-      line.clear();
-      printer->printRow(i);
-      line += "\n";
-      const char* str = line.c_str();
-      fwrite(str, 1, strlen(str), stdout);
-    }
-  }
-  return 0;
-}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools-c++/src/FileMetadata.cc
----------------------------------------------------------------------
diff --git a/tools-c++/src/FileMetadata.cc b/tools-c++/src/FileMetadata.cc
deleted file mode 100644
index d193ad8..0000000
--- a/tools-c++/src/FileMetadata.cc
+++ /dev/null
@@ -1,181 +0,0 @@
-/**
- * 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 <iostream>
-#include <fstream>
-#include <vector>
-#include <string>
-#include <sstream>
-#include <iomanip>
-
-#include "wrap/orc-proto-wrapper.hh"
-#include "orc/OrcFile.hh"
-
-using namespace orc::proto;
-
-uint64_t getTotalPaddingSize(Footer footer);
-
-int main(int argc, char* argv[])
-{
-  std::ifstream input;
-
-  GOOGLE_PROTOBUF_VERIFY_VERSION;
-
-  if (argc < 2) {
-    std::cout << "Usage: file-metadata <filename>\n";
-  }
-
-  std::cout << "Structure for " << argv[1] << std::endl;
-
-  input.open(argv[1], std::ios::in | std::ios::binary);
-  input.seekg(0,input.end);
-  std::streamoff fileSize = input.tellg();
-
-  // Read the postscript size
-  input.seekg(fileSize-1);
-  int result = input.get();
-  if (result == EOF) {
-    std::cerr << "Failed to read postscript size\n";
-    return -1;
-  }
-  std::streamoff postscriptSize = result;
-
-  // Read the postscript
-  input.seekg(fileSize - postscriptSize-1);
-  std::vector<char> buffer(static_cast<size_t>(postscriptSize));
-  input.read(buffer.data(), postscriptSize);
-  PostScript postscript ;
-  postscript.ParseFromArray(buffer.data(),
-                            static_cast<int>(postscriptSize));
-  std::cout << std::endl << " === Postscript === " << std::endl ;
-  postscript.PrintDebugString();
-
-  // Everything but the postscript is compressed
-  switch (static_cast<int>(postscript.compression())) {
-  case NONE:
-      break;
-  case ZLIB:
-  case SNAPPY:
-  case LZO:
-  default:
-      std::cout << "ORC files with compression are not supported" << std::endl ;
-      input.close();
-      return -1;
-  };
-
-  std::streamoff footerSize =
-    static_cast<std::streamoff>(postscript.footerlength());
-  std::streamoff metadataSize =
-    static_cast<std::streamoff>(postscript.metadatalength());
-
-  // Read the metadata
-  input.seekg(fileSize - 1 - postscriptSize - footerSize - metadataSize);
-  buffer.resize(static_cast<size_t>(metadataSize));
-  input.read(buffer.data(), metadataSize);
-  Metadata metadata ;
-  metadata.ParseFromArray(buffer.data(), static_cast<int>(metadataSize));
-
-  // Read the footer
-  //input.seekg(fileSize -1 - postscriptSize-footerSize);
-  buffer.resize(static_cast<size_t>(footerSize));
-  input.read(buffer.data(), footerSize);
-  Footer footer ;
-  footer.ParseFromArray(buffer.data(), static_cast<int>(footerSize));
-  std::cout << std::endl << " === Footer === " << std::endl ;
-  footer.PrintDebugString();
-
-  std::cout << std::endl << "=== Stripe Statistics ===" << std::endl;
-
-  StripeInformation stripe ;
-  Stream section;
-  ColumnEncoding encoding;
-  for (int stripeIx=0; stripeIx<footer.stripes_size(); stripeIx++)
-  {
-      std::cout << "Stripe " << stripeIx+1 <<": " << std::endl ;
-      stripe = footer.stripes(stripeIx);
-      stripe.PrintDebugString();
-
-      std::streamoff offset =
-        static_cast<std::streamoff>(stripe.offset() + stripe.indexlength() +
-                                    stripe.datalength());
-      std::streamoff tailLength =
-        static_cast<std::streamoff>(stripe.footerlength());
-
-      // read the stripe footer
-      input.seekg(offset);
-      buffer.resize(static_cast<size_t>(tailLength));
-      input.read(buffer.data(), tailLength);
-
-      StripeFooter stripeFooter;
-      stripeFooter.ParseFromArray(buffer.data(), static_cast<int>(tailLength));
-      //stripeFooter.PrintDebugString();
-      uint64_t stripeStart = stripe.offset();
-      uint64_t sectionStart = stripeStart;
-      for (int streamIx=0; streamIx<stripeFooter.streams_size(); streamIx++) {
-          section = stripeFooter.streams(streamIx);
-          std::cout << "    Stream: column " << section.column()
-                    << " section "
-                    << section.kind() << " start: " << sectionStart
-                    << " length " << section.length() << std::endl;
-          sectionStart += section.length();
-      };
-      for (int columnIx=0; columnIx<stripeFooter.columns_size();
-           columnIx++) {
-          encoding = stripeFooter.columns(columnIx);
-          std::cout << "    Encoding column " << columnIx << ": "
-                    << encoding.kind() ;
-          if (encoding.kind() == ColumnEncoding_Kind_DICTIONARY ||
-              encoding.kind() == ColumnEncoding_Kind_DICTIONARY_V2)
-              std::cout << "[" << encoding.dictionarysize() << "]";
-          std::cout << std::endl;
-      };
-  };
-
-  uint64_t paddedBytes = getTotalPaddingSize(footer);
-  // empty ORC file is ~45 bytes. Assumption here is file length always >0
-  double percentPadding =
-    static_cast<double>(paddedBytes) * 100 / static_cast<double>(fileSize);
-  std::cout << "File length: " << fileSize << " bytes" << std::endl;
-  std::cout <<"Padding length: " << paddedBytes << " bytes" << std::endl;
-  std::cout <<"Padding ratio: " << std::fixed << std::setprecision(2)
-            << percentPadding << " %" << std::endl;
-
-  input.close();
-
-
-
-  google::protobuf::ShutdownProtobufLibrary();
-
-  return 0;
-}
-
-uint64_t getTotalPaddingSize(Footer footer) {
-  uint64_t paddedBytes = 0;
-  StripeInformation stripe;
-  for (int stripeIx=1; stripeIx<footer.stripes_size(); stripeIx++) {
-      stripe = footer.stripes(stripeIx-1);
-      uint64_t prevStripeOffset = stripe.offset();
-      uint64_t prevStripeLen = stripe.datalength() + stripe.indexlength() +
-        stripe.footerlength();
-      paddedBytes += footer.stripes(stripeIx).offset() -
-        (prevStripeOffset + prevStripeLen);
-  };
-  return paddedBytes;
-}
-
-

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools-c++/src/FileScan.cc
----------------------------------------------------------------------
diff --git a/tools-c++/src/FileScan.cc b/tools-c++/src/FileScan.cc
deleted file mode 100644
index c74e92d..0000000
--- a/tools-c++/src/FileScan.cc
+++ /dev/null
@@ -1,56 +0,0 @@
-/**
- * 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 "orc/ColumnPrinter.hh"
-#include "orc/Exceptions.hh"
-
-#include <string>
-#include <memory>
-#include <iostream>
-#include <string>
-
-int main(int argc, char* argv[]) {
-  if (argc < 2) {
-    std::cout << "Usage: file-scan <filename>\n";
-  }
-
-  orc::ReaderOptions opts;
-  std::list<int64_t> cols;
-  cols.push_back(0);
-  opts.include(cols);
-
-  std::unique_ptr<orc::Reader> reader;
-  try{
-    reader = orc::createReader(orc::readLocalFile(std::string(argv[1])), opts);
-  } catch (orc::ParseError e) {
-    std::cout << "Error reading file " << argv[1] << "! "
-              << e.what() << std::endl;
-    return -1;
-  }
-
-  std::unique_ptr<orc::ColumnVectorBatch> batch = reader->createRowBatch(1000);
-  unsigned long rows = 0;
-  unsigned long batches = 0;
-  while (reader->next(*batch)) {
-    batches += 1;
-    rows += batch->numElements;
-  }
-  std::cout << "Rows: " << rows << std::endl;
-  std::cout << "Batches: " << batches << std::endl;
-  return 0;
-}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools-c++/src/FileStatistics.cc
----------------------------------------------------------------------
diff --git a/tools-c++/src/FileStatistics.cc b/tools-c++/src/FileStatistics.cc
deleted file mode 100644
index 1b59abf..0000000
--- a/tools-c++/src/FileStatistics.cc
+++ /dev/null
@@ -1,75 +0,0 @@
-/**
- * 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 "orc/ColumnPrinter.hh"
-#include "orc/Exceptions.hh"
-
-#include <string>
-#include <memory>
-#include <iostream>
-#include <string>
-
-int main(int argc, char* argv[]) {
-  if (argc < 2) {
-    std::cout << "Usage: file-metadata <filename>\n";
-  }
-
-  orc::ReaderOptions opts;
-  std::list<int64_t> cols;
-  cols.push_back(0);
-  opts.include(cols);
-
-  std::unique_ptr<orc::Reader> reader;
-  try{
-    reader = orc::createReader(orc::readLocalFile(std::string(argv[1])), opts);
-  } catch (orc::ParseError e) {
-    std::cout << "Error reading file " << argv[1] << "! "
-              << e.what() << std::endl;
-    return -1;
-  }
-
-  // print out all selected columns statistics.
-  std::unique_ptr<orc::Statistics> colStats = reader->getStatistics();
-  std::cout << "File " << argv[1] << " has "
-            << colStats->getNumberOfColumns() << " columns"  << std::endl;
-  for(uint32_t i=0; i < colStats->getNumberOfColumns(); ++i) {
-    std::cout << "*** Column " << i << " ***" << std::endl;
-    std::cout << colStats->getColumnStatistics(i)->toString() << std::endl;
-  }
-
-  // test stripe statistics
-  std::unique_ptr<orc::Statistics> stripeStats;
-  std::cout << "File " << argv[1] << " has " << reader->getNumberOfStripes()
-            << " stripes"  << std::endl;
-  if(reader->getNumberOfStripeStatistics() == 0){
-    std::cout << "File " << argv[1] << " doesn't have stripe statistics"  << std::endl;
-  }else{
-    for (unsigned int j = 0; j < reader->getNumberOfStripeStatistics(); j++) {
-      stripeStats = reader->getStripeStatistics(j);
-      std::cout << "*** Stripe " << j << " ***" << std::endl << std::endl ;
-
-      for(unsigned int k = 0; k < stripeStats->getNumberOfColumns(); ++k) {
-        std::cout << "--- Column " << k << " ---" << std::endl;
-        std::cout << stripeStats->getColumnStatistics(k)->toString()
-                  << std::endl;
-      }
-    }
-  }
-
-  return 0;
-}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools-c++/test/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/tools-c++/test/CMakeLists.txt b/tools-c++/test/CMakeLists.txt
deleted file mode 100644
index ade81ff..0000000
--- a/tools-c++/test/CMakeLists.txt
+++ /dev/null
@@ -1,40 +0,0 @@
-# Licensed 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_directories(
-  ${PROJECT_SOURCE_DIR}/c++/include
-  ${PROJECT_SOURCE_DIR}/c++/src
-  ${PROJECT_SOURCE_DIR}/tools-c++/src
-  ${PROJECT_BINARY_DIR}/c++/include
-  ${PROJECT_BINARY_DIR}/c++/src
-  ${PROTOBUF_INCLUDE_DIRS}
-  ${ZLIB_INCLUDE_DIRS}
-  ${GMOCK_INCLUDE_DIRS}
-)
-
-set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${CXX11_FLAGS} ${WARN_FLAGS}")
-
-add_executable (tool-test
-  gzip.cc
-  TestReader.cc
-  ToolTest.cc
-)
-
-target_link_libraries (tool-test
-  orc
-  ${PROTOBUF_LIBRARIES}
-  ${GMOCK_LIBRARIES}
-  ${ZLIB_LIBRARIES}
-  ${SNAPPY_LIBRARIES}
-)
-
-add_test (tool-test tool-test ${EXAMPLE_DIRECTORY})


[21/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/Exceptions.cc
----------------------------------------------------------------------
diff --git a/c++/src/Exceptions.cc b/c++/src/Exceptions.cc
new file mode 100644
index 0000000..ae0e3d1
--- /dev/null
+++ b/c++/src/Exceptions.cc
@@ -0,0 +1,59 @@
+/**
+ * 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 "Exceptions.hh"
+
+namespace orc {
+
+  NotImplementedYet::NotImplementedYet(const std::string& what_arg
+                                       ) : logic_error(what_arg) {
+    // PASS
+  }
+
+  NotImplementedYet::NotImplementedYet(const char* what_arg
+                                       ) :logic_error(what_arg) {
+    // PASS
+  }
+
+  NotImplementedYet::NotImplementedYet(const NotImplementedYet& error
+                                       ): logic_error(error) {
+    // PASS
+  }
+
+  NotImplementedYet::~NotImplementedYet() noexcept {
+    // PASS
+  }
+
+  ParseError::ParseError(const std::string& what_arg
+                         ): runtime_error(what_arg) {
+    // PASS
+  }
+
+  ParseError::ParseError(const char* what_arg
+                         ): runtime_error(what_arg) {
+    // PASS
+  }
+
+  ParseError::ParseError(const ParseError& error): runtime_error(error) {
+    // PASS
+  }
+
+  ParseError::~ParseError() noexcept {
+    // PASS
+  }
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/Exceptions.hh
----------------------------------------------------------------------
diff --git a/c++/src/Exceptions.hh b/c++/src/Exceptions.hh
new file mode 100644
index 0000000..4706085
--- /dev/null
+++ b/c++/src/Exceptions.hh
@@ -0,0 +1,50 @@
+/**
+ * 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 ORC_EXCEPTIONS_HH
+#define ORC_EXCEPTIONS_HH
+
+#include "Adaptor.hh"
+
+#include <stdexcept>
+#include <string>
+
+namespace orc {
+
+  class NotImplementedYet: public std::logic_error {
+  public:
+    explicit NotImplementedYet(const std::string& what_arg);
+    explicit NotImplementedYet(const char* what_arg);
+    virtual ~NotImplementedYet() noexcept;
+    NotImplementedYet(const NotImplementedYet&);
+  private:
+    NotImplementedYet& operator=(const NotImplementedYet&);
+  };
+
+  class ParseError: public std::runtime_error {
+  public:
+    explicit ParseError(const std::string& what_arg);
+    explicit ParseError(const char* what_arg);
+    virtual ~ParseError() noexcept;
+    ParseError(const ParseError&);
+  private:
+    ParseError& operator=(const ParseError&);
+  };
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/Int128.cc
----------------------------------------------------------------------
diff --git a/c++/src/Int128.cc b/c++/src/Int128.cc
new file mode 100644
index 0000000..ece7850
--- /dev/null
+++ b/c++/src/Int128.cc
@@ -0,0 +1,438 @@
+/**
+ * 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 "orc/Int128.hh"
+#include "Adaptor.hh"
+
+#include <iomanip>
+#include <iostream>
+#include <sstream>
+
+namespace orc {
+
+  Int128 Int128::maximumValue() {
+    return Int128(0x7fffffffffffffff, 0xfffffffffffffff);
+  }
+
+  Int128 Int128::minimumValue() {
+    return Int128(static_cast<int64_t>(0x8000000000000000), 0x0);
+  }
+
+  Int128::Int128(const std::string& str) {
+    lowbits = 0;
+    highbits = 0;
+    size_t length = str.length();
+    if (length > 0) {
+      bool isNegative = str[0] == '-';
+      size_t posn = isNegative ? 1 : 0;
+      while (posn < length) {
+        size_t group = std::min(18ul, length - posn);
+        int64_t chunk = std::stoll(str.substr(posn, group));
+        int64_t multiple = 1;
+        for(size_t i=0; i < group; ++i) {
+          multiple *= 10;
+        }
+        *this *= multiple;
+        *this += chunk;
+        posn += group;
+      }
+      if (isNegative) {
+        negate();
+      }
+    }
+  }
+
+  Int128& Int128::operator*=(const Int128 &right) {
+    const uint64_t INT_MASK = 0xffffffff;
+    const uint64_t CARRY_BIT = 1l << 32;
+
+    // Break the left and right numbers into 32 bit chunks
+    // so that we can multiply them without overflow.
+    uint64_t L0 = static_cast<uint64_t>(highbits) >> 32;
+    uint64_t L1 = static_cast<uint64_t>(highbits) & INT_MASK;
+    uint64_t L2 = lowbits >> 32;
+    uint64_t L3 = lowbits & INT_MASK;
+    uint64_t R0 = static_cast<uint64_t>(right.highbits) >> 32;
+    uint64_t R1 = static_cast<uint64_t>(right.highbits) & INT_MASK;
+    uint64_t R2 = right.lowbits >> 32;
+    uint64_t R3 = right.lowbits & INT_MASK;
+
+    uint64_t product = L3 * R3;
+    lowbits = product & INT_MASK;
+    uint64_t sum = product >> 32;
+    product = L2 * R3;
+    sum += product;
+    highbits = sum < product ? CARRY_BIT : 0;
+    product = L3 * R2;
+    sum += product;
+    if (sum < product) {
+      highbits += CARRY_BIT;
+    }
+    lowbits += sum << 32;
+    highbits += static_cast<int64_t>(sum >> 32);
+    highbits += L1 * R3 + L2 * R2 + L3 * R1;
+    highbits += (L0 * R3 + L1 * R2 + L2 * R1 + L3 * R0) << 32;
+    return *this;
+  }
+
+  /**
+   * Expands the given value into an array of ints so that we can work on
+   * it. The array will be converted to an absolute value and the wasNegative
+   * flag will be set appropriately. The array will remove leading zeros from
+   * the value.
+   * @param array an array of length 4 to set with the value
+   * @param wasNegative a flag for whether the value was original negative
+   * @result the output length of the array
+   */
+  int64_t Int128::fillInArray(uint32_t* array, bool &wasNegative) const {
+    uint64_t high;
+    uint64_t low;
+    if (highbits < 0) {
+      low = ~lowbits + 1;
+      high = static_cast<uint64_t>(~highbits);
+      if (low == 0) {
+        high += 1;
+      }
+      wasNegative = true;
+    } else {
+      low = lowbits;
+      high = static_cast<uint64_t>(highbits);
+      wasNegative = false;
+    }
+    if (high != 0) {
+      if (high > UINT32_MAX) {
+        array[0] = static_cast<uint32_t>(high >> 32);
+        array[1] = static_cast<uint32_t>(high);
+        array[2] = static_cast<uint32_t>(low >> 32);
+        array[3] = static_cast<uint32_t>(low);
+        return 4;
+      } else {
+        array[0] = static_cast<uint32_t>(high);
+        array[1] = static_cast<uint32_t>(low >> 32);
+        array[2] = static_cast<uint32_t>(low);
+        return 3;
+      }
+    } else if (low >= UINT32_MAX) {
+      array[0] = static_cast<uint32_t>(low >> 32);
+      array[1] = static_cast<uint32_t>(low);
+      return 2;
+    } else if (low == 0) {
+      return 0;
+    } else {
+      array[0] = static_cast<uint32_t>(low);
+      return 1;
+    }
+  }
+
+
+  /**
+   * Find last set bit in a 32 bit integer. Bit 1 is the LSB and bit 32 is
+   * the MSB. We can replace this with bsrq asm instruction on x64.
+   */
+  int64_t fls(uint32_t x) {
+    int64_t bitpos = 0;
+    while (x) {
+      x >>= 1;
+      bitpos += 1;
+    }
+    return bitpos;
+  }
+
+  /**
+   * Shift the number in the array left by bits positions.
+   * @param array the number to shift, must have length elements
+   * @param length the number of entries in the array
+   * @param bits the number of bits to shift (0 <= bits < 32)
+   */
+  void shiftArrayLeft(uint32_t* array, int64_t length, int64_t bits) {
+    if (length > 0 && bits != 0) {
+      for(int64_t i=0; i < length-1; ++i) {
+        array[i] = (array[i] << bits) | (array[i+1] >> (32 - bits));
+      }
+      array[length-1] <<= bits;
+    }
+  }
+
+  /**
+   * Shift the number in the array right by bits positions.
+   * @param array the number to shift, must have length elements
+   * @param length the number of entries in the array
+   * @param bits the number of bits to shift (0 <= bits < 32)
+   */
+  void shiftArrayRight(uint32_t* array, int64_t length, int64_t bits) {
+    if (length > 0 && bits != 0) {
+      for(int64_t i=length-1; i > 0; --i) {
+        array[i] = (array[i] >> bits) | (array[i-1] << (32 - bits));
+      }
+      array[0] >>= bits;
+    }
+  }
+
+  /**
+   * Fix the signs of the result and remainder at the end of the division
+   * based on the signs of the dividend and divisor.
+   */
+  void fixDivisionSigns(Int128 &result, Int128 &remainder,
+                        bool dividendWasNegative, bool divisorWasNegative) {
+    if (dividendWasNegative != divisorWasNegative) {
+      result.negate();
+    }
+    if (dividendWasNegative) {
+      remainder.negate();
+    }
+  }
+
+  /**
+   * Build a Int128 from a list of ints.
+   */
+  void buildFromArray(Int128& value, uint32_t* array, int64_t length) {
+    switch (length) {
+    case 0:
+      value = 0;
+      break;
+    case 1:
+      value = array[0];
+      break;
+    case 2:
+      value = Int128(0, (static_cast<uint64_t>(array[0]) << 32) + array[1]);
+      break;
+    case 3:
+      value = Int128(array[0],
+                     (static_cast<uint64_t>(array[1]) << 32) + array[2]);
+      break;
+    case 4:
+      value = Int128((static_cast<int64_t>(array[0]) << 32) + array[1],
+                     (static_cast<uint64_t>(array[2]) << 32) + array[3]);
+      break;
+    case 5:
+      if (array[0] != 0) {
+        throw std::logic_error("Can't build Int128 with 5 ints.");
+      }
+      value = Int128((static_cast<int64_t>(array[1]) << 32) + array[2],
+                     (static_cast<uint64_t>(array[3]) << 32) + array[4]);
+      break;
+    default:
+      throw std::logic_error("Unsupported length for building Int128");
+    }
+  }
+
+  /**
+   * Do a division where the divisor fits into a single 32 bit value.
+   */
+  Int128 singleDivide(uint32_t* dividend, int64_t dividendLength,
+                      uint32_t divisor, Int128& remainder,
+                      bool dividendWasNegative, bool divisorWasNegative) {
+    uint64_t r = 0;
+    uint32_t resultArray[5];
+    for(int64_t j=0; j < dividendLength; j++) {
+      r <<= 32;
+      r += dividend[j];
+      resultArray[j] = static_cast<uint32_t>(r / divisor);
+      r %= divisor;
+    }
+    Int128 result;
+    buildFromArray(result, resultArray, dividendLength);
+    remainder = static_cast<int64_t>(r);
+    fixDivisionSigns(result, remainder, dividendWasNegative,
+                     divisorWasNegative);
+    return result;
+  }
+
+  Int128 Int128::divide(const Int128 &divisor, Int128 &remainder) const {
+    // Split the dividend and divisor into integer pieces so that we can
+    // work on them.
+    uint32_t dividendArray[5];
+    uint32_t divisorArray[4];
+    bool dividendWasNegative;
+    bool divisorWasNegative;
+    // leave an extra zero before the dividend
+    dividendArray[0] = 0;
+    int64_t dividendLength = fillInArray(dividendArray + 1, dividendWasNegative)+1;
+    int64_t divisorLength = divisor.fillInArray(divisorArray, divisorWasNegative);
+
+    // Handle some of the easy cases.
+    if (dividendLength <= divisorLength) {
+      remainder = *this;
+      return 0;
+    } else if (divisorLength == 0) {
+      throw std::range_error("Division by 0 in Int128");
+    } else if (divisorLength == 1) {
+      return singleDivide(dividendArray, dividendLength, divisorArray[0],
+                          remainder, dividendWasNegative, divisorWasNegative);
+    }
+
+    int64_t resultLength = dividendLength - divisorLength;
+    uint32_t resultArray[4];
+
+    // Normalize by shifting both by a multiple of 2 so that
+    // the digit guessing is better. The requirement is that
+    // divisorArray[0] is greater than 2**31.
+    int64_t normalizeBits = 32 - fls(divisorArray[0]);
+    shiftArrayLeft(divisorArray, divisorLength, normalizeBits);
+    shiftArrayLeft(dividendArray, dividendLength, normalizeBits);
+
+    // compute each digit in the result
+    for(int64_t j=0; j < resultLength; ++j) {
+      // Guess the next digit. At worst it is two too large
+      uint32_t guess = UINT32_MAX;
+      uint64_t highDividend = static_cast<uint64_t>(dividendArray[j]) << 32 |
+        dividendArray[j+1];
+      if (dividendArray[j] != divisorArray[0]) {
+        guess = static_cast<uint32_t>(highDividend / divisorArray[0]);
+      }
+
+      // catch all of the cases where guess is two too large and most of the
+      // cases where it is one too large
+      uint32_t rhat =
+        static_cast<uint32_t>(highDividend - guess *
+                              static_cast<uint64_t>(divisorArray[0]));
+      while (static_cast<uint64_t>(divisorArray[1]) * guess >
+             (static_cast<uint64_t>(rhat) << 32) + dividendArray[j+2]) {
+        guess -= 1;
+        rhat += divisorArray[0];
+        if (static_cast<uint64_t>(rhat) < divisorArray[0]) {
+          break;
+        }
+      }
+
+      // subtract off the guess * divisor from the dividend
+      uint64_t mult = 0;
+      for(int64_t i=divisorLength-1; i >= 0; --i) {
+        mult += static_cast<uint64_t>(guess) * divisorArray[i];
+        uint32_t prev = dividendArray[j+i+1];
+        dividendArray[j+i+1] -= static_cast<uint32_t>(mult);
+        mult >>= 32;
+        if (dividendArray[j+i+1] > prev) {
+          mult += 1;
+        }
+      }
+      uint32_t prev = dividendArray[j];
+      dividendArray[j] -= static_cast<uint32_t>(mult);
+
+      // if guess was too big, we add back divisor
+      if (dividendArray[j] > prev) {
+        guess -= 1;
+        uint32_t carry = 0;
+        for(int64_t i=divisorLength-1; i >= 0; --i) {
+          uint64_t sum = static_cast<uint64_t>(divisorArray[i]) +
+            dividendArray[j+i+1] + carry;
+          dividendArray[j+i+1] = static_cast<uint32_t>(sum);
+          carry = static_cast<uint32_t>(sum >> 32);
+        }
+        dividendArray[j] += carry;
+      }
+
+      resultArray[j] = guess;
+    }
+
+    // denormalize the remainder
+    shiftArrayRight(dividendArray, dividendLength, normalizeBits);
+
+    // return result and remainder
+    Int128 result;
+    buildFromArray(result, resultArray, resultLength);
+    buildFromArray(remainder, dividendArray, dividendLength);
+    fixDivisionSigns(result, remainder,
+                     dividendWasNegative, divisorWasNegative);
+    return result;
+  }
+
+  std::string Int128::toString() const {
+    // 10**18 - the largest power of 10 less than 63 bits
+    const Int128 tenTo18(0xde0b6b3a7640000);
+    // 10**36
+    const Int128 tenTo36(0xc097ce7bc90715, 0xb34b9f1000000000);
+    Int128 remainder;
+    std::stringstream buf;
+    bool needFill = false;
+
+    // get anything above 10**36 and print it
+    Int128 top = divide(tenTo36, remainder);
+    if (top != 0) {
+      buf << top.toLong();
+      remainder.abs();
+      needFill = true;
+    }
+
+    // now get anything above 10**18 and print it
+    Int128 tail;
+    top = remainder.divide(tenTo18, tail);
+    if (needFill || top != 0) {
+      if (needFill) {
+        buf << std::setw(18) << std::setfill('0');
+      } else {
+        needFill = true;
+        tail.abs();
+      }
+      buf << top.toLong();
+    }
+
+    // finally print the tail, which is less than 10**18
+    if (needFill) {
+      buf << std::setw(18) << std::setfill('0');
+    }
+    buf << tail.toLong();
+    return buf.str();
+  }
+
+  std::string Int128::toDecimalString(int32_t scale) const {
+    std::string str = toString();
+    if (scale == 0) {
+      return str;
+    } else if (*this < 0) {
+      int32_t len = static_cast<int32_t>(str.length());
+      if (len - 1 > scale) {
+        return str.substr(0, static_cast<size_t>(len - scale)) + "." +
+          str.substr(static_cast<size_t>(len - scale),
+                     static_cast<size_t>(scale));
+      } else if (len - 1 == scale) {
+        return "-0." + str.substr(1, std::string::npos);
+      } else {
+        std::string result = "-0.";
+        for(int32_t i=0; i < scale - len + 1; ++i) {
+          result += "0";
+        }
+        return result + str.substr(1, std::string::npos);
+      }
+    } else {
+      int32_t len = static_cast<int32_t>(str.length());
+      if (len > scale) {
+        return str.substr(0, static_cast<size_t>(len - scale)) + "." +
+          str.substr(static_cast<size_t>(len - scale),
+                     static_cast<size_t>(scale));
+      } else if (len == scale) {
+        return "0." + str;
+      } else {
+        std::string result = "0.";
+        for(int32_t i=0; i < scale - len; ++i) {
+          result += "0";
+        }
+        return result + str;
+      }
+    }
+  }
+
+  std::string Int128::toHexString() const {
+    std::stringstream buf;
+    buf << std::hex << "0x"
+        << std::setw(16) << std::setfill('0') << highbits
+        << std::setw(16) << std::setfill('0') << lowbits;
+    return buf.str();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/MemoryPool.cc
----------------------------------------------------------------------
diff --git a/c++/src/MemoryPool.cc b/c++/src/MemoryPool.cc
new file mode 100644
index 0000000..28cc9e1
--- /dev/null
+++ b/c++/src/MemoryPool.cc
@@ -0,0 +1,232 @@
+/**
+ * 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 "orc/Int128.hh"
+#include "orc/MemoryPool.hh"
+
+#include "Adaptor.hh"
+
+#include <cstdlib>
+#include <iostream>
+#include <string.h>
+
+namespace orc {
+
+  MemoryPool::~MemoryPool() {
+    // PASS
+  }
+
+  class MemoryPoolImpl: public MemoryPool {
+  public:
+    virtual ~MemoryPoolImpl();
+
+    char* malloc(uint64_t size) override;
+    void free(char* p) override;
+  };
+
+  char* MemoryPoolImpl::malloc(uint64_t size) {
+    return static_cast<char*>(std::malloc(size));
+  }
+
+  void MemoryPoolImpl::free(char* p) {
+    std::free(p);
+  }
+
+  MemoryPoolImpl::~MemoryPoolImpl() {
+    // PASS
+  }
+
+  template <class T>
+  DataBuffer<T>::DataBuffer(MemoryPool& pool,
+                            uint64_t newSize
+                            ): memoryPool(pool),
+                               buf(nullptr),
+                               currentSize(0),
+                               currentCapacity(0) {
+    resize(newSize);
+  }
+
+  template <class T>
+  DataBuffer<T>::~DataBuffer(){
+    for(uint64_t i=currentSize; i > 0; --i) {
+      (buf + i - 1)->~T();
+    }
+    if (buf) {
+      memoryPool.free(reinterpret_cast<char*>(buf));
+    }
+  }
+
+  template <class T>
+  void DataBuffer<T>::resize(uint64_t newSize) {
+    reserve(newSize);
+    if (currentSize > newSize) {
+      for(uint64_t i=currentSize; i > newSize; --i) {
+        (buf + i - 1)->~T();
+      }
+    } else if (newSize > currentSize) {
+      for(uint64_t i=currentSize; i < newSize; ++i) {
+        new (buf + i) T();
+      }
+    }
+    currentSize = newSize;
+  }
+
+  template <class T>
+  void DataBuffer<T>::reserve(uint64_t newCapacity){
+    if (newCapacity > currentCapacity) {
+      if (buf) {
+        T* buf_old = buf;
+        buf = reinterpret_cast<T*>(memoryPool.malloc(sizeof(T) * newCapacity));
+        memcpy(buf, buf_old, sizeof(T) * currentSize);
+        memoryPool.free(reinterpret_cast<char*>(buf_old));
+      } else {
+        buf = reinterpret_cast<T*>(memoryPool.malloc(sizeof(T) * newCapacity));
+      }
+      currentCapacity = newCapacity;
+    }
+  }
+
+  // Specializations for char
+
+  template <>
+  DataBuffer<char>::~DataBuffer(){
+    if (buf) {
+      memoryPool.free(reinterpret_cast<char*>(buf));
+    }
+  }
+
+  template <>
+  void DataBuffer<char>::resize(uint64_t newSize) {
+    reserve(newSize);
+    if (newSize > currentSize) {
+      memset(buf + currentSize, 0, newSize - currentSize);
+    }
+    currentSize = newSize;
+  }
+
+  // Specializations for char*
+
+  template <>
+  DataBuffer<char*>::~DataBuffer(){
+    if (buf) {
+      memoryPool.free(reinterpret_cast<char*>(buf));
+    }
+  }
+
+  template <>
+  void DataBuffer<char*>::resize(uint64_t newSize) {
+    reserve(newSize);
+    if (newSize > currentSize) {
+      memset(buf + currentSize, 0, (newSize - currentSize) * sizeof(char*));
+    }
+    currentSize = newSize;
+  }
+
+  // Specializations for double
+
+  template <>
+  DataBuffer<double>::~DataBuffer(){
+    if (buf) {
+      memoryPool.free(reinterpret_cast<char*>(buf));
+    }
+  }
+
+  template <>
+  void DataBuffer<double>::resize(uint64_t newSize) {
+    reserve(newSize);
+    if (newSize > currentSize) {
+      memset(buf + currentSize, 0, (newSize - currentSize) * sizeof(double));
+    }
+    currentSize = newSize;
+  }
+
+  // Specializations for int64_t
+
+  template <>
+  DataBuffer<int64_t>::~DataBuffer(){
+    if (buf) {
+      memoryPool.free(reinterpret_cast<char*>(buf));
+    }
+  }
+
+  template <>
+  void DataBuffer<int64_t>::resize(uint64_t newSize) {
+    reserve(newSize);
+    if (newSize > currentSize) {
+      memset(buf + currentSize, 0, (newSize - currentSize) * sizeof(int64_t));
+    }
+    currentSize = newSize;
+  }
+
+  // Specializations for uint64_t
+
+  template <>
+  DataBuffer<uint64_t>::~DataBuffer(){
+    if (buf) {
+      memoryPool.free(reinterpret_cast<char*>(buf));
+    }
+  }
+
+  template <>
+  void DataBuffer<uint64_t>::resize(uint64_t newSize) {
+    reserve(newSize);
+    if (newSize > currentSize) {
+      memset(buf + currentSize, 0, (newSize - currentSize) * sizeof(uint64_t));
+    }
+    currentSize = newSize;
+  }
+
+  // Specializations for unsigned char
+
+  template <>
+  DataBuffer<unsigned char>::~DataBuffer(){
+    if (buf) {
+      memoryPool.free(reinterpret_cast<char*>(buf));
+    }
+  }
+
+  template <>
+  void DataBuffer<unsigned char>::resize(uint64_t newSize) {
+    reserve(newSize);
+    if (newSize > currentSize) {
+      memset(buf + currentSize, 0, newSize - currentSize);
+    }
+    currentSize = newSize;
+  }
+
+  #ifdef __clang__
+    #pragma clang diagnostic ignored "-Wweak-template-vtables"
+  #endif
+
+  template class DataBuffer<char>;
+  template class DataBuffer<char*>;
+  template class DataBuffer<double>;
+  template class DataBuffer<Int128>;
+  template class DataBuffer<int64_t>;
+  template class DataBuffer<uint64_t>;
+  template class DataBuffer<unsigned char>;
+
+  #ifdef __clang__
+    #pragma clang diagnostic ignored "-Wexit-time-destructors"
+  #endif
+
+  MemoryPool* getDefaultPool() {
+    static MemoryPoolImpl internal;
+    return &internal;
+  }
+} // namespace orc

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/OrcFile.cc
----------------------------------------------------------------------
diff --git a/c++/src/OrcFile.cc b/c++/src/OrcFile.cc
new file mode 100644
index 0000000..f8c22c4
--- /dev/null
+++ b/c++/src/OrcFile.cc
@@ -0,0 +1,102 @@
+/**
+ * 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 "orc/OrcFile.hh"
+
+#include "Adaptor.hh"
+#include "Exceptions.hh"
+
+#include <errno.h>
+#include <fcntl.h>
+#include <stdio.h>
+#include <sys/mman.h>
+#include <sys/stat.h>
+#include <sys/types.h>
+#include <unistd.h>
+
+namespace orc {
+
+  class FileInputStream : public InputStream {
+  private:
+    std::string filename ;
+    int file;
+    uint64_t totalLength;
+
+  public:
+    FileInputStream(std::string _filename) {
+      filename = _filename ;
+      file = open(filename.c_str(), O_RDONLY);
+      if (file == -1) {
+        throw ParseError("Can't open " + filename);
+      }
+      struct stat fileStat;
+      if (fstat(file, &fileStat) == -1) {
+        throw ParseError("Can't stat " + filename);
+      }
+      totalLength = static_cast<uint64_t>(fileStat.st_size);
+    }
+
+    ~FileInputStream();
+
+    uint64_t getLength() const override {
+      return totalLength;
+    }
+
+    void read(void* buf,
+              uint64_t length,
+              uint64_t offset) override {
+      if (!buf) {
+        throw ParseError("Buffer is null");
+      }
+      ssize_t bytesRead = pread(file, buf, length, static_cast<off_t>(offset));
+
+      if (bytesRead == -1) {
+        throw ParseError("Bad read of " + filename);
+      }
+      if (static_cast<uint64_t>(bytesRead) != length) {
+        throw ParseError("Short read of " + filename);
+      }
+    }
+
+    const std::string& getName() const override {
+      return filename;
+    }
+  };
+
+  FileInputStream::~FileInputStream() {
+    close(file);
+  }
+
+  std::unique_ptr<InputStream> readLocalFile(const std::string& path) {
+    return std::unique_ptr<InputStream>(new FileInputStream(path));
+  }
+}
+
+#ifndef HAS_STOLL
+
+  #include <sstream>
+
+  int64_t std::stoll(std::string str) {
+    int64_t val = 0;
+    stringstream ss ;
+    ss << str ;
+    ss >> val ;
+    return val;
+  }
+
+#endif

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/RLE.cc
----------------------------------------------------------------------
diff --git a/c++/src/RLE.cc b/c++/src/RLE.cc
new file mode 100644
index 0000000..51bd628
--- /dev/null
+++ b/c++/src/RLE.cc
@@ -0,0 +1,47 @@
+/**
+* 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 "RLEv1.hh"
+#include "RLEv2.hh"
+#include "Exceptions.hh"
+
+namespace orc {
+
+  RleDecoder::~RleDecoder() {
+    // PASS
+  }
+
+  std::unique_ptr<RleDecoder> createRleDecoder
+                         (std::unique_ptr<SeekableInputStream> input,
+                          bool isSigned,
+                          RleVersion version,
+                          MemoryPool& pool) {
+    switch (static_cast<int64_t>(version)) {
+    case RleVersion_1:
+      // We don't have std::make_unique() yet.
+      return std::unique_ptr<RleDecoder>(new RleDecoderV1(std::move(input),
+                                                          isSigned));
+    case RleVersion_2:
+      return std::unique_ptr<RleDecoder>(new RleDecoderV2(std::move(input),
+                                                          isSigned, pool));
+    default:
+      throw NotImplementedYet("Not implemented yet");
+    }
+  }
+
+}  // namespace orc

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/RLE.hh
----------------------------------------------------------------------
diff --git a/c++/src/RLE.hh b/c++/src/RLE.hh
new file mode 100644
index 0000000..0a44c95
--- /dev/null
+++ b/c++/src/RLE.hh
@@ -0,0 +1,78 @@
+/**
+ * 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 ORC_RLE_HH
+#define ORC_RLE_HH
+
+#include "Compression.hh"
+
+#include <memory>
+
+namespace orc {
+
+  inline int64_t unZigZag(uint64_t value) {
+    return value >> 1 ^ -(value & 1);
+  }
+
+  class RleDecoder {
+  public:
+    // must be non-inline!
+    virtual ~RleDecoder();
+
+    /**
+     * Seek to a particular spot.
+     */
+    virtual void seek(PositionProvider&) = 0;
+
+    /**
+     * Seek over a given number of values.
+     */
+    virtual void skip(uint64_t numValues) = 0;
+
+    /**
+     * Read a number of values into the batch.
+     * @param data the array to read into
+     * @param numValues the number of values to read
+     * @param notNull If the pointer is null, all values are read. If the
+     *    pointer is not null, positions that are false are skipped.
+     */
+    virtual void next(int64_t* data, uint64_t numValues,
+                      const char* notNull) = 0;
+  };
+
+  enum RleVersion {
+    RleVersion_1,
+    RleVersion_2
+  };
+
+  /**
+   * Create an RLE decoder.
+   * @param input the input stream to read from
+   * @param isSigned true if the number sequence is signed
+   * @param version version of RLE decoding to do
+   * @param pool memory pool to use for allocation
+   */
+  std::unique_ptr<RleDecoder> createRleDecoder
+                      (std::unique_ptr<SeekableInputStream> input,
+                       bool isSigned,
+                       RleVersion version,
+                       MemoryPool& pool);
+
+}  // namespace orc
+
+#endif  // ORC_RLE_HH

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/RLEv1.cc
----------------------------------------------------------------------
diff --git a/c++/src/RLEv1.cc b/c++/src/RLEv1.cc
new file mode 100644
index 0000000..91bb79d
--- /dev/null
+++ b/c++/src/RLEv1.cc
@@ -0,0 +1,190 @@
+/**
+ * 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 "Compression.hh"
+#include "Exceptions.hh"
+#include "RLEv1.hh"
+
+#include <algorithm>
+
+namespace orc {
+
+const uint64_t MINIMUM_REPEAT = 3;
+const uint64_t BASE_128_MASK = 0x7f;
+
+signed char RleDecoderV1::readByte() {
+  if (bufferStart == bufferEnd) {
+    int bufferLength;
+    const void* bufferPointer;
+    if (!inputStream->Next(&bufferPointer, &bufferLength)) {
+      throw ParseError("bad read in readByte");
+    }
+    bufferStart = static_cast<const char*>(bufferPointer);
+    bufferEnd = bufferStart + bufferLength;
+  }
+  return *(bufferStart++);
+}
+
+uint64_t RleDecoderV1::readLong() {
+  uint64_t result = 0;
+  int64_t offset = 0;
+  signed char ch = readByte();
+  if (ch >= 0) {
+    result = static_cast<uint64_t>(ch);
+  } else {
+    result = static_cast<uint64_t>(ch) & BASE_128_MASK;
+    while ((ch = readByte()) < 0) {
+      offset += 7;
+      result |= (static_cast<uint64_t>(ch) & BASE_128_MASK) << offset;
+    }
+    result |= static_cast<uint64_t>(ch) << (offset + 7);
+  }
+  return result;
+}
+
+void RleDecoderV1::skipLongs(uint64_t numValues) {
+  while (numValues > 0) {
+    if (readByte() >= 0) {
+      --numValues;
+    }
+  }
+}
+
+void RleDecoderV1::readHeader() {
+  signed char ch = readByte();
+  if (ch < 0) {
+    remainingValues = static_cast<uint64_t>(-ch);
+    repeating = false;
+  } else {
+    remainingValues = static_cast<uint64_t>(ch) + MINIMUM_REPEAT;
+    repeating = true;
+    delta = readByte();
+    value = isSigned
+        ? unZigZag(readLong())
+        : static_cast<int64_t>(readLong());
+  }
+}
+
+RleDecoderV1::RleDecoderV1(std::unique_ptr<SeekableInputStream> input,
+                           bool hasSigned)
+    : inputStream(std::move(input)),
+      isSigned(hasSigned),
+      remainingValues(0),
+      bufferStart(nullptr),
+      bufferEnd(bufferStart) {
+}
+
+void RleDecoderV1::seek(PositionProvider& location) {
+  // move the input stream
+  inputStream->seek(location);
+  // force a re-read from the stream
+  bufferEnd = bufferStart;
+  // read a new header
+  readHeader();
+  // skip ahead the given number of records
+  skip(location.next());
+}
+
+void RleDecoderV1::skip(uint64_t numValues) {
+  while (numValues > 0) {
+    if (remainingValues == 0) {
+      readHeader();
+    }
+    uint64_t count = std::min(numValues, remainingValues);
+    remainingValues -= count;
+    numValues -= count;
+    if (repeating) {
+      value += delta * static_cast<int64_t>(count);
+    } else {
+      skipLongs(count);
+    }
+  }
+}
+
+void RleDecoderV1::next(int64_t* const data,
+                        const uint64_t numValues,
+                        const char* const notNull) {
+  uint64_t position = 0;
+  // skipNulls()
+  if (notNull) {
+    // Skip over null values.
+    while (position < numValues && !notNull[position]) {
+      ++position;
+    }
+  }
+  while (position < numValues) {
+    // If we are out of values, read more.
+    if (remainingValues == 0) {
+      readHeader();
+    }
+    // How many do we read out of this block?
+    uint64_t count = std::min(numValues - position, remainingValues);
+    uint64_t consumed = 0;
+    if (repeating) {
+      if (notNull) {
+        for (uint64_t i = 0; i < count; ++i) {
+          if (notNull[position + i]) {
+            data[position + i] = value + static_cast<int64_t>(consumed) * delta;
+            consumed += 1;
+          }
+        }
+      } else {
+        for (uint64_t i = 0; i < count; ++i) {
+          data[position + i] = value + static_cast<int64_t>(i) * delta;
+        }
+        consumed = count;
+      }
+      value += static_cast<int64_t>(consumed) * delta;
+    } else {
+      if (notNull) {
+        for (uint64_t i = 0 ; i < count; ++i) {
+          if (notNull[i]) {
+            data[position + i] = isSigned
+                ? unZigZag(readLong())
+                : static_cast<int64_t>(readLong());
+            ++consumed;
+          }
+        }
+      } else {
+        if (isSigned) {
+          for (uint64_t i = 0; i < count; ++i) {
+            data[position + i] = unZigZag(readLong());
+          }
+        } else {
+          for (uint64_t i = 0; i < count; ++i) {
+            data[position + i] = static_cast<int64_t>(readLong());
+          }
+        }
+        consumed = count;
+      }
+    }
+    remainingValues -= consumed;
+    position += count;
+
+    // skipNulls()
+    if (notNull) {
+      // Skip over null values.
+      while (position < numValues && !notNull[position]) {
+        ++position;
+      }
+    }
+  }
+}
+
+}  // namespace orc

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/RLEv1.hh
----------------------------------------------------------------------
diff --git a/c++/src/RLEv1.hh b/c++/src/RLEv1.hh
new file mode 100644
index 0000000..95e50a3
--- /dev/null
+++ b/c++/src/RLEv1.hh
@@ -0,0 +1,70 @@
+/**
+* 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 ORC_RLEV1_HH
+#define ORC_RLEV1_HH
+
+#include "Adaptor.hh"
+#include "RLE.hh"
+
+#include <memory>
+
+namespace orc {
+
+class RleDecoderV1 : public RleDecoder {
+public:
+    RleDecoderV1(std::unique_ptr<SeekableInputStream> input,
+                 bool isSigned);
+
+    /**
+    * Seek to a particular spot.
+    */
+    void seek(PositionProvider&) override;
+
+    /**
+    * Seek over a given number of values.
+    */
+    void skip(uint64_t numValues) override;
+
+    /**
+    * Read a number of values into the batch.
+    */
+    void next(int64_t* data, uint64_t numValues,
+              const char* notNull) override;
+
+private:
+    inline signed char readByte();
+
+    inline void readHeader();
+
+    inline uint64_t readLong();
+
+    inline void skipLongs(uint64_t numValues);
+
+    const std::unique_ptr<SeekableInputStream> inputStream;
+    const bool isSigned;
+    uint64_t remainingValues;
+    int64_t value;
+    const char *bufferStart;
+    const char *bufferEnd;
+    int64_t delta;
+    bool repeating;
+};
+}  // namespace orc
+
+#endif  // ORC_RLEV1_HH

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/RLEv2.cc
----------------------------------------------------------------------
diff --git a/c++/src/RLEv2.cc b/c++/src/RLEv2.cc
new file mode 100644
index 0000000..43428b4
--- /dev/null
+++ b/c++/src/RLEv2.cc
@@ -0,0 +1,482 @@
+/**
+ * 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 "Compression.hh"
+#include "RLEv2.hh"
+
+#define MIN_REPEAT 3
+
+namespace orc {
+
+struct FixedBitSizes {
+  enum FBS {
+    ONE = 0, TWO, THREE, FOUR, FIVE, SIX, SEVEN, EIGHT, NINE, TEN, ELEVEN, TWELVE,
+    THIRTEEN, FOURTEEN, FIFTEEN, SIXTEEN, SEVENTEEN, EIGHTEEN, NINETEEN,
+    TWENTY, TWENTYONE, TWENTYTWO, TWENTYTHREE, TWENTYFOUR, TWENTYSIX,
+    TWENTYEIGHT, THIRTY, THIRTYTWO, FORTY, FORTYEIGHT, FIFTYSIX, SIXTYFOUR
+  };
+};
+
+inline uint32_t decodeBitWidth(uint32_t n) {
+  if (n >= FixedBitSizes::ONE &&
+      n <= FixedBitSizes::TWENTYFOUR) {
+    return n + 1;
+  } else if (n == FixedBitSizes::TWENTYSIX) {
+    return 26;
+  } else if (n == FixedBitSizes::TWENTYEIGHT) {
+    return 28;
+  } else if (n == FixedBitSizes::THIRTY) {
+    return 30;
+  } else if (n == FixedBitSizes::THIRTYTWO) {
+    return 32;
+  } else if (n == FixedBitSizes::FORTY) {
+    return 40;
+  } else if (n == FixedBitSizes::FORTYEIGHT) {
+    return 48;
+  } else if (n == FixedBitSizes::FIFTYSIX) {
+    return 56;
+  } else {
+    return 64;
+  }
+}
+
+inline uint32_t getClosestFixedBits(uint32_t n) {
+  if (n == 0) {
+    return 1;
+  }
+
+  if (n >= 1 && n <= 24) {
+    return n;
+  } else if (n > 24 && n <= 26) {
+    return 26;
+  } else if (n > 26 && n <= 28) {
+    return 28;
+  } else if (n > 28 && n <= 30) {
+    return 30;
+  } else if (n > 30 && n <= 32) {
+    return 32;
+  } else if (n > 32 && n <= 40) {
+    return 40;
+  } else if (n > 40 && n <= 48) {
+    return 48;
+  } else if (n > 48 && n <= 56) {
+    return 56;
+  } else {
+    return 64;
+  }
+}
+
+int64_t RleDecoderV2::readLongBE(uint64_t bsz) {
+  int64_t ret = 0, val;
+  uint64_t n = bsz;
+  while (n > 0) {
+    n--;
+    val = readByte();
+    ret |= (val << (n * 8));
+  }
+  return ret;
+}
+
+inline int64_t RleDecoderV2::readVslong() {
+  return unZigZag(readVulong());
+}
+
+uint64_t RleDecoderV2::readVulong() {
+  uint64_t ret = 0, b;
+  uint64_t offset = 0;
+  do {
+    b = readByte();
+    ret |= (0x7f & b) << offset;
+    offset += 7;
+  } while (b >= 0x80);
+  return ret;
+}
+
+RleDecoderV2::RleDecoderV2(std::unique_ptr<SeekableInputStream> input,
+                           bool _isSigned, MemoryPool& pool
+                           ): inputStream(std::move(input)),
+                              isSigned(_isSigned),
+                              firstByte(0),
+                              runLength(0),
+                              runRead(0),
+                              bufferStart(nullptr),
+                              bufferEnd(bufferStart),
+                              deltaBase(0),
+                              byteSize(0),
+                              firstValue(0),
+                              prevValue(0),
+                              bitSize(0),
+                              bitsLeft(0),
+                              curByte(0),
+                              patchBitSize(0),
+                              base(0),
+                              curGap(0),
+                              patchMask(0),
+                              actualGap(0),
+                              unpacked(pool, 0),
+                              unpackedPatch(pool, 0) {
+  // PASS
+}
+
+void RleDecoderV2::seek(PositionProvider& location) {
+  // move the input stream
+  inputStream->seek(location);
+  // clear state
+  bufferEnd = bufferStart = 0;
+  runRead = runLength = 0;
+  // skip ahead the given number of records
+  skip(location.next());
+}
+
+void RleDecoderV2::skip(uint64_t numValues) {
+  // simple for now, until perf tests indicate something encoding specific is
+  // needed
+  const uint64_t N = 64;
+  int64_t dummy[N];
+
+  while (numValues) {
+    uint64_t nRead = std::min(N, numValues);
+    next(dummy, nRead, nullptr);
+    numValues -= nRead;
+  }
+}
+
+void RleDecoderV2::next(int64_t* const data,
+                        const uint64_t numValues,
+                        const char* const notNull) {
+  uint64_t nRead = 0;
+
+  while (nRead < numValues) {
+    // Skip any nulls before attempting to read first byte.
+    while (notNull && !notNull[nRead]) {
+      if (++nRead == numValues) {
+        return; // ended with null values
+      }
+    }
+
+    if (runRead == runLength) {
+      resetRun();
+      firstByte = readByte();
+    }
+
+    uint64_t offset = nRead, length = numValues - nRead;
+
+    EncodingType enc = static_cast<EncodingType>
+        ((firstByte >> 6) & 0x03);
+    switch(static_cast<int64_t>(enc)) {
+    case SHORT_REPEAT:
+      nRead += nextShortRepeats(data, offset, length, notNull);
+      break;
+    case DIRECT:
+      nRead += nextDirect(data, offset, length, notNull);
+      break;
+    case PATCHED_BASE:
+      nRead += nextPatched(data, offset, length, notNull);
+      break;
+    case DELTA:
+      nRead += nextDelta(data, offset, length, notNull);
+      break;
+    default:
+      throw ParseError("unknown encoding");
+    }
+  }
+}
+
+uint64_t RleDecoderV2::nextShortRepeats(int64_t* const data,
+                                        uint64_t offset,
+                                        uint64_t numValues,
+                                        const char* const notNull) {
+  if (runRead == runLength) {
+    // extract the number of fixed bytes
+    byteSize = (firstByte >> 3) & 0x07;
+    byteSize += 1;
+
+    runLength = firstByte & 0x07;
+    // run lengths values are stored only after MIN_REPEAT value is met
+    runLength += MIN_REPEAT;
+    runRead = 0;
+
+    // read the repeated value which is store using fixed bytes
+    firstValue = readLongBE(byteSize);
+
+    if (isSigned) {
+      firstValue = unZigZag(static_cast<uint64_t>(firstValue));
+    }
+  }
+
+  uint64_t nRead = std::min(runLength - runRead, numValues);
+
+  if (notNull) {
+    for(uint64_t pos = offset; pos < offset + nRead; ++pos) {
+      if (notNull[pos]) {
+        data[pos] = firstValue;
+        ++runRead;
+      }
+    }
+  } else {
+    for(uint64_t pos = offset; pos < offset + nRead; ++pos) {
+      data[pos] = firstValue;
+      ++runRead;
+    }
+  }
+
+  return nRead;
+}
+
+uint64_t RleDecoderV2::nextDirect(int64_t* const data,
+                                  uint64_t offset,
+                                  uint64_t numValues,
+                                  const char* const notNull) {
+  if (runRead == runLength) {
+    // extract the number of fixed bits
+    unsigned char fbo = (firstByte >> 1) & 0x1f;
+    bitSize = decodeBitWidth(fbo);
+
+    // extract the run length
+    runLength = static_cast<uint64_t>(firstByte & 0x01) << 8;
+    runLength |= readByte();
+    // runs are one off
+    runLength += 1;
+    runRead = 0;
+  }
+
+  uint64_t nRead = std::min(runLength - runRead, numValues);
+
+  runRead += readLongs(data, offset, nRead, bitSize, notNull);
+
+  if (isSigned) {
+    if (notNull) {
+      for (uint64_t pos = offset; pos < offset + nRead; ++pos) {
+        if (notNull[pos]) {
+          data[pos] = unZigZag(static_cast<uint64_t>(data[pos]));
+        }
+      }
+    } else {
+      for (uint64_t pos = offset; pos < offset + nRead; ++pos) {
+        data[pos] = unZigZag(static_cast<uint64_t>(data[pos]));
+      }
+    }
+  }
+
+  return nRead;
+}
+
+uint64_t RleDecoderV2::nextPatched(int64_t* const data,
+                                   uint64_t offset,
+                                   uint64_t numValues,
+                                   const char* const notNull) {
+  if (runRead == runLength) {
+    // extract the number of fixed bits
+    unsigned char fbo = (firstByte >> 1) & 0x1f;
+    bitSize = decodeBitWidth(fbo);
+
+    // extract the run length
+    runLength = static_cast<uint64_t>(firstByte & 0x01) << 8;
+    runLength |= readByte();
+    // runs are one off
+    runLength += 1;
+    runRead = 0;
+
+    // extract the number of bytes occupied by base
+    uint64_t thirdByte = readByte();
+    byteSize = (thirdByte >> 5) & 0x07;
+    // base width is one off
+    byteSize += 1;
+
+    // extract patch width
+    uint32_t pwo = thirdByte & 0x1f;
+    patchBitSize = decodeBitWidth(pwo);
+
+    // read fourth byte and extract patch gap width
+    uint64_t fourthByte = readByte();
+    uint32_t pgw = (fourthByte >> 5) & 0x07;
+    // patch gap width is one off
+    pgw += 1;
+
+    // extract the length of the patch list
+    size_t pl = fourthByte & 0x1f;
+    if (pl == 0) {
+      throw ParseError("Corrupt PATCHED_BASE encoded data (pl==0)!");
+    }
+
+    // read the next base width number of bytes to extract base value
+    base = readLongBE(byteSize);
+    int64_t mask = (static_cast<int64_t>(1) << ((byteSize * 8) - 1));
+    // if mask of base value is 1 then base is negative value else positive
+    if ((base & mask) != 0) {
+      base = base & ~mask;
+      base = -base;
+    }
+
+    // TODO: something more efficient than resize
+    unpacked.resize(runLength);
+    unpackedIdx = 0;
+    readLongs(unpacked.data(), 0, runLength, bitSize);
+    // any remaining bits are thrown out
+    resetReadLongs();
+
+    // TODO: something more efficient than resize
+    unpackedPatch.resize(pl);
+    patchIdx = 0;
+    // TODO: Skip corrupt?
+    //    if ((patchBitSize + pgw) > 64 && !skipCorrupt) {
+    if ((patchBitSize + pgw) > 64) {
+      throw ParseError("Corrupt PATCHED_BASE encoded data "
+                       "(patchBitSize + pgw > 64)!");
+    }
+    uint32_t cfb = getClosestFixedBits(patchBitSize + pgw);
+    readLongs(unpackedPatch.data(), 0, pl, cfb);
+    // any remaining bits are thrown out
+    resetReadLongs();
+
+    // apply the patch directly when decoding the packed data
+    patchMask = ((static_cast<int64_t>(1) << patchBitSize) - 1);
+
+    adjustGapAndPatch();
+  }
+
+  uint64_t nRead = std::min(runLength - runRead, numValues);
+
+  for(uint64_t pos = offset; pos < offset + nRead; ++pos) {
+    // skip null positions
+    if (notNull && !notNull[pos]) {
+      continue;
+    }
+    if (static_cast<int64_t>(unpackedIdx) != actualGap) {
+      // no patching required. add base to unpacked value to get final value
+      data[pos] = base + unpacked[unpackedIdx];
+    } else {
+      // extract the patch value
+      int64_t patchedVal = unpacked[unpackedIdx] | (curPatch << bitSize);
+
+      // add base to patched value
+      data[pos] = base + patchedVal;
+
+      // increment the patch to point to next entry in patch list
+      ++patchIdx;
+
+      if (patchIdx < unpackedPatch.size()) {
+        adjustGapAndPatch();
+
+        // next gap is relative to the current gap
+        actualGap += unpackedIdx;
+      }
+    }
+
+    ++runRead;
+    ++unpackedIdx;
+  }
+
+  return nRead;
+}
+
+uint64_t RleDecoderV2::nextDelta(int64_t* const data,
+                                 uint64_t offset,
+                                 uint64_t numValues,
+                                 const char* const notNull) {
+  if (runRead == runLength) {
+    // extract the number of fixed bits
+    unsigned char fbo = (firstByte >> 1) & 0x1f;
+    if (fbo != 0) {
+      bitSize = decodeBitWidth(fbo);
+    } else {
+      bitSize = 0;
+    }
+
+    // extract the run length
+    runLength = static_cast<uint64_t>(firstByte & 0x01) << 8;
+    runLength |= readByte();
+    ++runLength; // account for first value
+    runRead = deltaBase = 0;
+
+    // read the first value stored as vint
+    if (isSigned) {
+      firstValue = static_cast<int64_t>(readVslong());
+    } else {
+      firstValue = static_cast<int64_t>(readVulong());
+    }
+
+    prevValue = firstValue;
+
+    // read the fixed delta value stored as vint (deltas can be negative even
+    // if all number are positive)
+    deltaBase = static_cast<int64_t>(readVslong());
+  }
+
+  uint64_t nRead = std::min(runLength - runRead, numValues);
+
+  uint64_t pos = offset;
+  for ( ; pos < offset + nRead; ++pos) {
+    // skip null positions
+    if (!notNull || notNull[pos]) break;
+  }
+  if (runRead == 0 && pos < offset + nRead) {
+    data[pos++] = firstValue;
+    ++runRead;
+  }
+
+  if (bitSize == 0) {
+    // add fixed deltas to adjacent values
+    for ( ; pos < offset + nRead; ++pos) {
+      // skip null positions
+      if (notNull && !notNull[pos]) {
+        continue;
+      }
+      prevValue = data[pos] = prevValue + deltaBase;
+      ++runRead;
+    }
+  } else {
+    for ( ; pos < offset + nRead; ++pos) {
+      // skip null positions
+      if (!notNull || notNull[pos]) break;
+    }
+    if (runRead < 2 && pos < offset + nRead) {
+      // add delta base and first value
+      prevValue = data[pos++] = firstValue + deltaBase;
+      ++runRead;
+    }
+
+    // write the unpacked values, add it to previous value and store final
+    // value to result buffer. if the delta base value is negative then it
+    // is a decreasing sequence else an increasing sequence
+    uint64_t remaining = (offset + nRead) - pos;
+    runRead += readLongs(data, pos, remaining, bitSize, notNull);
+
+    if (deltaBase < 0) {
+      for ( ; pos < offset + nRead; ++pos) {
+        // skip null positions
+        if (notNull && !notNull[pos]) {
+          continue;
+        }
+        prevValue = data[pos] = prevValue - data[pos];
+      }
+    } else {
+      for ( ; pos < offset + nRead; ++pos) {
+        // skip null positions
+        if (notNull && !notNull[pos]) {
+          continue;
+        }
+        prevValue = data[pos] = prevValue + data[pos];
+      }
+    }
+  }
+  return nRead;
+}
+
+}  // namespace orc

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/RLEv2.hh
----------------------------------------------------------------------
diff --git a/c++/src/RLEv2.hh b/c++/src/RLEv2.hh
new file mode 100644
index 0000000..2923009
--- /dev/null
+++ b/c++/src/RLEv2.hh
@@ -0,0 +1,175 @@
+/**
+* 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 ORC_RLEV2_HH
+#define ORC_RLEV2_HH
+
+#include "Adaptor.hh"
+#include "Exceptions.hh"
+#include "RLE.hh"
+
+#include <vector>
+
+namespace orc {
+
+class RleDecoderV2 : public RleDecoder {
+public:
+
+  enum EncodingType { SHORT_REPEAT=0, DIRECT=1, PATCHED_BASE=2, DELTA=3 };
+
+  RleDecoderV2(std::unique_ptr<SeekableInputStream> input,
+               bool isSigned, MemoryPool& pool);
+
+  /**
+  * Seek to a particular spot.
+  */
+  void seek(PositionProvider&) override;
+
+  /**
+  * Seek over a given number of values.
+  */
+  void skip(uint64_t numValues) override;
+
+  /**
+  * Read a number of values into the batch.
+  */
+  void next(int64_t* data, uint64_t numValues,
+            const char* notNull) override;
+
+private:
+
+  // Used by PATCHED_BASE
+  void adjustGapAndPatch() {
+    curGap = static_cast<uint64_t>(unpackedPatch[patchIdx]) >>
+      patchBitSize;
+    curPatch = unpackedPatch[patchIdx] & patchMask;
+    actualGap = 0;
+
+    // special case: gap is >255 then patch value will be 0.
+    // if gap is <=255 then patch value cannot be 0
+    while (curGap == 255 && curPatch == 0) {
+      actualGap += 255;
+      ++patchIdx;
+      curGap = static_cast<uint64_t>(unpackedPatch[patchIdx]) >>
+        patchBitSize;
+      curPatch = unpackedPatch[patchIdx] & patchMask;
+    }
+    // add the left over gap
+    actualGap += curGap;
+  }
+
+  void resetReadLongs() {
+    bitsLeft = 0;
+    curByte = 0;
+  }
+
+  void resetRun() {
+    resetReadLongs();
+    bitSize = 0;
+  }
+
+  unsigned char readByte() {
+  if (bufferStart == bufferEnd) {
+    int bufferLength;
+    const void* bufferPointer;
+    if (!inputStream->Next(&bufferPointer, &bufferLength)) {
+      throw ParseError("bad read in RleDecoderV2::readByte");
+    }
+    bufferStart = static_cast<const char*>(bufferPointer);
+    bufferEnd = bufferStart + bufferLength;
+  }
+
+  unsigned char result = static_cast<unsigned char>(*bufferStart++);
+  return result;
+}
+
+  int64_t readLongBE(uint64_t bsz);
+  int64_t readVslong();
+  uint64_t readVulong();
+  uint64_t readLongs(int64_t *data, uint64_t offset, uint64_t len,
+                     uint64_t fb, const char* notNull = nullptr) {
+  uint64_t ret = 0;
+
+  // TODO: unroll to improve performance
+  for(uint64_t i = offset; i < (offset + len); i++) {
+    // skip null positions
+    if (notNull && !notNull[i]) {
+      continue;
+    }
+    uint64_t result = 0;
+    uint64_t bitsLeftToRead = fb;
+    while (bitsLeftToRead > bitsLeft) {
+      result <<= bitsLeft;
+      result |= curByte & ((1 << bitsLeft) - 1);
+      bitsLeftToRead -= bitsLeft;
+      curByte = readByte();
+      bitsLeft = 8;
+    }
+
+    // handle the left over bits
+    if (bitsLeftToRead > 0) {
+      result <<= bitsLeftToRead;
+      bitsLeft -= static_cast<uint32_t>(bitsLeftToRead);
+      result |= (curByte >> bitsLeft) & ((1 << bitsLeftToRead) - 1);
+    }
+    data[i] = static_cast<int64_t>(result);
+    ++ret;
+  }
+
+  return ret;
+}
+
+
+  uint64_t nextShortRepeats(int64_t* data, uint64_t offset, uint64_t numValues,
+                            const char* notNull);
+  uint64_t nextDirect(int64_t* data, uint64_t offset, uint64_t numValues,
+                      const char* notNull);
+  uint64_t nextPatched(int64_t* data, uint64_t offset, uint64_t numValues,
+                       const char* notNull);
+  uint64_t nextDelta(int64_t* data, uint64_t offset, uint64_t numValues,
+                     const char* notNull);
+
+  const std::unique_ptr<SeekableInputStream> inputStream;
+  const bool isSigned;
+
+  unsigned char firstByte;
+  uint64_t runLength;
+  uint64_t runRead;
+  const char *bufferStart;
+  const char *bufferEnd;
+  int64_t deltaBase; // Used by DELTA
+  uint64_t byteSize; // Used by SHORT_REPEAT and PATCHED_BASE
+  int64_t firstValue; // Used by SHORT_REPEAT and DELTA
+  int64_t prevValue; // Used by DELTA
+  uint32_t bitSize; // Used by DIRECT, PATCHED_BASE and DELTA
+  uint32_t bitsLeft; // Used by anything that uses readLongs
+  uint32_t curByte; // Used by anything that uses readLongs
+  uint32_t patchBitSize; // Used by PATCHED_BASE
+  uint64_t unpackedIdx; // Used by PATCHED_BASE
+  uint64_t patchIdx; // Used by PATCHED_BASE
+  int64_t base; // Used by PATCHED_BASE
+  uint64_t curGap; // Used by PATCHED_BASE
+  int64_t curPatch; // Used by PATCHED_BASE
+  int64_t patchMask; // Used by PATCHED_BASE
+  int64_t actualGap; // Used by PATCHED_BASE
+  DataBuffer<int64_t> unpacked; // Used by PATCHED_BASE
+  DataBuffer<int64_t> unpackedPatch; // Used by PATCHED_BASE
+};
+}  // namespace orc
+
+#endif  // ORC_RLEV2_HH


[04/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools-c++/test/TestReader.cc
----------------------------------------------------------------------
diff --git a/tools-c++/test/TestReader.cc b/tools-c++/test/TestReader.cc
deleted file mode 100644
index 4f82d8a..0000000
--- a/tools-c++/test/TestReader.cc
+++ /dev/null
@@ -1,2950 +0,0 @@
-/**
- * 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 "orc/Adaptor.hh"
-
-#include "gzip.hh"
-#include "orc/ColumnPrinter.hh"
-#include "orc/OrcFile.hh"
-#include "ToolTest.hh"
-
-#include "wrap/gmock.h"
-#include "wrap/gtest-wrapper.h"
-
-#include <sstream>
-
-#ifdef __clang__
-  DIAGNOSTIC_IGNORE("-Wmissing-variable-declarations")
-#endif
-
-namespace orc {
-
-  class OrcFileDescription {
-  public:
-    std::string filename;
-    std::string json;
-    std::string typeString;
-    std::string formatVersion;
-    uint64_t rowCount;
-    uint64_t contentLength;
-    uint64_t stripeCount;
-    CompressionKind compression;
-    size_t compressionSize;
-    uint64_t rowIndexStride;
-    std::map<std::string, std::string> userMeta;
-
-    OrcFileDescription(const std::string& _filename,
-                       const std::string& _json,
-                       const std::string& _typeString,
-                       const std::string& _version,
-                       uint64_t _rowCount,
-                       uint64_t _contentLength,
-                       uint64_t _stripeCount,
-                       CompressionKind _compression,
-                       size_t _compressionSize,
-                       uint64_t _rowIndexStride,
-                       const std::map<std::string, std::string>& _meta
-                       ): filename(_filename),
-                          json(_json),
-                          typeString(_typeString),
-                          formatVersion(_version),
-                          rowCount(_rowCount),
-                          contentLength(_contentLength),
-                          stripeCount(_stripeCount),
-                          compression(_compression),
-                          compressionSize(_compressionSize),
-                          rowIndexStride(_rowIndexStride),
-                          userMeta(_meta) {
-      // PASS
-    }
-
-    friend std::ostream& operator<< (std::ostream& stream,
-                                     OrcFileDescription const& obj);
-  };
-
-  std::ostream& operator<< (std::ostream& stream,
-                            OrcFileDescription const& obj) {
-    stream << obj.filename;
-    return stream;
-  }
-
-  class MatchTest: public testing::TestWithParam<OrcFileDescription> {
-  public:
-    virtual ~MatchTest();
-
-    std::string getFilename() {
-      std::ostringstream filename;
-      filename << exampleDirectory << "/" << GetParam().filename;
-      return filename.str();
-    }
-
-    std::string getJsonFilename() {
-      std::ostringstream filename;
-      filename << exampleDirectory << "/expected/" << GetParam().json;
-      return filename.str();
-    }
-  };
-
-  MatchTest::~MatchTest() {
-    // PASS
-  }
-
-  TEST_P(MatchTest, Metadata) {
-    orc::ReaderOptions opts;
-    std::unique_ptr<Reader> reader =
-      createReader(readLocalFile(getFilename()), opts);
-
-    EXPECT_EQ(GetParam().compression, reader->getCompression());
-    EXPECT_EQ(GetParam().compressionSize, reader->getCompressionSize());
-    EXPECT_EQ(GetParam().stripeCount, reader->getNumberOfStripes());
-    EXPECT_EQ(GetParam().rowCount, reader->getNumberOfRows());
-    EXPECT_EQ(GetParam().rowIndexStride, reader->getRowIndexStride());
-    EXPECT_EQ(GetParam().contentLength, reader->getContentLength());
-    EXPECT_EQ(GetParam().formatVersion, reader->getFormatVersion());
-    EXPECT_EQ(getFilename(), reader->getStreamName());
-    EXPECT_EQ(GetParam().userMeta.size(), reader->getMetadataKeys().size());
-    for(std::map<std::string, std::string>::const_iterator itr =
-          GetParam().userMeta.begin();
-        itr != GetParam().userMeta.end();
-        ++itr) {
-      ASSERT_EQ(true, reader->hasMetadataValue(itr->first));
-      std::string val = reader->getMetadataValue(itr->first);
-      EXPECT_EQ(itr->second, val);
-    }
-    EXPECT_EQ(true, !reader->hasMetadataValue("foo"));
-    EXPECT_EQ(18446744073709551615UL, reader->getRowNumber());
-
-    EXPECT_EQ(GetParam().typeString, reader->getType().toString());
-  }
-
-  std::string getOutput(FILE* outputFile) {
-    size_t posn = static_cast<size_t>(ftell(outputFile));
-    rewind(outputFile);
-    char *buffer = new char[posn];
-    size_t sizeRead = fread(buffer, 1, posn, outputFile);
-    if (sizeRead != posn) {
-      throw std::runtime_error("Bad read");
-    }
-    rewind(outputFile);
-    return std::string(buffer, posn);
-  }
-
-  TEST_P(MatchTest, Contents) {
-    orc::ReaderOptions opts;
-    std::unique_ptr<Reader> reader =
-      createReader(readLocalFile(getFilename()), opts);
-    unsigned long rowCount = 0;
-    std::unique_ptr<ColumnVectorBatch> batch = reader->createRowBatch(1024);
-    std::string line;
-    std::unique_ptr<orc::ColumnPrinter> printer =
-      orc::createColumnPrinter(line, reader->getType());
-    GzipTextReader expected(getJsonFilename());
-    std::string expectedLine;
-    while (reader->next(*batch)) {
-      EXPECT_EQ(rowCount, reader->getRowNumber());
-      printer->reset(*batch);
-      for(size_t i=0; i < batch->numElements; ++i) {
-        ASSERT_EQ(true, expected.nextLine(expectedLine));
-        line.clear();
-        printer->printRow(i);
-        EXPECT_EQ(expectedLine, line)
-          << "wrong output at row " << (rowCount + i);
-      }
-      rowCount += batch->numElements;
-    }
-    EXPECT_EQ(GetParam().rowCount, rowCount);
-    EXPECT_EQ(GetParam().rowCount, reader->getRowNumber());
-  }
-
-  std::map<std::string, std::string> makeMetadata();
-
-  INSTANTIATE_TEST_CASE_P(TestReader, MatchTest,
-    testing::Values(
-                    OrcFileDescription("TestOrcFile.columnProjection.orc",
-                                       "TestOrcFile.columnProjection.jsn.gz",
-                                       "struct<int1:int,string1:string>",
-                                       "0.12",
-                                       21000,
-                                       428406,
-                                       5,
-                                       CompressionKind_NONE,
-                                       262144,
-                                       1000,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("TestOrcFile.emptyFile.orc",
-                                       "TestOrcFile.emptyFile.jsn.gz",
-                                       "struct<boolean1:boolean,byte1:tinyint,"
-                                       "short1:smallint,int1:int,long1:bigint,"
-                                       "float1:float,double1:double,"
-                                       "bytes1:binary,string1:string,"
-                                       "middle:struct<list:array<struct<"
-                                       "int1:int,string1:string>>>,"
-                                       "list:array<struct<int1:int,string1:"
-                                       "string>>,map:map<string,struct<int1:"
-                                       "int,string1:string>>>",
-                                       "0.12",
-                                       0,
-                                       3,
-                                       0,
-                                       CompressionKind_NONE,
-                                       262144,
-                                       10000,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("TestOrcFile.metaData.orc",
-                                       "TestOrcFile.metaData.jsn.gz",
-                                       "struct<boolean1:boolean,byte1:tinyint,"
-                                       "short1:smallint,int1:int,long1:bigint,"
-                                       "float1:float,double1:double,"
-                                       "bytes1:binary,string1:string,"
-                                       "middle:struct<list:array<struct<"
-                                       "int1:int,string1:string>>>,"
-                                       "list:array<struct<int1:int,string1:"
-                                       "string>>,map:map<string,struct<int1:"
-                                       "int,string1:string>>>",
-                                       "0.12",
-                                       1,
-                                       980,
-                                       1,
-                                       CompressionKind_NONE,
-                                       262144,
-                                       10000,
-                                       makeMetadata()),
-                    OrcFileDescription("TestOrcFile.test1.orc",
-                                       "TestOrcFile.test1.jsn.gz",
-                                       "struct<boolean1:boolean,byte1:tinyint,"
-                                       "short1:smallint,int1:int,long1:bigint,"
-                                       "float1:float,double1:double,"
-                                       "bytes1:binary,string1:string,"
-                                       "middle:struct<list:array<struct<"
-                                       "int1:int,string1:string>>>,"
-                                       "list:array<struct<int1:int,string1:"
-                                       "string>>,map:map<string,struct<int1:"
-                                       "int,string1:string>>>",
-                                       "0.12",
-                                       2,
-                                       1015,
-                                       1,
-                                       CompressionKind_ZLIB,
-                                       10000,
-                                       10000,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("TestOrcFile.testMemoryManagementV11"
-                                       ".orc",
-                                       "TestOrcFile.testMemoryManagementV11"
-                                       ".jsn.gz",
-                                       "struct<int1:int,string1:string>",
-                                       "0.11",
-                                       2500,
-                                       18779,
-                                       25,
-                                       CompressionKind_NONE,
-                                       262144,
-                                       0,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("TestOrcFile.testMemoryManagementV12"
-                                       ".orc",
-                                       "TestOrcFile.testMemoryManagementV12"
-                                       ".jsn.gz",
-                                       "struct<int1:int,string1:string>",
-                                       "0.12",
-                                       2500,
-                                       10618,
-                                       4,
-                                       CompressionKind_NONE,
-                                       262144,
-                                       0,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("TestOrcFile.testPredicatePushdown.orc",
-                                       "TestOrcFile.testPredicatePushdown"
-                                       ".jsn.gz",
-                                       "struct<int1:int,string1:string>",
-                                       "0.12",
-                                       3500,
-                                       15529,
-                                       1,
-                                       CompressionKind_NONE,
-                                       262144,
-                                       1000,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("TestOrcFile.testSeek.orc",
-                                       "TestOrcFile.testSeek.jsn.gz",
-                                       "struct<boolean1:boolean,byte1:tinyint,"
-                                       "short1:smallint,int1:int,long1:bigint,"
-                                       "float1:float,double1:double,bytes1:"
-                                       "binary,string1:string,middle:struct<"
-                                       "list:array<struct<int1:int,string1:"
-                                       "string>>>,list:array<struct<int1:int,"
-                                       "string1:string>>,map:map<string,"
-                                       "struct<int1:int,string1:string>>>",
-                                       "0.12",
-                                       32768,
-                                       1896379,
-                                       7,
-                                       CompressionKind_ZLIB,
-                                       65536,
-                                       1000,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("TestOrcFile.testSnappy.orc",
-                                       "TestOrcFile.testSnappy.jsn.gz",
-                                       "struct<int1:int,string1:string>",
-                                       "0.12",
-                                       10000,
-                                       126061,
-                                       2,
-                                       CompressionKind_SNAPPY,
-                                       100,
-                                       10000,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("TestOrcFile.testStringAndBinaryStat"
-                                       "istics.orc",
-                                       "TestOrcFile.testStringAndBinaryStat"
-                                       "istics.jsn.gz",
-                                       "struct<bytes1:binary,string1:string>",
-                                       "0.12",
-                                       4,
-                                       185,
-                                       1,
-                                       CompressionKind_ZLIB,
-                                       10000,
-                                       10000,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("TestOrcFile.testStripeLevelStats.orc",
-                                       "TestOrcFile.testStripeLevelStats"
-                                       ".jsn.gz",
-                                       "struct<int1:int,string1:string>",
-                                       "0.12",
-                                       11000,
-                                       597,
-                                       3,
-                                       CompressionKind_ZLIB,
-                                       10000,
-                                       10000,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("TestOrcFile.testTimestamp.orc",
-                                       "TestOrcFile.testTimestamp.jsn.gz",
-                                       "timestamp",
-                                       "0.11",
-                                       12,
-                                       188,
-                                       1,
-                                       CompressionKind_ZLIB,
-                                       10000,
-                                       10000,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("TestOrcFile.testUnionAndTimestamp.orc",
-                                       "TestOrcFile.testUnionAndTimestamp"
-                                       ".jsn.gz",
-                                       "struct<time:timestamp,union:uniontype"
-                                       "<int,string>,decimal:decimal(38,18)>",
-                                       "0.12",
-                                       5077,
-                                       20906,
-                                       2,
-                                       CompressionKind_NONE,
-                                       262144,
-                                       10000,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("TestOrcFile.testWithoutIndex.orc",
-                                       "TestOrcFile.testWithoutIndex.jsn.gz",
-                                       "struct<int1:int,string1:string>",
-                                       "0.12",
-                                       50000,
-                                       214643,
-                                       10,
-                                       CompressionKind_SNAPPY,
-                                       1000,
-                                       0,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("decimal.orc",
-                                       "decimal.jsn.gz",
-                                       "struct<_col0:decimal(10,5)>",
-                                       "0.12",
-                                       6000,
-                                       16186,
-                                       1,
-                                       CompressionKind_NONE,
-                                       262144,
-                                       10000,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("demo-11-none.orc",
-                                       "demo-12-zlib.jsn.gz",
-                                       ("struct<_col0:int,_col1:string,"
-                                        "_col2:string,_col3:string,_col4:int,"
-                                        "_col5:string,_col6:int,_col7:int,"
-                                        "_col8:int>"),
-                                       "0.11",
-                                       1920800,
-                                       5069718,
-                                       385,
-                                       CompressionKind_NONE,
-                                       262144,
-                                       10000,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("demo-11-zlib.orc",
-                                       "demo-12-zlib.jsn.gz",
-                                       ("struct<_col0:int,_col1:string,"
-                                        "_col2:string,_col3:string,_col4:int,"
-                                        "_col5:string,_col6:int,_col7:int,"
-                                        "_col8:int>"),
-                                       "0.11",
-                                       1920800,
-                                       396823,
-                                       385,
-                                       CompressionKind_ZLIB,
-                                       262144,
-                                       10000,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("demo-12-zlib.orc",
-                                       "demo-12-zlib.jsn.gz",
-                                       ("struct<_col0:int,_col1:string,"
-                                        "_col2:string,_col3:string,_col4:int,"
-                                        "_col5:string,_col6:int,_col7:int,"
-                                        "_col8:int>"),
-                                       "0.12",
-                                       1920800,
-                                       45592,
-                                       1,
-                                       CompressionKind_ZLIB,
-                                       262144,
-                                       10000,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("nulls-at-end-snappy.orc",
-                                       "nulls-at-end-snappy.jsn.gz",
-                                       ("struct<_col0:tinyint,_col1:smallint,"
-                                        "_col2:int,_col3:bigint,_col4:float,"
-                                        "_col5:double,_col6:boolean>"),
-                                       "0.12",
-                                       70000,
-                                       366347,
-                                       1,
-                                       CompressionKind_SNAPPY,
-                                       262144,
-                                       10000,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("orc-file-11-format.orc",
-                                       "orc-file-11-format.jsn.gz",
-                                       ("struct<boolean1:boolean,"
-                                        "byte1:tinyint,short1:smallint,"
-                                        "int1:int,long1:bigint,float1:float,"
-                                        "double1:double,bytes1:binary,"
-                                        "string1:string,middle:struct<list:"
-                                        "array<struct<int1:int,"
-                                        "string1:string>>>,list:array<struct"
-                                        "<int1:int,string1:string>>,map:map"
-                                        "<string,struct<int1:int,string1:"
-                                        "string>>,ts:timestamp,"
-                                        "decimal1:decimal(0,0)>"),
-                                       "0.11",
-                                       7500,
-                                       372542,
-                                       2,
-                                       CompressionKind_NONE,
-                                       262144,
-                                       10000,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("orc_split_elim.orc",
-                                       "orc_split_elim.jsn.gz",
-                                       ("struct<userid:bigint,string1:string,"
-                                        "subtype:double,decimal1:decimal(0,0),"
-                                        "ts:timestamp>"),
-                                       "0.12",
-                                       25000,
-                                       245568,
-                                       5,
-                                       CompressionKind_NONE,
-                                       262144,
-                                       10000,
-                                       std::map<std::string, std::string>()),
-                    OrcFileDescription("over1k_bloom.orc",
-                                       "over1k_bloom.jsn.gz",
-                                       "struct<_col0:tinyint,_col1:smallint,"
-                                       "_col2:int,_col3:bigint,_col4:float,"
-                                       "_col5:double,_col6:boolean,"
-                                       "_col7:string,_col8:timestamp,"
-                                       "_col9:decimal(4,2),_col10:binary>",
-                                       "0.12",
-                                       2098,
-                                       41780,
-                                       2,
-                                       CompressionKind_ZLIB,
-                                       262144,
-                                       10000,
-                                       std::map<std::string, std::string>())
-                    ));
-
-#ifdef HAS_PRE_1970
-INSTANTIATE_TEST_CASE_P(TestReader1900, MatchTest,
-    testing::Values(
-                    OrcFileDescription("TestOrcFile.testDate1900.orc",
-                                       "TestOrcFile.testDate1900.jsn.gz",
-                                       "struct<time:timestamp,date:date>",
-                                       "0.12",
-                                       70000,
-                                       30478,
-                                       8,
-                                       CompressionKind_ZLIB,
-                                       10000,
-                                       10000,
-                                       std::map<std::string, std::string>())
-		    ));
-#endif
-
-#ifdef HAS_POST_2038
-  INSTANTIATE_TEST_CASE_P(TestReader2038, MatchTest,
-    testing::Values(
-                    OrcFileDescription("TestOrcFile.testDate2038.orc",
-                                       "TestOrcFile.testDate2038.jsn.gz",
-                                       "struct<time:timestamp,date:date>",
-                                       "0.12",
-                                       212000,
-                                       94762,
-                                       28,
-                                       CompressionKind_ZLIB,
-                                       10000,
-                                       10000,
-                                       std::map<std::string, std::string>())
-		    ));
-#endif
-
-  TEST(Reader, columnSelectionTest) {
-    ReaderOptions opts;
-    std::list<int64_t> includes;
-    for(int i=1; i < 10; i += 2) {
-      includes.push_back(i);
-    }
-    opts.include(includes);
-    std::ostringstream filename;
-    filename << exampleDirectory << "/demo-11-none.orc";
-    std::unique_ptr<Reader> reader =
-      createReader(readLocalFile(filename.str()), opts);
-
-    EXPECT_EQ(CompressionKind_NONE, reader->getCompression());
-    EXPECT_EQ(256 * 1024, reader->getCompressionSize());
-    EXPECT_EQ(385, reader->getNumberOfStripes());
-    EXPECT_EQ(1920800, reader->getNumberOfRows());
-    EXPECT_EQ(10000, reader->getRowIndexStride());
-    EXPECT_EQ(5069718, reader->getContentLength());
-    EXPECT_EQ(filename.str(), reader->getStreamName());
-    EXPECT_THAT(reader->getMetadataKeys(), testing::IsEmpty());
-    EXPECT_FALSE(reader->hasMetadataValue("foo"));
-    EXPECT_EQ(18446744073709551615UL, reader->getRowNumber());
-
-    const Type& rootType = reader->getType();
-    EXPECT_EQ(0, rootType.getColumnId());
-    EXPECT_EQ(STRUCT, rootType.getKind());
-    ASSERT_EQ(9, rootType.getSubtypeCount());
-    EXPECT_EQ("_col0", rootType.getFieldName(0));
-    EXPECT_EQ("_col1", rootType.getFieldName(1));
-    EXPECT_EQ("_col2", rootType.getFieldName(2));
-    EXPECT_EQ("_col3", rootType.getFieldName(3));
-    EXPECT_EQ("_col4", rootType.getFieldName(4));
-    EXPECT_EQ("_col5", rootType.getFieldName(5));
-    EXPECT_EQ("_col6", rootType.getFieldName(6));
-    EXPECT_EQ("_col7", rootType.getFieldName(7));
-    EXPECT_EQ("_col8", rootType.getFieldName(8));
-    EXPECT_EQ(INT, rootType.getSubtype(0).getKind());
-    EXPECT_EQ(STRING, rootType.getSubtype(1).getKind());
-    EXPECT_EQ(STRING, rootType.getSubtype(2).getKind());
-    EXPECT_EQ(STRING, rootType.getSubtype(3).getKind());
-    EXPECT_EQ(INT, rootType.getSubtype(4).getKind());
-    EXPECT_EQ(STRING, rootType.getSubtype(5).getKind());
-    EXPECT_EQ(INT, rootType.getSubtype(6).getKind());
-    EXPECT_EQ(INT, rootType.getSubtype(7).getKind());
-    EXPECT_EQ(INT, rootType.getSubtype(8).getKind());
-    for(unsigned int i=0; i < 9; ++i) {
-      EXPECT_EQ(i + 1, rootType.getSubtype(i).getColumnId())
-        << "fail on " << i;
-    }
-
-    const std::vector<bool> selected = reader->getSelectedColumns();
-    EXPECT_EQ(true, selected[0]) << "fail on " << 0;
-    for (size_t i = 1; i < 10; ++i) {
-      EXPECT_EQ(i%2==1?true:false, selected[i]) << "fail on " << i;
-    }
-
-    unsigned long rowCount = 0;
-    std::unique_ptr<ColumnVectorBatch> batch = reader->createRowBatch(1024);
-    LongVectorBatch* longVector =
-      dynamic_cast<LongVectorBatch*>
-      (dynamic_cast<StructVectorBatch&>(*batch).fields[0]);
-    int64_t* idCol = longVector->data.data();
-    while (reader->next(*batch)) {
-      EXPECT_EQ(rowCount, reader->getRowNumber());
-      for(unsigned int i=0; i < batch->numElements; ++i) {
-        EXPECT_EQ(rowCount + i + 1, idCol[i]) << "Bad id for " << i;
-      }
-      rowCount += batch->numElements;
-    }
-    EXPECT_EQ(1920800, rowCount);
-    EXPECT_EQ(1920800, reader->getRowNumber());
-  }
-
-  TEST(Reader, stripeInformationTest) {
-    ReaderOptions opts;
-    std::ostringstream filename;
-    filename << exampleDirectory << "/demo-11-none.orc";
-    std::unique_ptr<Reader> reader =
-      createReader(readLocalFile(filename.str()), opts);
-
-    EXPECT_EQ(385, reader->getNumberOfStripes());
-
-    std::unique_ptr<StripeInformation> stripeInfo = reader->getStripe(7);
-    EXPECT_EQ(92143, stripeInfo->getOffset());
-    EXPECT_EQ(13176, stripeInfo->getLength());
-    EXPECT_EQ(234, stripeInfo->getIndexLength());
-    EXPECT_EQ(12673, stripeInfo->getDataLength());
-    EXPECT_EQ(269, stripeInfo->getFooterLength());
-    EXPECT_EQ(5000, stripeInfo->getNumberOfRows());
-  }
-
-  TEST(Reader, readRangeTest) {
-    ReaderOptions fullOpts, lastOpts, oobOpts, offsetOpts;
-    // stripes[N-1]
-    lastOpts.range(5067085, 1);
-    // stripes[N]
-    oobOpts.range(5067086, 4096);
-    // stripes[7, 16]
-    offsetOpts.range(80000, 130722);
-    std::ostringstream filename;
-    filename << exampleDirectory << "/demo-11-none.orc";
-    std::unique_ptr<Reader> fullReader =
-      createReader(readLocalFile(filename.str()), fullOpts);
-    std::unique_ptr<Reader> lastReader =
-      createReader(readLocalFile(filename.str()), lastOpts);
-    std::unique_ptr<Reader> oobReader =
-      createReader(readLocalFile(filename.str()), oobOpts);
-    std::unique_ptr<Reader> offsetReader =
-      createReader(readLocalFile(filename.str()), offsetOpts);
-
-    std::unique_ptr<ColumnVectorBatch> oobBatch =
-      oobReader->createRowBatch(5000);
-    EXPECT_FALSE(oobReader->next(*oobBatch));
-
-    // advance fullReader to align with offsetReader
-    std::unique_ptr<ColumnVectorBatch> fullBatch =
-      fullReader->createRowBatch(5000);
-    for (int i=0; i < 7; ++i) {
-      EXPECT_TRUE(fullReader->next(*fullBatch));
-      EXPECT_EQ(5000, fullBatch->numElements);
-    }
-
-    std::unique_ptr<ColumnVectorBatch> offsetBatch =
-      offsetReader->createRowBatch(5000);
-    LongVectorBatch* fullLongVector =
-      dynamic_cast<LongVectorBatch*>
-      (dynamic_cast<StructVectorBatch&>(*fullBatch).fields[0]);
-    int64_t* fullId = fullLongVector->data.data();
-    LongVectorBatch* offsetLongVector =
-      dynamic_cast<LongVectorBatch*>
-      (dynamic_cast<StructVectorBatch&>(*offsetBatch).fields[0]);
-    int64_t* offsetId = offsetLongVector->data.data();
-    for (int i=7; i < 17; ++i) {
-      EXPECT_TRUE(fullReader->next(*fullBatch));
-      EXPECT_TRUE(offsetReader->next(*offsetBatch));
-      EXPECT_EQ(fullBatch->numElements, offsetBatch->numElements);
-      for (unsigned j=0; j < fullBatch->numElements; ++j) {
-        EXPECT_EQ(fullId[j], offsetId[j]);
-      }
-    }
-    EXPECT_FALSE(offsetReader->next(*offsetBatch));
-
-    // advance fullReader to align with lastReader
-    for (int i=17; i < 384; ++i) {
-      EXPECT_TRUE(fullReader->next(*fullBatch));
-      EXPECT_EQ(5000, fullBatch->numElements);
-    }
-
-    std::unique_ptr<ColumnVectorBatch> lastBatch =
-      lastReader->createRowBatch(5000);
-    LongVectorBatch* lastLongVector =
-      dynamic_cast<LongVectorBatch*>
-      (dynamic_cast<StructVectorBatch&>(*lastBatch).fields[0]);
-    int64_t* lastId = lastLongVector->data.data();
-    EXPECT_TRUE(fullReader->next(*fullBatch));
-    EXPECT_TRUE(lastReader->next(*lastBatch));
-    EXPECT_EQ(fullBatch->numElements, lastBatch->numElements);
-    for (unsigned i=0; i < fullBatch->numElements; ++i) {
-      EXPECT_EQ(fullId[i], lastId[i]);
-    }
-    EXPECT_FALSE(fullReader->next(*fullBatch));
-    EXPECT_FALSE(lastReader->next(*lastBatch));
-  }
-
-TEST(Reader, columnStatistics) {
-  orc::ReaderOptions opts;
-  std::ostringstream filename;
-  filename << exampleDirectory << "/demo-11-none.orc";
-  std::unique_ptr<orc::Reader> reader =
-    orc::createReader(orc::readLocalFile(filename.str()), opts);
-
-  // corrupt stats test
-  EXPECT_EQ(true, reader->hasCorrectStatistics());
-
-  // test column statistics
-  std::unique_ptr<orc::Statistics> stats = reader->getStatistics();
-  EXPECT_EQ(10, stats->getNumberOfColumns());
-
-  // 6th real column, start from 1
-  std::unique_ptr<orc::ColumnStatistics> col_6 =
-    reader->getColumnStatistics(6);
-  const orc::StringColumnStatistics& strStats =
-    dynamic_cast<const orc::StringColumnStatistics&> (*(col_6.get()));
-  EXPECT_EQ("Good", strStats.getMinimum());
-  EXPECT_EQ("Unknown", strStats.getMaximum());
-
-  // 7th real column
-  std::unique_ptr<orc::ColumnStatistics> col_7 =
-    reader->getColumnStatistics(7);
-  const orc::IntegerColumnStatistics& intStats =
-    dynamic_cast<const orc::IntegerColumnStatistics&> (*(col_7.get()));
-  EXPECT_EQ(0, intStats.getMinimum());
-  EXPECT_EQ(6, intStats.getMaximum());
-  EXPECT_EQ(5762400, intStats.getSum());
-}
-
-TEST(Reader, stripeStatistics) {
-  orc::ReaderOptions opts;
-  std::ostringstream filename;
-  filename << exampleDirectory << "/demo-11-none.orc";
-  std::unique_ptr<orc::Reader> reader =
-    orc::createReader(orc::readLocalFile(filename.str()), opts);
-
-  // test stripe statistics
-  EXPECT_EQ(385, reader->getNumberOfStripeStatistics());
-
-  // stripe[384]: 385th stripe, last stripe
-  unsigned long stripeIdx = 384;
-  std::unique_ptr<orc::Statistics> stripeStats =
-    reader->getStripeStatistics(stripeIdx);
-  EXPECT_EQ(10, stripeStats->getNumberOfColumns());
-
-  // 6th real column
-  const orc::StringColumnStatistics* col_6 =
-    dynamic_cast<const orc::StringColumnStatistics*>
-    (stripeStats->getColumnStatistics(6));
-  EXPECT_EQ("Unknown", col_6->getMinimum());
-  EXPECT_EQ("Unknown", col_6->getMaximum());
-
-  // 7th real column
-  const orc::IntegerColumnStatistics* col_7 =
-    dynamic_cast<const orc::IntegerColumnStatistics*>
-    (stripeStats->getColumnStatistics(7));
-  EXPECT_EQ(6, col_7->getMinimum());
-  EXPECT_EQ(6, col_7->getMaximum());
-  EXPECT_EQ(4800, col_7->getSum());
-}
-
-TEST(Reader, corruptStatistics) {
-  orc::ReaderOptions opts;
-  std::ostringstream filename;
-  // read the file has corrupt statistics
-  filename << exampleDirectory << "/orc_split_elim.orc";
-  std::unique_ptr<orc::Reader> reader =
-    orc::createReader(orc::readLocalFile(filename.str()), opts);
-
-  EXPECT_EQ(true, !reader->hasCorrectStatistics());
-
-  // 2nd real column, string
-  std::unique_ptr<orc::ColumnStatistics> col_2 =
-    reader->getColumnStatistics(2);
-  const orc::StringColumnStatistics& strStats =
-    dynamic_cast<const orc::StringColumnStatistics&> (*(col_2.get()));
-  EXPECT_EQ(true, !strStats.hasMinimum());
-  EXPECT_EQ(true, !strStats.hasMaximum());
-
-  // stripe statistics
-  unsigned long stripeIdx = 1;
-  std::unique_ptr<orc::Statistics> stripeStats =
-    reader->getStripeStatistics(stripeIdx);
-
-  // 4th real column, Decimal
-  const orc::DecimalColumnStatistics* col_4 =
-    dynamic_cast<const orc::DecimalColumnStatistics*>
-    (stripeStats->getColumnStatistics(4));
-  EXPECT_EQ(true, !col_4->hasMinimum());
-  EXPECT_EQ(true, !col_4->hasMaximum());
-}
-
-TEST(Reader, noStripeStatistics) {
-  orc::ReaderOptions opts;
-  std::ostringstream filename;
-  // read the file has no stripe statistics
-  filename << exampleDirectory << "/orc-file-11-format.orc";
-  std::unique_ptr<orc::Reader> reader =
-    orc::createReader(orc::readLocalFile(filename.str()), opts);
-
-  EXPECT_EQ(0, reader->getNumberOfStripeStatistics());
-}
-
-TEST(Reader, seekToRow) {
-  /* Test with a regular file */
-  {
-    orc::ReaderOptions opts;
-    std::ostringstream filename;
-    filename << exampleDirectory << "/demo-11-none.orc";
-    std::unique_ptr<orc::Reader> reader =
-        orc::createReader(orc::readLocalFile(filename.str()), opts);
-    EXPECT_EQ(1920800, reader->getNumberOfRows());
-
-    std::unique_ptr<orc::ColumnVectorBatch> batch =
-        reader->createRowBatch(5000); // Stripe size
-    reader->next(*batch);
-    EXPECT_EQ(5000, batch->numElements);
-    EXPECT_EQ(0, reader->getRowNumber());
-
-    // We only load data till the end of the current stripe
-    reader->seekToRow(11000);
-    reader->next(*batch);
-    EXPECT_EQ(4000, batch->numElements);
-    EXPECT_EQ(11000, reader->getRowNumber());
-
-    // We only load data till the end of the current stripe
-    reader->seekToRow(99999);
-    reader->next(*batch);
-    EXPECT_EQ(1, batch->numElements);
-    EXPECT_EQ(99999, reader->getRowNumber());
-
-    // Skip more rows than available
-    reader->seekToRow(1920800);
-    reader->next(*batch);
-    EXPECT_EQ(0, batch->numElements);
-    EXPECT_EQ(1920800, reader->getRowNumber());
-  }
-
-  /* Test with a portion of the file */
-  {
-    orc::ReaderOptions opts;
-    std::ostringstream filename;
-    filename << exampleDirectory << "/demo-11-none.orc";
-    opts.range(13126, 13145);   // Read only the second stripe (rows 5000..9999)
-
-    std::unique_ptr<orc::Reader> reader =
-        orc::createReader(orc::readLocalFile(filename.str()), opts);
-    EXPECT_EQ(1920800, reader->getNumberOfRows());
-
-    std::unique_ptr<orc::ColumnVectorBatch> batch =
-        reader->createRowBatch(5000); // Stripe size
-    reader->next(*batch);
-    EXPECT_EQ(5000, batch->numElements);
-
-    reader->seekToRow(7000);
-    reader->next(*batch);
-    EXPECT_EQ(3000, batch->numElements);
-    EXPECT_EQ(7000, reader->getRowNumber());
-
-    reader->seekToRow(1000);
-    reader->next(*batch);
-    EXPECT_EQ(0, batch->numElements);
-    EXPECT_EQ(10000, reader->getRowNumber());
-
-    reader->seekToRow(11000);
-    reader->next(*batch);
-    EXPECT_EQ(0, batch->numElements);
-    EXPECT_EQ(10000, reader->getRowNumber());
-  }
-
-  /* Test with an empty file */
-  {
-    orc::ReaderOptions opts;
-    std::ostringstream filename;
-    filename << exampleDirectory << "/TestOrcFile.emptyFile.orc";
-    std::unique_ptr<orc::Reader> reader =
-        orc::createReader(orc::readLocalFile(filename.str()), opts);
-    EXPECT_EQ(0, reader->getNumberOfRows());
-
-    std::unique_ptr<orc::ColumnVectorBatch> batch =
-        reader->createRowBatch(5000);
-    reader->next(*batch);
-    EXPECT_EQ(0, batch->numElements);
-
-    reader->seekToRow(0);
-    reader->next(*batch);
-    EXPECT_EQ(0, batch->numElements);
-    EXPECT_EQ(0, reader->getRowNumber());
-
-    reader->seekToRow(1);
-    reader->next(*batch);
-    EXPECT_EQ(0, batch->numElements);
-    EXPECT_EQ(0, reader->getRowNumber());
-  }
-}
-
-TEST(Reader, futureFormatVersion) {
-  std::ostringstream filename;
-  filename << exampleDirectory << "/version1999.orc";
-  orc::ReaderOptions opts;
-  std::ostringstream errorMsg;
-  opts.setErrorStream(errorMsg);
-  std::unique_ptr<orc::Reader> reader =
-    orc::createReader(orc::readLocalFile(filename.str()), opts);
-  EXPECT_EQ(("Warning: ORC file " + filename.str() +
-             " was written in an unknown format version 19.99\n"),
-            errorMsg.str());
-  EXPECT_EQ("19.99", reader->getFormatVersion());
-}
-
-  std::map<std::string, std::string> makeMetadata() {
-    std::map<std::string, std::string> result;
-    result["my.meta"] = "\x01\x02\x03\x04\x05\x06\x07\xff\xfe\x7f\x80";
-    result["clobber"] = "\x05\x07\x0b\x0d\x11\x13";
-    const unsigned char buffer[] = {
-96,180,32,187,56,81,217,212,122,203,147,61,190,112,57,155,246,201,45,163,
-58,240,29,79,183,112,233,140,3,37,244,29,62,186,248,152,109,167,18,200,
-43,205,77,85,75,240,181,64,35,194,155,98,77,233,239,156,47,147,30,252,
-88,15,154,251,8,27,18,225,7,177,232,5,242,180,245,240,241,208,12,45,
-15,98,99,70,112,146,28,80,88,103,255,32,246,168,51,94,152,175,135,37,
-56,85,134,180,31,239,242,5,180,224,90,0,8,35,247,139,95,143,92,2,
-67,156,232,246,122,120,29,144,203,230,191,26,231,242,188,64,164,151,9,160,
-108,14,49,73,155,240,41,105,202,66,210,3,229,102,188,198,150,222,8,250,
-1,2,160,253,46,35,48,176,150,74,187,124,68,48,32,222,28,173,9,191,
-214,56,31,251,148,218,175,187,144,196,237,145,160,97,58,209,220,75,71,3,
-175,132,193,214,59,26,135,105,33,198,213,134,157,97,204,185,142,209,58,230,
-192,154,19,252,145,225,73,34,243,1,207,139,207,147,67,21,166,4,157,47,
-7,217,131,250,169,27,143,78,114,101,236,184,21,167,203,171,193,69,12,183,
-43,60,116,16,119,23,170,36,172,66,242,91,108,103,132,118,125,14,53,70,
-196,247,37,1,145,163,182,170,162,182,77,18,110,85,131,176,76,17,50,89,
-201,72,225,208,179,155,185,86,12,213,64,155,110,202,254,219,200,172,175,238,
-167,77,183,248,90,223,148,190,154,133,161,221,75,3,170,136,131,29,210,156,
-64,120,129,11,58,40,210,45,102,128,182,79,203,177,178,55,194,68,18,52,
-206,171,191,218,216,124,49,21,72,246,121,2,116,185,46,106,89,29,58,177,
-166,11,115,64,11,196,116,197,45,60,188,242,251,174,114,182,230,212,159,176,
-177,133,19,54,250,44,84,12,223,191,120,200,219,73,44,101,231,91,1,242,
-86,10,157,196,86,254,164,3,66,134,86,158,48,134,234,100,151,36,149,156,
-68,8,146,218,235,114,76,6,229,19,58,201,170,148,16,238,186,45,84,254,
-138,253,248,80,125,33,19,226,2,106,147,122,228,57,152,43,206,121,204,36,
-14,38,74,246,205,67,204,48,37,102,107,185,179,127,129,231,20,21,103,173,
-104,201,130,73,128,219,236,205,20,219,32,168,112,88,1,113,21,182,16,82,
-57,102,75,118,131,96,156,24,178,230,82,218,235,107,179,120,187,63,120,128,
-92,187,129,182,154,249,239,253,13,66,111,145,188,18,34,248,23,137,195,144,
-219,40,101,90,30,2,29,96,158,157,25,12,222,85,168,201,35,237,85,18,
-244,72,205,0,24,151,115,7,95,78,212,28,87,207,209,123,139,190,52,2,
-21,185,209,248,226,186,91,123,35,200,54,253,59,89,143,89,220,21,119,196,
-157,151,25,184,177,147,91,128,45,32,152,163,112,27,106,65,71,222,102,86,
-183,253,36,79,11,33,22,190,57,233,40,217,234,84,4,186,183,105,85,13,
-246,140,86,48,227,219,154,2,253,245,209,49,130,27,150,90,189,48,247,209,
-133,115,52,22,177,69,12,237,253,3,174,217,74,188,227,213,6,248,240,0,
-139,40,121,189,39,22,210,78,95,141,155,182,188,127,23,136,50,15,115,77,
-90,104,55,235,130,241,252,79,85,60,247,70,138,139,90,240,208,165,207,223,
-54,19,46,197,250,49,33,156,0,163,227,139,104,148,156,232,107,6,11,38,
-177,188,99,11,39,114,53,250,170,178,143,165,54,168,3,82,2,136,46,127,
-132,245,39,53,8,50,183,129,196,69,203,125,221,220,75,133,165,11,85,51,
-102,209,201,93,140,62,231,96,186,105,70,122,120,4,111,141,220,91,55,180,
-196,21,73,55,1,233,160,82,217,81,160,70,186,207,251,2,21,167,243,244,
-173,73,119,179,108,45,221,194,204,113,207,190,145,114,114,189,43,62,119,155,
-172,71,73,123,222,80,46,60,228,51,229,145,135,85,152,99,68,212,96,24,
-130,253,85,233,201,56,209,85,202,2,218,42,157,135,181,148,16,122,0,1,
-192,42,13,152,157,54,208,67,183,89,5,15,237,166,143,228,253,63,51,193,
-49,111,45,76,60,34,122,56,16,175,204,109,163,7,81,95,98,98,3,176,
-210,97,62,97,194,56,147,104,49,69,75,174,26,166,97,90,176,204,132,43,
-164,94,248,171,60,143,223,88,243,250,139,189,116,106,229,216,246,180,249,228,
-94,165,148,214,32,29,120,148,50,95,204,0,21,223,192,130,110,177,133,10,
-141,63,221,79,208,177,227,165,69,121,76,138,241,231,3,157,67,149,29,249,
-144,163,34,5,177,90,23,157,207,59,205,105,17,141,244,6,237,108,194,224,
-175,115,99,176,73,25,78,173,104,163,90,161,171,115,1,77,71,204,93,209,
-42,92,79,248,134,238,185,189,41,78,155,81,245,102,165,161,60,245,208,105,
-215,200,156,80,249,39,109,202,174,11,23,192,253,242,228,5,151,61,178,48,
-178,91,35,105,32,63,92,31,146,225,101,237,187,27,154,182,4,86,70,58,
-62,234,219,238,252,181,158,168,17,211,164,79,12,22,202,150,251,117,143,135,
-137,184,169,5,166,127,209,45,232,222,164,137,84,95,39,29,140,34,175,77,
-103,199,22,175,142,238,38,204,148,135,22,97,80,99,131,209,10,110,169,151,
-217,77,22,13,211,196,203,240,73,64,176,65,46,195,189,136,228,13,47,11,
-191,118,213,54,140,68,243,158,192,78,111,85,155,134,217,132,158,35,35,64,
-128,51,239,49,161,95,76,188,142,106,237,81,147,97,85,23,213,114,117,58,
-133,96,185,67,8,196,113,114,31,144,76,48,181,159,167,115,30,23,58,76,
-96,47,183,19,234,37,43,194,58,195,128,33,120,49,237,11,142,230,42,181,
-195,150,8,22,31,218,88,209,166,197,104,228,0,114,22,181,229,21,222,136,
-185,163,236,240,158,167,236,21,174,18,105,188,124,184,6,136,39,236,158,185,
-97,185,201,18,108,57,229,44,103,188,64,45,200,197,71,247,94,153,43,226,
-126,159,221,223,62,247,181,89,237,101,57,238,24,83,100,252,113,212,82,2,
-149,177,109,147,207,152,105,10,6,246,175,154,40,85,251,150,130,114,234,69,
-195,0,42,61,185,54,96,131,52,128,205,92,92,127,218,241,171,148,200,158,
-68,126,190,55,105,251,67,90,197,19,234,232,175,17,21,97,215,11,245,4,
-173,94,10,192,235,149,50,70,45,84,95,166,173,12,54,171,19,56,73,242,
-10,75,178,73,237,203,77,225,40,206,97,16,39,189,165,91,52,80,236,57,
-153,127,42,236,57,110,219,35,139,189,122,217,84,219,136,154,107,83,56,173,
-5,174,77,186,194,170,21,139,112,8,202,77,40,135,137,120,197,202,23,160,
-75,201,27,31,45,183,41,9,76,159,235,57,237,128,52,122,241,222,232,63,
-152,60,185,23,134,45,12,10,144,157,235,181,97,242,249,234,35,237,35,111,
-102,44,61,28,2,194,192,209,253,239,139,149,236,194,193,154,84,226,118,213,
-190,61,24,172,239,71,191,90,69,164,55,115,196,127,160,116,138,34,53,88,
-127,217,14,187,112,14,247,68,167,236,7,143,216,246,193,190,169,191,249,242,
-170,40,199,52,36,121,132,5,111,170,71,38,234,210,164,180,106,131,157,235,
-135,101,71,54,74,177,12,176,90,244,49,71,234,7,173,234,121,117,24,120,
-192,104,16,9,255,117,216,51,230,219,245,49,113,2,236,60,228,42,117,147,
-95,52,171,205,163,61,9,247,106,65,163,38,46,180,237,84,86,53,174,234,
-50,156,96,35,214,175,158,104,90,191,232,24,42,224,166,12,245,111,215,209,
-210,219,213,190,144,251,127,171,220,34,47,75,98,151,203,109,154,251,166,62,
-196,192,221,122,192,24,69,112,6,60,96,212,62,8,196,49,95,38,31,138,
-79,6,22,154,205,131,155,149,149,79,202,223,30,96,238,152,60,190,92,33,
-128,146,215,95,114,177,108,178,58,133,107,0,196,195,152,152,142,159,131,176,
-21,252,231,249,201,37,184,182,74,190,228,38,14,36,186,17,228,27,252,246,
-100,82,220,128,34,18,136,206,4,101,253,176,91,18,28,220,8,250,1,205,
-172,178,200,244,226,221,187,184,63,232,49,140,65,194,109,87,165,129,63,171,
-82,79,82,16,25,15,115,201,132,189,106,89,185,207,66,76,50,118,89,133,
-226,229,148,205,220,163,208,244,91,210,31,73,224,174,105,177,250,84,120,93,
-201,113,34,31,217,34,21,251,182,8,95,129,95,181,94,0,100,145,189,230,
-170,154,141,156,216,141,204,42,26,119,41,53,199,241,111,89,10,33,60,152,
-44,195,245,177,252,58,73,34,171,176,77,8,200,61,174,60,169,164,145,66,
-138,83,24,22,81,58,5,119,94,133,244,213,213,161,10,104,53,225,167,56,
-166,3,123,47,66,50,93,193,136,94,35,75,206,253,197,124,161,66,100,147,
-123,127,46,98,245,59,32,43,56,171,118,79,240,72,42,95,118,152,19,231,
-234,237,17,60,94,121,128,210,210,100,214,137,25,50,200,151,86,160,221,90,
-103,23,73,227,70,108,96,79,24,33,188,70,12,113,99,214,40,68,77,138,
-101,86,184,171,136,129,41,116,198,65,236,27,218,209,66,36,12,135,133,239,
-177,67,173,116,107,38,20,224,12,177,170,11,189,176,28,218,39,50,27,94,
-148,243,246,100,218,54,132,91,198,112,215,96,27,197,67,199,76,177,184,134,
-95,18,50,161,163,9,7,1,238,86,243,75,23,246,219,8,103,165,180,191,
-160,223,109,201,6,142,215,218,53,110,116,69,105,180,100,152,194,155,193,184,
-229,15,113,192,39,19,100,107,179,161,148,48,231,24,114,74,119,209,4,38,
-114,215,181,62,231,167,45,95,44,127,15,32,170,152,225,230,162,202,16,38,
-165,199,193,164,142,49,108,86,68,131,131,150,145,249,106,214,137,92,226,178,
-211,113,61,216,240,166,104,208,233,142,211,66,88,141,22,144,170,222,199,158,
-153,30,7,60,230,247,159,125,117,11,204,227,17,48,83,106,250,24,134,117,
-96,17,192,218,153,213,79,44,246,213,242,209,117,188,224,123,8,161,185,12,
-194,241,131,199,33,93,231,66,36,244,50,168,241,158,11,215,108,82,226,88,
-80,190,182,109,46,122,93,37,82,19,190,189,254,90,172,51,163,251,87,224,
-91,179,47,148,137,140,235,74,135,183,41,186,58,176,176,251,233,47,27,4,
-99,113,93,212,169,123,208,193,28,118,105,197,159,229,93,196,154,179,36,93,
-213,188,91,219,90,70,8,40,35,119,205,58,158,166,172,69,184,27,117,44,
-115,55,142,64,136,28,63,229,152,254,93,0,244,240,232,89,250,61,200,68,
-43,1,45,198,171,59,196,61,19,17,164,204,118,211,88,30,190,42,243,165,
-124,5,136,201,184,91,53,140,136,167,122,246,165,61,11,5,244,58,181,53,
-22,138,124,158,83,191,218,235,128,178,79,42,68,186,235,11,166,240,210,168,
-23,167,234,248,8,81,196,7,41,37,134,104,20,40,146,186,31,154,241,131,
-30,4,235,121,113,37,178,2,121,209,167,46,221,196,45,37,33,91,137,182,
-189,245,41,218,207,233,36,97,82,5,197,64,127,76,52,57,135,50,247,55,
-161,96,170,49,179,205,86,36,142,13,61,147,102,55,163,207,2,230,123,139,
-73,13,247,80,156,19,243,194,144,140,185,137,191,35,29,59,152,67,116,132,
-68,146,220,248,160,40,197,139,215,213,236,118,195,33,73,94,3,11,200,105,
-154,138,57,37,43,118,116,159,46,94,188,55,10,194,174,63,43,240,68,96,
-113,111,90,196,101,158,183,233,85,44,137,66,52,44,153,145,110,11,80,135,
-60,155,8,224,251,170,179,26,137,225,12,167,100,144,51,150,54,227,77,127,
-200,39,147,89,245,37,207,106,200,65,50,108,42,223,2,171,8,103,14,216,
-129,209,3,4,56,56,61,142,35,253,52,42,165,34,106,158,245,253,62,190,
-171,68,223,116,136,37,166,237,116,66,99,235,159,122,186,99,233,82,177,171,
-124,222,190,95,203,197,67,34,82,56,136,18,62,255,141,240,135,193,244,31,
-86,50,100,78,177,241,176,135,106,83,124,209,117,39,112,238,42,156,84,127,
-173,147,94,4,219,222,84,43,126,46,6,84,26,155,5,209,75,42,38,149,
-29,158,163,43,166,126,74,92,73,193,250,95,170,85,125,10,145,101,124,249,
-102,112,206,197,0,103,21,104,68,139,163,7,92,195,30,194,140,121,217,133,
-194,223,31,38,116,167,7,99,162,140,161,162,95,4,65,171,156,108,46,230,
-55,147,182,156,244,137,173,70,6,21,196,61,192,127,108,164,54,2,232,100,
-6,247,209,72,41,97,177,254,84,37,66,80,3,141,138,74,165,201,116,146,
-216,189,228,34,135,165,38,180,163,171,0,112,231,243,11,131,172,7,147,151,
-230,179,50,44,76,84,84,23,163,21,108,87,43,233,41,225,236,191,213,51,
-111,66,63,127,35,81,204,165,112,35,251,135,223,58,55,189,247,41,194,179,
-24,55,230,64,11,61,193,151,217,230,5,164,211,90,229,239,205,253,147,31,
-37,187,133,95,36,131,4,31,178,152,225,14,159,119,190,29,232,72,79,45,
-203,105,196,40,13,43,152,105,172,124,143,98,29,113,128,166,109,242,243,149,
-106,105,45,182,203,187,244,181,22,206,53,241,84,174,175,47,221,71,9,243,
-187,38,202,15,205,211,124,158,197,209,162,55,210,167,181,184,158,109,194,156,
-77,152,42,7,153,94,180,91,10,113,74,39,167,61,151,123,80,62,172,23,
-12,228,195,68,227,182,147,240,18,55,114,167,41,230,160,192,212,231,209,160,
-219,109,30,48,84,206,26,46,228,72,167,244,117,76,29,42,86,117,28,10,
-148,140,132,176,171,97,199,53,211,72,128,171,159,249,216,161,89,99,28,182,
-106,201,130,200,8,197,236,35,44,114,130,92,131,7,235,245,182,168,28,38,
-119,48,145,200,37,192,159,149,127,135,141,255,234,99,84,213,33,37,44,236,
-39,49,218,171,182,144,100,176,147,101,220,102,55,108,166,131,68,106,30,103,
-69,24,110,98,13,171,136,147,239,55,112,177,78,127,178,56,243,90,68,121,
-109,247,198,235,154,166,151,27,85,186,255,74,95,104,248,201,161,208,172,212,
-246,226,91,209,109,215,178,21,121,173,45,179,214,170,5,197,192,117,97,189,
-180,184,189,139,51,50,63,167,25,230,141,134,141,156,225,118,146,214,241,151,
-119,126,142,113,64,130,45,67,218,185,119,47,66,240,154,154,91,90,167,221,
-11,173,207,21,97,50,205,73,145,192,245,218,146,221,226,73,133,142,162,208,
-174,75,149,56,230,186,214,74,37,130,231,122,176,247,92,113,240,223,79,210,
-129,20,145,189,99,46,11,17,96,215,221,245,20,251,98,150,7,158,110,125,
-22,106,119,215,103,153,83,186,15,181,136,16,61,49,13,113,247,50,63,198,
-242,211,87,108,123,221,143,201,20,183,254,59,98,36,153,175,50,231,77,190,
-101,76,67,13,229,9,85,176,77,230,103,212,62,115,25,26,100,236,52,86,
-252,55,147,193,118,23,102,224,55,200,43,116,238,209,102,214,90,234,17,249,
-219,38,234,97,172,180,53,78,230,243,157,88,182,243,124,56,111,208,137,183,
-64,86,189,213,120,89,207,110,197,75,236,172,10,49,44,158,138,208,173,156,
-41,209,209,119,187,250,122,4,160,165,148,47,60,14,241,154,41,133,139,182,
-16,185,98,213,181,233,95,219,134,240,129,70,235,0,116,80,135,71,203,72,
-16,41,16,6,188,144,103,141,247,123,184,53,107,195,232,96,174,248,91,202,
-53,120,240,48,254,125,217,107,27,123,244,249,249,106,191,83,118,242,142,101,
-46,48,118,1,201,6,64,20,58,179,141,93,144,221,37,13,200,239,106,184,
-126,169,90,51,75,12,181,50,198,156,167,100,238,245,173,167,96,62,61,23,
-144,214,77,175,114,17,223,119,193,40,174,65,55,54,241,184,38,33,41,9,
-40,190,84,41,171,105,165,43,45,105,107,248,93,18,34,199,27,120,192,134,
-248,244,144,151,213,192,155,133,100,227,56,67,191,74,154,142,171,166,151,162,
-110,254,62,63,66,99,181,64,176,113,99,111,143,56,207,237,103,33,16,7,
-13,53,250,160,209,140,106,191,139,205,206,236,38,50,203,40,85,61,145,166,
-184,114,92,178,58,94,142,125,4,117,75,48,224,242,140,193,73,69,37,11,
-187,218,247,92,156,73,43,247,156,134,225,201,160,172,187,122,197,21,136,28,
-69,58,236,219,209,22,20,63,52,92,170,113,253,204,170,220,220,255,17,80,
-149,97,205,127,56,70,202,176,139,98,94,78,198,125,3,16,133,170,192,130,
-68,33,226,79,234,27,97,66,52,250,48,131,177,199,46,244,52,36,6,149,
-255,249,242,77,169,22,47,149,216,137,88,115,80,28,221,15,109,126,84,26,
-184,76,106,53,212,243,178,48,173,65,158,150,39,242,219,173,18,243,14,251,
-87,245,226,67,210,109,237,65,70,1,0,25,192,14,254,217,215,156,173,142,
-157,152,241,126,27,192,79,56,172,116,103,148,198,244,100,243,103,191,91,14,
-136,109,61,203,101,255,37,151,6,0,41,137,102,154,210,249,139,134,34,168,
-34,221,130,45,156,137,215,129,83,196,153,129,27,255,56,221,7,58,209,175,
-163,187,252,203,76,56,145,248,55,180,174,173,167,202,163,63,130,177,193,121,
-251,73,20,207,59,107,181,126,201,250,113,80,97,121,245,107,246,123,13,108,
-116,242,194,115,111,196,154,44,216,56,32,49,104,216,6,73,247,54,90,0,
-19,115,15,169,233,104,87,235,241,175,84,88,174,99,239,129,51,215,208,157,
-48,93,133,15,203,226,242,117,179,93,6,195,16,174,13,147,88,138,69,13,
-45,24,139,82,224,49,204,106,177,65,89,122,209,63,154,58,243,135,156,94,
-166,71,97,247,46,133,93,157,191,200,215,129,126,58,36,16,29,109,19,46,
-42,49,220,7,53,40,108,69,216,226,174,233,93,16,36,121,160,164,6,62,
-162,228,233,35,28,183,171,204,11,58,168,151,227,141,108,178,139,150,47,226,
-20,235,105,220,54,29,213,70,223,159,61,20,122,46,237,216,35,177,160,241,
-99,80,229,45,164,215,154,167,54,38,99,33,101,229,251,225,230,3,240,179,
-37,22,120,154,51,17,71,34,20,199,37,154,115,201,204,114,60,18,87,233,
-75,64,124,124,45,100,104,205,244,192,179,120,240,230,48,100,174,108,6,23,
-83,155,135,66,156,62,243,84,250,193,115,189,23,158,135,197,242,215,30,36,
-59,62,17,48,133,118,78,7,64,192,249,203,166,236,219,49,211,55,64,174,
-54,248,219,102,152,155,103,24,222,175,162,225,108,41,116,169,107,2,93,29,
-50,37,112,39,188,62,59,242,36,234,221,151,198,111,206,229,255,89,154,21,
-63,10,231,78,137,26,96,177,236,5,15,93,63,25,47,189,164,18,87,122,
-42,70,242,43,137,185,175,20,52,178,177,1,8,245,63,186,107,10,28,153,
-151,174,181,168,25,25,103,150,82,175,110,129,153,86,251,6,224,120,69,215,
-81,247,230,223,79,62,11,215,62,25,230,210,193,96,163,18,227,10,177,184,
-29,122,228,212,181,238,232,123,47,176,108,58,38,137,116,65,205,212,114,142,
-0,165,105,109,255,75,170,68,184,120,94,214,68,223,245,108,216,168,77,247,
-97,29,157,82,180,54,159,78,97,184,234,20,5,176,113,22,173,15,41,108,
-35,155,166,209,100,202,87,108,240,52,65,127,98,166,203,95,178,112,153,129,
-141,122,76,42,117,151,17,146,106,111,87,200,165,51,122,163,3,175,185,44,
-254,246,205,150,162,213,125,61,12,76,170,166,247,78,178,132,205,25,190,64,
-154,23,197,60,51,55,8,198,9,33,113,248,152,245,3,13,152,102,32,110,
-94,84,66,248,234,171,115,136,90,173,16,100,227,113,60,214,181,72,61,122,
-66,253,207,23,103,24,11,105,61,226,242,32,125,46,232,152,211,122,229,153,
-193,38,47,79,176,137,123,4,136,3,200,30,221,33,231,102,170,182,206,181,
-141,75,26,106,238,10,39,64,196,216,191,135,122,175,17,149,177,0,117,103,
-253,21,108,86,240,182,183,190,178,117,28,105,10,220,172,132,109,232,232,170,
-244,102,75,199,80,12,33,148,89,19,11,115,216,43,174,234,241,6,21,27,
-100,119,80,3,57,208,200,94,104,166,230,66,60,43,184,125,227,51,231,115,
-95,173,118,216,209,168,34,47,62,231,184,140,186,216,232,205,34,68,8,224,
-130,231,23,124,155,0,64,156,38,53,204,198,61,142,204,152,22,51,233,40,
-150,176,76,181,66,128,154,208,212,31,228,82,16,196,76,154,180,151,246,166,
-225,109,25,135,180,125,82,224,200,13,31,229,158,199,104,193,157,60,193,173,
-223,19,216,25,159,76,91,236,75,7,105,24,47,17,189,19,70,144,150,78,
-121,67,82,56,94,181,204,146,1,51,69,152,216,165,89,97,108,220,138,13,
-204,236,110,182,1,254,135,38,187,123,122,27,180,160,63,152,193,207,81,19,
-184,157,79,254,216,76,244,207,234,59,39,105,196,28,207,150,154,229,223,122,
-157,251,164,157,117,126,101,224,24,114,90,119,159,218,104,213,253,157,28,78,
-64,147,101,15,101,227,153,4,18,194,31,1,187,69,11,19,173,92,53,153,
-152,197,191,140,168,201,217,173,232,136,140,223,200,44,5,230,7,213,90,241,
-211,156,32,8,170,142,132,48,192,76,238,200,215,25,128,17,88,123,106,212,
-62,164,58,48,146,7,86,98,161,46,132,31,31,165,164,187,63,158,153,233,
-6,44,106,108,83,205,175,115,73,47,216,210,39,138,62,231,223,57,143,69,
-84,198,181,189,135,150,158,179,22,116,2,22,77,174,103,117,235,197,110,74,
-169,162,156,144,210,68,52,10,166,35,159,71,225,139,193,188,117,111,143,21,
-15,252,131,22,85,56,27,147,100,121,228,77,186,174,224,46,239,202,240,187,
-18,113,223,55,45,153,143,196,164,119,116,246,203,210,31,95,83,138,16,25,
-229,133,132,146,80,55,180,123,51,140,153,105,1,237,86,222,196,210,67,78,
-162,150,255,64,51,26,200,216,111,58,167,15,123,218,29,82,86,105,229,129,
-216,13,113,9,135,35,115,211,84,24,71,20,85,250,19,97,242,93,9,97,
-69,255,179,216,150,111,118,155,73,73,17,76,26,151,234,102,105,212,99,151,
-45,5,22,187,115,13,193,90,246,135,92,74,104,202,16,218,138,143,117,215,
-187,145,199,106,187,237,214,151,163,53,130,109,200,64,247,79,8,190,53,192,
-243,228,18,164,218,178,37,79,15,175,30,49,209,225,219,46,127,34,188,158,
-227,5,110,227,61,95,251,252,244,19,102,47,223,56,14,61,150,175,180,185,
-194,213,222,147,198,59,62,210,177,109,170,238,11,111,202,118,203,81,232,27,
-106,23,93,157,119,227,182,200,7,34,114,105,162,150,238,33,102,211,244,250,
-157,27,32,253,178,184,196,127,84,103,242,163,115,7,117,186,185,20,226,73,
-142,48,52,57,147,152,202,186,148,44,17,29,20,137,206,78,164,202,206,97,
-222,220,100,200,105,224,239,39,104,17,154,235,255,240,194,239,222,174,4,104,
-196,137,186,65,235,47,82,116,94,127,55,7,19,254,155,128,124,50,170,12,
-140,135,250,187,36,99,147,169,175,129,239,76,129,170,167,162,123,195,47,60,
-147,198,138,215,193,213,242,155,48,72,226,211,239,242,146,81,57,59,247,79,
-150,198,230,3,233,217,161,90,35,93,168,247,169,151,227,171,92,110,134,246,
-221,156,77,134,160,17,1,64,110,124,139,105,7,9,54,42,44,77,235,36,
-194,184,54,16,60,180,153,191,119,162,240,93,46,163,246,58,116,104,181,100,
-105,227,239,126,218,2,224,57,16,210,162,198,61,129,199,83,155,58,245,62,
-79,125,92,183,163,219,156,146,128,215,19,129,193,32,17,230,203,132,200,182,
-112,205,126,70,255,172,254,77,70,8,210,224,64,43,150,2,15,59,145,13,
-160,62,98,75,132,227,143,5,251,238,2,35,184,137,121,136,116,234,27,189,
-118,236,92,235,77,122,248,231,26,225,72,81,244,49,55,28,178,179,14,9,
-109,57,71,36,107,213,72,60,141,28,80,177,149,170,84,31,156,234,89,153,
-192,30,32,32,93,190,90,162,191,156,112,142,138,140,207,194,157,241,186,78,
-63,231,149,45,193,204,120,72,248,78,91,240,208,204,7,219,5,62,75,147,
-21,168,82,11,158,105,32,176,122,208,82,55,3,76,95,49,78,70,16,157,
-78,253,185,222,109,242,22,226,212,60,216,34,143,39,74,137,88,99,125,18,
-189,163,189,140,88,86,170,234,65,230,177,169,48,143,135,119,145,187,58,14,
-251,108,133,40,0,208,177,222,78,102,12,104,57,187,251,67,61,216,78,112,
-213,153,184,65,85,50,105,18,133,191,126,203,109,171,167,61,170,163,15,185,
-119,145,233,178,84,179,86,223,183,54,151,204,250,126,13,208,23,137,64,76,
-107,151,167,16,97,20,110,203,199,172,93,165,175,158,173,69,135,243,126,155,
-81,70,118,80,209,198,49,111,243,26,224,242,180,196,25,142,210,39,73,244,
-4,136,191,193,54,219,40,54,223,124,106,90,151,55,236,31,49,104,178,98,
-13,85,173,59,170,231,70,243,255,253,97,128,1,191,110,235,49,176,84,178,
-127,8,207,164,154,147,108,168,37,162,111,231,169,26,82,255,231,69,27,23,
-235,139,19,121,210,242,231,10,40,187,189,49,28,118,81,156,111,72,87,58,
-108,78,98,211,179,229,223,22,220,160,37,37,222,247,151,235,129,214,162,151,
-134,118,232,195,127,138,196,33,176,197,214,78,248,3,201,40,56,119,6,97,
-76,92,115,174,226,253,91,95,137,249,222,139,29,114,114,138,7,254,16,196,
-249,40,181,27,112,9,231,147,235,23,64,104,206,244,90,162,70,100,170,232,
-92,98,79,200,159,27,242,195,36,219,160,69,108,122,250,246,173,23,191,197,
-161,104,21,29,78,198,174,192,31,128,161,100,223,248,174,86,38,23,215,170,
-221,44,104,167,46,249,101,180,64,236,115,193,213,194,218,153,140,34,18,12,
-194,201,164,87,233,192,135,17,172,39,205,209,75,253,228,177,231,119,20,151,
-235,171,56,125,37,41,137,111,155,183,106,32,76,129,188,27,199,228,182,246,
-148,167,99,149,28,164,176,229,70,201,229,169,36,21,251,170,210,199,148,141,
-100,70,168,22,26,99,149,136,187,143,119,209,31,145,251,176,17,101,124,150,
-123,60,115,81,149,130,49,24,68,5,227,112,33,169,92,204,81,108,18,3,
-120,152,254,247,248,22,62,3,128,94,147,180,187,41,169,178,158,189,118,215,
-132,34,66,232,226,41,222,250,87,175,162,238,168,251,240,143,169,204,157,20,
-210,199,59,114,192,142,85,25,210,160,221,81,193,107,4,103,11,41,238,134,
-88,56,205,206,102,38,63,44,35,95,164,0,6,128,145,131,70,187,61,226,
-117,138,38,113,204,248,160,144,212,19,30,215,194,1,165,3,197,16,47,158,
-58,221,14,249,45,236,179,48,143,13,141,140,5,159,64,79,155,253,123,98,
-200,170,108,204,201,248,22,240,71,40,159,130,39,144,250,183,236,37,97,211,
-224,190,174,126,174,219,192,58,201,170,56,35,70,152,213,45,228,17,68,133,
-102,10,204,107,4,100,164,220,253,121,88,162,2,192,228,186,212,248,56,107,
-20,158,54,9,3,87,205,244,78,109,89,162,14,15,35,135,83,224,124,92,
-174,54,30,222,194,165,45,56,141,28,152,115,222,226,28,92,134,104,227,237,
-134,139,152,116,33,163,235,69,120,176,146,232,96,202,250,139,113,165,75,212,
-36,93,255,91,174,130,148,175,171,219,4,142,207,145,129,7,242,114,209,222,
-58,188,149,101,9,50,79,237,100,161,254,219,152,19,180,79,172,165,148,183,
-33,35,25,178,177,158,81,129,202,100,153,117,70,208,244,90,179,183,152,202,
-117,69,195,246,26,254,25,247,40,67,226,32,236,59,243,59,118,46,71,28,
-103,99,139,85,172,214,215,139,251,0,190,122,128,51,169,219,147,50,223,232,
-19,98,165,202,227,35,209,36,173,143,94,201,120,248,90,242,96,159,248,194,
-146,109,127,72,231,63,18,149,27,225,195,24,132,233,53,184,51,160,227,170,
-171,103,120,141,34,183,152,82,186,56,156,56,191,197,36,121,15,190,108,65,
-131,230,167,205,52,1,130,20,218,83,152,180,5,71,28,22,126,118,246,29,
-247,215,46,224,220,56,194,192,225,220,58,47,204,46,200,115,2,2,176,170,
-45,38,160,97,24,78,73,183,24,92,232,200,181,81,68,89,213,171,174,97,
-74,19,249,234,174,231,85,147,113,246,221,73,82,249,235,6,129,225,6,0,
-114,182,108,62,123,34,255,53,172,87,5,142,187,156,181,85,18,131,209,78,
-37,171,32,164,199,176,25,128,165,110,167,87,127,167,4,248,47,7,32,199,
-247,183,76,227,169,5,107,35,152,137,211,15,106,79,36,91,217,135,11,214,
-42,64,239,161,40,159,39,171,158,37,64,164,96,201,156,1,14,200,111,48,
-18,61,45,116,173,165,16,23,46,184,163,183,124,98,145,188,202,165,187,146,
-247,196,247,180,36,152,222,80,230,247,140,245,7,11,139,32,0,196,146,182,
-60,112,222,38,73,152,180,196,182,118,92,226,80,150,243,149,149,30,255,255,
-179,255,141,2,49,207,2,217,154,100,46,225,207,25,196,9,5,95,114,119,
-176,106,123,32,32,141,58,211,152,115,47,43,250,108,91,207,255,229,55,35,
-85,198,56,135,135,240,237,3,103,3,130,172,113,50,162,251,195,71,38,151,
-40,73,140,223,196,204,242,99,20,15,255,192,47,29,31,37,87,232,150,26,
-234,197,129,222,37,97,98,174,250,7,208,27,209,186,198,99,30,152,205,79,
-198,31,172,133,201,146,218,241,7,228,114,42,77,15,144,240,238,198,187,44,
-104,59,244,98,53,108,168,198,153,48,137,41,160,133,115,99,235,214,179,122,
-179,59,198,35,20,181,38,229,95,155,3,37,22,148,142,118,139,247,140,53,
-202,236,225,87,224,152,137,17,254,63,184,144,75,147,119,86,4,78,106,6,
-1,120,150,85,239,176,176,186,27,172,42,6,203,52,83,53,250,190,233,86,
-144,123,130,214,17,1,22,213,171,150,95,20,209,34,223,215,69,86,85,167,
-195,156,68,176,184,174,238,183,19,93,239,216,212,206,56,175,29,132,13,27,
-214,229,184,105,192,40,124,200,74,54,50,151,117,254,31,45,52,123,60,167,
-134,154,42,72,31,231,226,201,220,202,131,175,24,177,45,172,50,72,142,35,
-152,73,202,54,183,214,214,83,225,129,54,18,99,243,26,233,108,58,121,6,
-141,73,151,115,74,33,142,137,128,171,121,195,170,183,66,133,33,250,225,210,
-235,205,176,39,235,154,185,193,217,247,79,252,195,160,250,167,5,183,4,55,
-118,243,163,47,96,79,188,70,226,90,116,88,126,250,192,203,21,174,253,39,
-40,136,191,251,230,83,38,123,42,255,55,228,213,17,249,79,127,143,51,62,
-60,167,129,84,75,175,246,68,13,54,196,8,63,158,44,255,178,207,154,30,
-91,87,62,24,210,243,64,41,47,63,41,131,4,56,138,128,8,45,119,36,
-175,250,197,1,168,215,209,161,181,192,236,246,127,194,149,250,149,252,143,125,
-123,150,149,205,35,79,134,2,237,233,77,28,16,37,133,84,18,237,236,220,
-213,185,149,119,66,205,145,222,187,79,92,139,212,246,231,160,23,166,138,145,
-55,62,132,207,246,177,92,203,206,114,49,254,201,42,195,160,146,130,241,3,
-213,235,1,45,143,51,21,213,114,76,0,124,168,32,210,199,91,180,27,229,
-82,41,163,4,155,166,157,226,35,112,143,99,231,148,191,73,98,106,246,26,
-90,214,137,208,5,243,249,164,238,103,214,254,186,221,57,83,44,122,194,94,
-236,186,93,139,234,142,118,224,235,107,122,191,8,219,239,51,151,94,33,74,
-145,19,215,190,66,218,240,114,176,146,233,29,168,67,32,1,74,40,183,243,
-41,180,107,31,73,195,174,31,171,64,185,69,220,187,216,182,205,230,35,112,
-46,180,241,212,86,219,150,95,104,20,228,69,1,243,204,243,0,232,255,93,
-18,167,198,240,153,165,245,14,97,217,91,57,1,0,205,65,186,135,9,102,
-25,150,118,35,186,36,14,221,162,0,120,153,161,220,158,202,70,168,243,13,
-169,182,236,71,89,203,7,254,217,215,104,126,37,122,173,112,129,58,156,89,
-184,225,148,191,178,133,104,102,150,80,216,126,114,98,7,197,187,197,227,225,
-129,72,116,210,201,225,48,207,230,82,141,162,97,139,212,198,162,204,243,215,
-240,210,238,70,46,224,32,80,125,111,65,184,191,184,152,228,53,221,116,247,
-93,130,107,191,205,135,47,34,81,158,187,237,238,205,169,250,116,35,209,133,
-75,224,209,16,102,238,120,197,15,214,228,80,191,43,96,128,195,153,16,205,
-165,246,33,174,224,220,10,227,125,112,139,104,165,221,74,115,143,42,82,107,
-239,201,33,107,198,232,65,244,28,187,35,100,7,61,57,126,71,250,121,176,
-169,76,210,214,94,65,151,19,19,206,174,100,225,84,221,57,148,2,71,94,
-69,107,150,48,228,176,172,16,90,178,54,11,248,254,200,76,31,176,61,108,
-19,14,25,28,230,139,27,220,45,122,94,165,193,77,163,100,45,230,153,187,
-132,37,205,76,146,160,68,73,226,159,119,209,113,184,191,38,115,167,229,70,
-248,160,113,181,165,24,131,9,215,81,109,28,227,218,56,241,64,189,190,208,
-119,174,178,36,84,8,63,164,240,125,183,98,66,65,16,81,27,211,28,72,
-92,230,19,1,203,98,159,185,240,131,203,178,2,72,118,77,168,138,255,137,
-57,206,49,10,186,86,73,96,223,99,15,199,57,192,19,102,65,201,204,13,
-52,135,245,104,123,157,231,1,155,18,221,208,33,90,219,223,246,86,178,223,
-197,111,238,209,110,226,93,249,129,216,219,228,196,13,33,160,44,235,101,182,
-34,190,7,224,235,208,250,31,237,9,223,48,49,136,204,222,1,3,235,249,
-121,43,180,96,247,242,216,191,55,15,249,64,114,10,81,60,207,137,211,106,
-124,242,203,179,157,169,108,208,123,103,117,124,75,16,72,83,78,23,39,13,
-14,112,140,218,135,248,86,119,123,14,207,2,225,70,253,62,23,10,181,4,
-105,209,225,21,207,58,188,23,234,96,218,88,249,50,48,173,160,65,63,150,
-143,117,93,33,67,121,84,66,192,55,61,216,220,122,41,172,14,71,227,132,
-17,76,92,131,147,194,218,11,192,155,244,171,32,141,150,212,40,80,222,30,
-78,135,17,201,234,34,215,182,200,11,183,65,173,11,120,130,77,165,100,197,
-47,14,24,47,54,213,229,78,252,142,27,132,138,210,5,204,42,42,105,136,
-239,188,185,111,204,57,181,255,234,163,184,193,248,85,24,168,77,74,75,177,
-145,46,134,113,211,134,49,185,60,191,42,82,42,222,104,114,33,24,40,24,
-1,64,114,61,105,90,33,176,216,255,250,31,154,37,85,209,154,3,111,237,
-63,27,86,116,83,5,44,150,129,96,217,64,127,122,100,133,149,177,213,221,
-185,138,101,96,46,193,192,127,228,72,248,95,89,206,145,84,249,108,105,124,
-104,165,39,213,34,234,194,8,213,52,4,41,203,59,243,86,75,141,73,248,
-203,245,122,116,51,28,27,101,223,86,170,125,114,66,196,94,255,33,231,245,
-67,165,203,130,89,42,252,121,81,205,195,208,253,150,61,195,122,104,47,63,
-123,62,249,165,226,220,45,57,252,103,236,90,61,96,222,219,163,14,199,182,
-131,132,41,160,59,69,48,234,95,179,134,179,120,219,227,138,30,116,202,204,
-87,187,155,240,232,35,216,150,96,236,152,84,138,14,117,237,219,147,2,59,
-24,38,241,123,52,244,202,253,48,2,3,64,29,43,98,75,171,227,215,112,
-55,141,31,25,241,71,6,73,71,207,152,195,93,175,105,201,67,0,106,136,
-71,172,2,39,179,124,75,43,176,172,207,62,68,203,192,80,170,70,84,193,
-124,139,122,191,235,175,47,66,234,154,192,180,52,131,251,108,81,34,59,9,
-102,154,60,122,154,171,152,100,135,162,237,121,140,130,20,250,5,183,185,35,
-165,197,199,92,238,211,18,108,131,129,221,204,1,219,56,124,187,72,48,87,
-194,77,9,74,28,195,231,10,238,78,147,36,161,250,156,161,14,8,87,235,
-166,137,11,204,248,195,25,114,34,141,55,216,117,253,41,238,231,46,0,158,
-112,217,119,98,66,15,13,181,83,22,97,132,113,154,154,209,135,246,11,139,
-185,189,203,158,35,34,150,134,57,73,76,44,88,124,60,33,159,230,168,83,
-32,131,131,196,105,186,210,238,43,148,95,154,99,212,20,97,46,112,166,210,
-116,85,176,160,247,29,91,95,1,244,205,168,249,240,168,55,107,29,5,72,
-40,131,118,110,115,170,67,15,94,248,44,215,143,197,238,103,163,113,48,77,
-104,141,76,89,10,65,68,109,38,251,252,225,6,195,134,104,51,243,58,239,
-96,208,118,100,146,248,191,82,52,20,63,132,6,14,150,156,160,67,244,241,
-158,181,68,179,16,209,162,117,95,6,109,213,68,75,170,62,27,73,179,130,
-135,184,9,116,193,200,156,159,173,248,188,42,171,79,121,225,212,28,186,101,
-167,122,184,254,255,193,131,129,220,107,179,83,255,99,86,218,181,202,207,231,
-72,96,248,248,120,51,38,32,165,140,131,193,48,144,77,254,231,235,178,54,
-26,30,72,213,255,186,236,68,176,109,220,15,43,219,96,16,4,73,227,155,
-28,18,132,172,20,107,230,201,58,199,90,252,48,36,239,239,113,93,33,139,
-181,186,31,3,51,27,215,242,164,92,175,185,147,78,163,20,221,29,220,45,
-151,17,118,7,10,152,38,196,66,83,192,229,122,156,241,207,228,163,197,151,
-127,213,192,55,169,71,0,109,14,132,226,217,130,80,34,196,170,135,2,42,
-227,136,40,204,233,225,51,68,227,218,67,223,17,188,70,234,193,43,248,117,
-154,236,116,198,229,105,19,212,167,227,103,121,202,91,170,5,162,16,207,202,
-122,168,90,19,168,16,212,99,130,38,145,32,107,148,61,234,138,204,177,49,
-169,189,70,236,239,54,123,110,120,7,7,108,23,168,12,228,234,113,111,86,
-88,69,187,111,220,240,1,22,162,40,165,204,238,214,109,253,79,63,208,199,
-240,112,209,101,205,232,3,125,77,19,129,95,242,107,25,86,64,202,221,84,
-65,184,205,10,200,126,209,31,109,16,242,138,197,5,152,140,209,61,7,142,
-158,3,25,62,194,88,68,243,89,38,215,93,223,23,238,54,29,193,95,11,
-204,136,91,31,103,159,16,136,254,30,153,97,123,195,71,40,128,185,185,63,
-225,2,21,123,216,174,135,12,230,228,187,165,118,166,53,4,119,26,21,11,
-82,25,98,103,217,43,131,17,167,79,233,42,5,224,66,202,237,167,171,62,
-176,58,123,62,120,115,194,117,76,9,165,88,111,207,72,234,126,164,81,57,
-247,36,18,107,228,26,235,224,193,121,194,13,30,59,189,111,99,2,137,215,
-113,54,123,195,27,138,89,4,135,170,181,70,241,207,225,194,20,24,210,174,
-93,255,139,190,100,245,56,117,205,122,84,180,36,247,224,88,192,88,223,68,
-228,166,19,138,94,200,7,77,30,62,156,239,173,32,218,10,206,205,25,53,
-206,53,88,117,217,77,169,146,236,193,80,231,215,20,140,188,201,203,46,114,
-253,191,72,156,1,11,200,114,55,140,69,135,132,234,10,10,182,220,125,191,
-210,3,242,235,121,248,3,185,0,82,14,138,163,55,75,148,141,249,111,171,
-146,186,229,128,143,234,138,154,20,168,231,203,200,228,78,221,53,165,85,56,
-151,17,134,204,117,16,78,37,233,141,97,70,101,83,78,171,149,110,157,6,
-4,169,189,57,199,19,238,143,75,212,99,143,100,148,245,35,202,193,231,47,
-145,184,90,19,73,192,155,22,191,242,154,99,199,47,227,94,238,24,224,179,
-78,11,9,203,4,220,65,166,111,217,204,102,102,49,5,172,108,40,162,41,
-58,163,218,227,85,22,128,119,255,213,11,162,167,82,29,75,85,22,59,246,
-38,214,72,141,43,128,186,15,14,15,8,234,184,216,110,146,216,194,28,179,
-225,150,90,105,16,60,12,43,250,97,84,147,34,210,7,169,9,149,144,51,
-7,66,166,56,236,164,68,22,46,250,75,39,28,178,37,194,25,57,181,91,
-219,162,223,41,140,253,122,210,11,241,48,241,64,24,135,39,17,111,248,63,
-106,130,204,125,96,185,135,56,166,65,144,79,162,102,63,74,2,233,97,174,
-163,94,60,205,53,0,31,32,100,160,241,125,82,149,76,71,254,57,115,144,
-110,202,80,127,62,217,192,137,18,160,66,162,35,58,61,164,122,94,152,9,
-165,96,151,114,227,34,57,23,109,13,177,32,126,11,129,218,100,201,15,244,
-196,147,218,132,69,74,106,107,51,188,76,0,155,98,176,208,201,227,66,240,
-199,57,40,164,181,150,207,108,9,189,118,224,13,85,210,226,164,40,188,216,
-154,69,197,120,141,192,63,82,232,59,131,20,73,98,17,203,69,62,9,30,
-233,21,93,178,93,31,62,239,64,58,23,244,124,152,220,217,201,232,112,22,
-214,236,106,255,214,199,247,167,56,75,249,119,3,70,181,200,18,51,181,222,
-254,55,140,73,188,226,219,178,32,154,100,243,94,152,228,42,117,217,169,238,
-233,3,170,202,27,20,77,228,2,0,10,187,38,71,241,178,207,80,173,179,
-104,240,240,80,254,18,85,68,207,84,123,221,135,130,191,220,55,240,49,153,
-246,31,237,180,227,161,85,117,63,204,125,118,249,128,166,241,105,134,99,161,
-178,246,122,147,149,219,230,146,35,88,155,243,41,33,5,152,199,207,17,189,
-41,10,104,251,49,135,172,167,7,100,179,23,144,187,125,136,109,148,147,158,
-237,75,13,85,205,32,69,77,68,110,118,89,190,233,2,112,240,32,70,83,
-112,39,250,213,0,176,33,214,184,105,133,81,105,210,164,178,134,53,140,70,
-51,137,91,247,197,34,237,125,28,155,18,177,245,232,26,128,60,81,42,3,
-247,101,208,40,173,141,192,201,237,169,215,69,81,131,18,0,24,68,176,228,
-31,206,90,110,23,192,49,155,162,125,216,12,254,71,97,127,150,95,119,148,
-14,7,146,49,189,142,124,176,34,2,149,148,100,138,253,177,13,246,184,73,
-52,245,31,124,115,57,56,29,66,30,143,67,234,238,200,142,124,181,71,242,
-104,213,66,217,221,20,205,200,67,8,50,115,24,106,220,110,83,213,211,5,
-148,88,229,71,246,198,83,201,245,113,148,255,253,232,68,100,128,179,111,236,
-214,122,252,181,24,33,104,159,110,126,134,253,180,42,124,129,154,50,53,169,
-118,215,133,223,11,89,206,48,197,73,21,44,24,123,68,237,34,34,54,159,
-230,5,176,186,140,34,157,126,180,93,193,51,173,74,126,88,81,193,87,75,
-42,5,148,86,18,82,51,168,201,248,81,171,180,161,21,157,76,52,196,209,
-18,23,83,190,47,202,10,11,160,27,240,128,175,25,190,101,253,166,203,170,
-194,4,231,187,35,51,210,18,230,190,68,233,251,186,153,39,244,198,190,6,
-93,238,90,219,90,142,2,83,200,145,27,116,104,90,49,178,83,221,152,135,
-146,160,218,127,167,237,201,3,217,140,130,80,3,203,85,152,181,25,171,109,
-150,125,174,208,57,51,122,210,183,235,67,216,76,91,153,177,194,37,59,7,
-29,59,22,128,158,28,159,251,75,14,73,67,61,219,104,179,87,238,1,10,
-152,8,234,201,81,73,235,252,237,6,60,34,228,76,203,208,80,70,152,244,
-46,87,35,235,199,163,55,234,8,161,249,218,143,195,243,240,166,225,146,251,
-61,140,39,144,183,177,182,191,207,17,157,17,116,132,183,13,244,170,61,199,
-55,55,238,42,15,104,235,201,145,12,224,107,186,2,5,147,2,118,221,151,
-229,109,106,164,181,231,159,81,183,193,104,130,240,35,71,154,217,35,1,131,
-30,136,139,54,225,211,60,180,149,251,29,27,239,124,46,76,143,192,112,239,
-181,168,152,234,231,161,186,125,133,125,125,26,155,92,234,121,35,32,44,42,
-36,194,215,204,95,243,27,83,34,121,94,14,201,210,193,248,242,131,217,12,
-0,27,156,3,28,97,107,71,100,41,161,113,199,191,147,65,59,198,154,110,
-149,242,190,51,131,247,181,250,139,149,20,219,8,216,136,120,248,207,144,172,
-125,141,138,65,122,73,47,61,155,41,112,122,143,225,183,127,164,117,124,86,
-79,139,221,58,178,44,121,253,74,235,215,212,186,80,99,190,158,139,75,87,
-46,223,119,184,190,58,172,169,163,133,100,90,129,131,197,188,229,227,27,94,
-112,177,64,142,92,184,37,58,118,146,3,84,182,212,200,247,84,175,58,175,
-43,60,59,41,135,22,187,150,220,33,217,155,144,161,103,62,23,245,206,220,
-171,226,42,93,135,18,95,199,133,57,223,240,155,92,79,149,108,247,201,97,
-168,186,156,216,14,144,99,98,150,157,189,72,201,161,150,206,12,250,164,129,
-10,185,137,7,132,251,44,168,93,36,254,70,172,216,36,71,23,139,217,7,
-243,248,116,173,192,78,51,86,66,212,117,143,161,159,187,246,47,241,58,168,
-44,167,76,102,31,243,14,143,26,207,248,73,63,25,143,194,99,10,233,190,
-59,176,115,186,138,173,254,126,21,28,164,168,208,229,163,252,142,20,29,224,
-74,203,80,193,0,198,231,211,236,111,144,219,157,145,202,199,11,190,198,60,
-100,115,54,19,227,130,133,144,87,113,109,37,29,54,179,1,173,54,74,95,
-118,253,121,173,33,239,186,221,238,163,16,240,129,233,33,39,35,206,203,54,
-46,78,158,102,208,49,148,181,52,157,43,195,130,12,74,16,20,129,133,103,
-254,21,80,193,128,30,193,26,241,196,124,211,156,54,24,106,224,128,154,162,
-117,128,60,37,115,90,43,235,217,95,146,68,141,171,126,152,174,226,213,105,
-213,162,204,20,123,131,248,24,146,202,81,231,197,230,218,6,18,33,154,37,
-100,114,237,16,241,232,93,163,8,137,227,220,248,1,178,200,123,49,86,250,
-108,239,48,14,108,80,175,244,10,191,73,100,192,44,114,62,2,92,62,248,
-193,9,145,1,168,139,173,23,161,7,219,88,186,109,102,38,39,130,254,93,
-109,240,230,224,139,158,64,135,158,200,221,218,192,244,60,31,216,56,167,33,
-195,13,235,74,10,214,28,239,166,31,122,231,45,151,184,133,117,209,85,251,
-38,101,46,137,31,52,45,46,157,34,3,116,13,83,212,226,176,79,41,70,
-15,22,154,124,164,19,97,212,197,119,97,140,254,38,25,158,173,158,4,251,
-178,226,79,210,126,4,193,65,67,45,86,119,121,214,200,249,13,45,66,5,
-211,234,50,132,16,31,141,8,109,90,104,107,245,131,48,213,107,211,101,37,
-30,145,187,189,114,96,236,136,8,138,69,59,143,121,87,195,74,230,109,77,
-21,15,255,203,231,2,100,245,252,171,166,205,224,33,216,68,122,28,22,177,
-26,185,11,158,249,6,196,254,27,26,51,75,144,232,51,43,48,94,12,179,
-170,187,171,196,236,206,91,59,134,134,99,169,42,201,195,170,166,248,163,116,
-250,232,74,254,125,105,103,7,25,55,151,171,200,199,32,41,234,164,214,21,
-126,168,89,182,6,114,45,18,66,117,167,185,223,174,145,213,173,13,120,2,
-17,2,234,154,247,211,152,151,68,178,189,243,191,100,69,56,220,204,135,8,
-8,243,106,81,200,161,112,254,168,188,204,223,41,199,179,118,175,24,80,181,
-170,251,119,53,153,144,106,152,193,156,110,2,76,22,210,189,54,142,117,176,
-41,12,75,205,151,114,6,13,189,73,198,171,142,128,101,182,209,222,11,114,
-45,39,232,119,242,223,111,146,173,163,117,232,158,82,227,218,55,181,153,209,
-191,44,235,153,169,79,18,202,33,154,149,43,125,156,37,109,244,43,86,226,
-56,81,141,51,169,137,156,143,5,117,89,207,170,127,215,28,116,171,7,16,
-180,47,117,179,112,16,184,46,223,127,230,15,226,49,143,34,161,0,74,190,
-164,205,33,6,69,47,76,151,176,62,237,109,170,137,103,59,171,2,60,229,
-86,41,126,159,61,9,180,190,18,223,84,23,218,160,202,214,171,221,113,204,
-223,96,236,3,154,57,72,156,53,119,189,98,142,172,23,47,223,207,157,224,
-57,119,67,120,18,183,228,129,205,184,146,183,200,106,135,187,11,115,171,131,
-251,158,86,210,209,130,59,30,225,207,255,154,136,104,232,164,216,111,202,39,
-238,160,200,71,247,26,87,37,249,213,204,171,77,16,203,192,251,40,218,194,
-95,209,125,2,40,146,99,233,97,2,230,42,37,159,217,203,58,10,96,182,
-87,9,157,37,95,143,33,75,0,126,233,64,250,86,148,124,114,213,94,31,
-0,78,79,79,222,171,188,103,37,48,242,25,135,37,248,43,201,102,126,30,
-100,113,223,59,89,234,155,103,117,168,48,180,178,58,35,117,89,197,128,226,
-143,119,68,57,227,238,78,14,8,80,192,90,193,1,100,162,163,120,57,94,
-116,120,206,152,147,169,54,85,42,86,144,120,19,99,62,206,56,224,76,128,
-251,235,161,87,69,139,212,197,6,191,205,144,101,220,184,119,103,245,6,58,
-183,235,138,35,144,193,33,211,70,5,69,3,102,203,97,194,49,102,114,200,
-52,199,181,175,114,151,75,40,18,3,10,73,189,60,47,246,169,114,6,229,
-65,165,111,45,114,7,51,136,91,213,86,46,225,115,110,169,124,107,27,26,
-32,226,191,146,204,127,15,192,70,59,90,187,83,189,66,74,43,106,248,135,
-173,68,134,6,70,29,237,15,147,187,75,94,62,253,39,202,198,17,10,42,
-17,170,214,42,24,75,111,46,1,39,12,20,163,215,188,178,161,61,191,80,
-127,254,5,8,191,218,123,54,100,108,192,225,250,11,159,198,32,103,77,33,
-72,35,22,227,209,71,206,205,179,45,195,11,59,1,125,216,24,109,81,44,
-177,108,134,55,180,112,120,22,112,47,193,108,184,206,50,45,91,19,179,17,
-64,241,139,188,116,249,37,142,99,243,134,29,141,233,131,199,53,191,77,133,
-1,130,226,123,237,139,236,88,124,251,156,103,146,19,194,154,14,104,17,186,
-224,105,154,25,43,154,2,100,58,218,113,159,64,135,241,111,232,168,201,201,
-255,210,172,125,247,147,19,14,159,203,141,177,112,138,78,106,147,76,34,124,
-193,212,81,244,140,12,98,186,140,231,112,24,201,156,35,87,63,226,132,82,
-242,34,126,13,25,93,191,210,33,108,109,167,141,54,137,243,188,174,7,154,
-34,150,166,58,77,250,40,77,63,235,52,242,134,63,14,44,133,88,72,217,
-4,216,7,212,165,80,138,105,150,212,172,80,213,55,164,231,22,234,128,223,
-80,53,230,137,13,232,174,84,130,18,222,110,192,9,114,210,213,161,76,212,
-89,121,158,133,55,189,140,211,65,214,202,133,208,12,223,229,237,202,131,113,
-131,128,189,56,172,139,26,159,43,75,179,164,52,199,246,144,186,76,255,50,
-0,243,35,107,159,192,242,228,194,64,173,204,29,200,146,187,117,191,55,215,
-123,148,157,165,204,129,176,90,56,155,10,86,24,237,70,159,164,238,197,183,
-218,222,23,227,139,159,121,169,153,244,163,59,220,128,211,45,186,64,205,15,
-108,212,2,16,99,232,130,209,70,224,159,2,183,217,187,41,43,62,121,158,
-237,206,181,1,41,115,64,118,11,122,24,47,119,19,252,15,79,206,35,27,
-244,246,90,32,145,55,118,149,255,220,93,169,24,253,78,247,129,226,45,106,
-190,92,81,86,92,107,235,67,94,236,37,184,157,40,70,69,210,210,229,208,
-45,199,17,25,21,63,155,44,200,30,112,139,74,123,197,199,131,127,176,222,
-164,226,78,206,56,140,9,137,152,44,246,78,127,250,215,40,181,188,57,235,
-36,37,183,16,21,70,166,215,58,172,137,159,76,226,5,140,62,246,223,238,
-54,15,2,212,228,87,10,80,27,172,202,63,128,132,117,56,254,85,163,79,
-47,191,180,135,19,60,133,249,43,165,183,91,179,0,184,162,237,99,61,44,
-62,72,27,64,12,82,15,231,212,134,147,94,247,63,62,65,203,62,63,208,
-1,254,11,10,73,133,42,53,81,151,235,54,106,23,106,133,187,45,67,11,
-220,225,54,217,139,72,65,24,218,101,144,141,196,88,138,115,231,49,182,77,
-105,63,10,186,191,129,125,7,195,86,255,239,79,88,91,104,198,237,25,142,
-184,215,24,191,254,220,211,216,176,142,167,54,180,238,41,193,177,183,134,162,
-33,208,20,151,217,77,238,191,232,55,76,60,117,52,87,56,61,110,238,135,
-122,201,202,58,85,125,4,23,134,232,10,23,18,149,122,116,28,243,196,33,
-22,171,33,243,48,180,94,122,145,168,54,194,80,56,79,188,144,153,154,185,
-219,200,76,64,155,221,18,102,43,112,37,62,215,95,63,190,161,149,194,23,
-15,237,228,170,241,171,168,210,81,232,66,145,32,105,240,146,3,115,36,144,
-15,223,90,251,152,171,182,191,144,220,16,72,80,126,119,247,78,146,88,3,
-164,181,76,161,23,92,211,178,84,20,250,218,69,22,222,204,41,89,82,160,
-91,96,118,47,242,172,147,51,15,233,200,124,62,53,24,91,89,45,83,20,
-221,234,89,42,62,67,221,9,63,113,34,170,117,42,102,152,22,20,48,74,
-15,0,163,53,243,82,198,54,209,39,124,42,178,14,199,11,222,84,44,34,
-240,146,136,128,225,36,94,254,224,229,25,174,220,11,76,118,61,19,226,54,
-62,108,218,152,42,27,191,132,254,53,232,167,57,220,70,207,187,24,35,210,
-251,71,39,236,56,136,113,157,188,131,165,134,182,87,231,60,96,211,101,222,
-235,215,86,227,123,98,83,56,175,252,13,209,18,192,168,169,155,106,207,99,
-176,59,188,244,30,58,77,168,180,214,80,200,228,12,206,13,250,58,114,62,
-128,147,68,34,88,176,235,17,174,36,154,126,129,108,165,166,178,92,123,210,
-71,101,101,240,159,50,171,95,93,19,216,232,131,90,247,173,1,228,223,125,
-209,118,54,67,56,130,23,3,132,88,68,93,130,52,250,57,196,172,165,212,
-49,90,87,98,33,200,254,203,199,43,50,177,150,18,41,218,3,84,7,124,
-244,188,128,197,143,83,175,18,113,47,6,233,102,247,217,228,88,218,113,248,
-102,10,110,96,67,13,104,162,181,228,52,147,61,6,240,61,222,31,174,255,
-180,217,140,153,208,148,41,216,59,245,15,182,31,113,248,211,85,34,120,161,
-67,179,95,46,232,227,209,191,162,73,67,98,20,32,16,159,167,118,72,41,
-110,17,152,39,57,140,245,99,248,197,255,63,72,155,132,18,87,207,241,17,
-82,232,4,159,9,135,30,54,7,228,220,133,170,58,103,50,123,159,107,57,
-143,7,186,29,130,69,199,39,60,242,30,36,40,159,246,70,127,45,45,194,
-113,31,113,85,63,94,248,45,180,242,52,57,50,68,243,122,83,40,233,249,
-30,93,74,34,165,209,93,158,88,39,29,67,73,239,137,0,141,105,31,61,
-52,206,146,61,43,169,249,144,64,24,112,195,219,141,197,15,11,232,55,225,
-231,167,33,227,86,182,213,83,138,184,25,111,80,135,115,203,120,134,43,54,
-118,104,222,51,33,70,91,144,226,163,132,241,117,252,80,218,68,28,246,147,
-63,212,205,175,225,156,189,64,77,113,141,85,64,155,158,11,243,177,60,194,
-116,132,214,175,191,86,106,132,37,70,166,36,75,129,212,95,197,227,49,17,
-143,54,18,21,146,162,132,154,42,222,252,176,80,59,219,26,118,103,154,204,
-236,158,63,124,94,30,37,38,93,195,182,7,89,157,254,0,129,176,6,40,
-241,126,20,247,202,48,206,118,190,200,72,2,241,47,223,122,191,82,127,46,
-14,130,30,170,123,249,18,54,201,151,143,78,73,6,168,152,122,24,7,42,
-175,56,56,193,112,137,173,34,199,12,236,144,192,161,186,205,246,141,100,79,
-70,188,140,93,131,10,138,255,185,149,22,176,215,163,131,214,9,59,11,29,
-100,208,239,15,98,76,78,35,129,191,16,208,123,147,25,232,200,252,179,69,
-39,240,34,1,183,255,65,243,65,220,163,110,151,85,72,180,100,208,49,1,
-7,23,52,220,14,181,106,192,148,47,175,168,40,216,118,15,65,245,218,140,
-0,165,169,58,246,118,79,14,59,45,185,250,39,20,207,87,73,123,90,194,
-55,45,168,21,97,59,116,137,97,91,86,130,18,150,187,123,43,112,93,14,
-224,175,47,228,179,207,130,237,155,51,185,249,83,94,38,59,64,41,93,226,
-113,129,48,253,166,49,56,39,114,108,196,118,81,159,250,186,49,110,79,18,
-21,223,66,227,97,0,162,114,114,199,164,192,194,73,125,139,210,195,254,69,
-31,101,63,227,15,249,180,137,93,98,190,214,238,130,92,193,44,108,114,33,
-107,86,186,84,26,20,100,160,12,59,225,43,22,80,238,66,159,61,86,200,
-165,127,34,57,0,9,251,63,83,172,206,229,225,69,128,199,49,120,62,127,
-178,144,72,29,81,158,57,107,9,77,182,232,171,220,215,183,72,176,85,237,
-203,99,218,1,81,89,225,227,51,208,227,20,132,193,205,96,94,10,55,27,
-92,83,0,184,199,133,173,21,105,13,70,95,92,4,249,251,121,220,53,91,
-65,147,235,105,36,86,201,240,222,182,105,96,40,197,230,253,131,255,226,157,
-42,155,119,55,31,85,225,88,220,231,106,136,239,127,240,39,157,123,27,191,
-8,157,247,206,2,108,170,74,29,153,99,184,243,218,200,154,6,73,43,169,
-66,181,29,98,199,150,30,35,217,171,174,195,35,75,174,104,4,131,70,53,
-8,143,41,74,84,5,128,26,84,37,82,225,53,4,29,96,197,228,149,248,
-42,230,118,220,15,243,112,95,74,227,83,99,107,253,242,48,127,74,53,128,
-33,248,100,24,132,142,2,117,218,29,108,153,194,119,239,219,189,200,233,24,
-117,64,117,148,160,162,178,36,201,162,148,103,215,94,223,86,195,140,214,224,
-106,7,87,9,58,84,89,1,178,176,152,250,44,117,94,147,182,113,102,108,
-45,47,89,19,184,156,100,20,202,74,239,141,205,178,73,175,15,254,59,124,
-137,62,153,142,151,223,200,94,240,244,61,31,201,41,36,117,85,1,87,96,
-50,204,126,221,209,135,142,85,32,255,234,138,110,3,186,130,226,205,230,20,
-68,128,58,30,94,102,17,242,24,159,133,118,219,128,128,69,54,31,40,94,
-50,241,17,58,163,44,191,220,192,250,39,44,221,51,179

<TRUNCATED>

[16/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/Reader.cc
----------------------------------------------------------------------
diff --git a/c++/src/orc/Reader.cc b/c++/src/orc/Reader.cc
deleted file mode 100644
index 2343e91..0000000
--- a/c++/src/orc/Reader.cc
+++ /dev/null
@@ -1,1902 +0,0 @@
-/**
- * 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 "orc/Adaptor.hh"
-#include "orc/Reader.hh"
-#include "orc/OrcFile.hh"
-#include "ColumnReader.hh"
-#include "Exceptions.hh"
-#include "RLE.hh"
-#include "TypeImpl.hh"
-#include "orc/Int128.hh"
-
-#include "wrap/coded-stream-wrapper.h"
-
-#include <algorithm>
-#include <iostream>
-#include <limits>
-#include <memory>
-#include <sstream>
-#include <string>
-#include <vector>
-
-namespace orc {
-
-  struct ReaderOptionsPrivate {
-    std::list<int64_t> includedColumns;
-    uint64_t dataStart;
-    uint64_t dataLength;
-    uint64_t tailLocation;
-    bool throwOnHive11DecimalOverflow;
-    int32_t forcedScaleOnHive11Decimal;
-    std::ostream* errorStream;
-    MemoryPool* memoryPool;
-    std::string serializedTail;
-
-    ReaderOptionsPrivate() {
-      includedColumns.assign(1,0);
-      dataStart = 0;
-      dataLength = std::numeric_limits<uint64_t>::max();
-      tailLocation = std::numeric_limits<uint64_t>::max();
-      throwOnHive11DecimalOverflow = true;
-      forcedScaleOnHive11Decimal = 6;
-      errorStream = &std::cerr;
-      memoryPool = getDefaultPool();
-    }
-  };
-
-  ReaderOptions::ReaderOptions():
-    privateBits(std::unique_ptr<ReaderOptionsPrivate>
-                (new ReaderOptionsPrivate())) {
-    // PASS
-  }
-
-  ReaderOptions::ReaderOptions(const ReaderOptions& rhs):
-    privateBits(std::unique_ptr<ReaderOptionsPrivate>
-                (new ReaderOptionsPrivate(*(rhs.privateBits.get())))) {
-    // PASS
-  }
-
-  ReaderOptions::ReaderOptions(ReaderOptions& rhs) {
-    // swap privateBits with rhs
-    ReaderOptionsPrivate* l = privateBits.release();
-    privateBits.reset(rhs.privateBits.release());
-    rhs.privateBits.reset(l);
-  }
-
-  ReaderOptions& ReaderOptions::operator=(const ReaderOptions& rhs) {
-    if (this != &rhs) {
-      privateBits.reset(new ReaderOptionsPrivate(*(rhs.privateBits.get())));
-    }
-    return *this;
-  }
-
-  ReaderOptions::~ReaderOptions() {
-    // PASS
-  }
-
-  ReaderOptions& ReaderOptions::include(const std::list<int64_t>& include) {
-    privateBits->includedColumns.assign(include.begin(), include.end());
-    return *this;
-  }
-
-  ReaderOptions& ReaderOptions::include(std::vector<int64_t> include) {
-    privateBits->includedColumns.assign(include.begin(), include.end());
-    return *this;
-  }
-
-  ReaderOptions& ReaderOptions::range(uint64_t offset,
-                                      uint64_t length) {
-    privateBits->dataStart = offset;
-    privateBits->dataLength = length;
-    return *this;
-  }
-
-  ReaderOptions& ReaderOptions::setTailLocation(uint64_t offset) {
-    privateBits->tailLocation = offset;
-    return *this;
-  }
-
-  ReaderOptions& ReaderOptions::setMemoryPool(MemoryPool& pool) {
-    privateBits->memoryPool = &pool;
-    return *this;
-  }
-
-  ReaderOptions& ReaderOptions::setSerializedFileTail(const std::string& value
-                                                      ) {
-    privateBits->serializedTail = value;
-    return *this;
-  }
-
-  MemoryPool* ReaderOptions::getMemoryPool() const{
-    return privateBits->memoryPool;
-  }
-
-  const std::list<int64_t>& ReaderOptions::getInclude() const {
-    return privateBits->includedColumns;
-  }
-
-  uint64_t ReaderOptions::getOffset() const {
-    return privateBits->dataStart;
-  }
-
-  uint64_t ReaderOptions::getLength() const {
-    return privateBits->dataLength;
-  }
-
-  uint64_t ReaderOptions::getTailLocation() const {
-    return privateBits->tailLocation;
-  }
-
-  ReaderOptions& ReaderOptions::throwOnHive11DecimalOverflow(bool shouldThrow){
-    privateBits->throwOnHive11DecimalOverflow = shouldThrow;
-    return *this;
-  }
-
-  bool ReaderOptions::getThrowOnHive11DecimalOverflow() const {
-    return privateBits->throwOnHive11DecimalOverflow;
-  }
-
-  ReaderOptions& ReaderOptions::forcedScaleOnHive11Decimal(int32_t forcedScale
-                                                           ) {
-    privateBits->forcedScaleOnHive11Decimal = forcedScale;
-    return *this;
-  }
-
-  int32_t ReaderOptions::getForcedScaleOnHive11Decimal() const {
-    return privateBits->forcedScaleOnHive11Decimal;
-  }
-
-  ReaderOptions& ReaderOptions::setErrorStream(std::ostream& stream) {
-    privateBits->errorStream = &stream;
-    return *this;
-  }
-
-  std::ostream* ReaderOptions::getErrorStream() const {
-    return privateBits->errorStream;
-  }
-
-  std::string ReaderOptions::getSerializedFileTail() const {
-    return privateBits->serializedTail;
-  }
-
-  StripeInformation::~StripeInformation() {
-
-  }
-
-  class ColumnStatisticsImpl: public ColumnStatistics {
-  private:
-    uint64_t valueCount;
-
-  public:
-    ColumnStatisticsImpl(const proto::ColumnStatistics& stats);
-    virtual ~ColumnStatisticsImpl();
-
-    uint64_t getNumberOfValues() const override {
-      return valueCount;
-    }
-
-    std::string toString() const override {
-      std::ostringstream buffer;
-      buffer << "Column has " << valueCount << " values" << std::endl;
-      return buffer.str();
-    }
-  };
-
-  class BinaryColumnStatisticsImpl: public BinaryColumnStatistics {
-  private:
-    bool _hasTotalLength;
-    uint64_t valueCount;
-    uint64_t totalLength;
-
-  public:
-    BinaryColumnStatisticsImpl(const proto::ColumnStatistics& stats,
-                               bool correctStats);
-    virtual ~BinaryColumnStatisticsImpl();
-
-    bool hasTotalLength() const override {
-      return _hasTotalLength;
-    }
-    uint64_t getNumberOfValues() const override {
-      return valueCount;
-    }
-
-    uint64_t getTotalLength() const override {
-      if(_hasTotalLength){
-        return totalLength;
-      }else{
-        throw ParseError("Total length is not defined.");
-      }
-    }
-
-    std::string toString() const override {
-      std::ostringstream buffer;
-      buffer << "Data type: Binary" << std::endl
-             << "Values: " << valueCount << std::endl;
-      if(_hasTotalLength){
-        buffer << "Total length: " << totalLength << std::endl;
-      }else{
-        buffer << "Total length: not defined" << std::endl;
-      }
-      return buffer.str();
-    }
-  };
-
-  class BooleanColumnStatisticsImpl: public BooleanColumnStatistics {
-  private:
-    bool _hasCount;
-    uint64_t valueCount;
-    uint64_t trueCount;
-
-  public:
-    BooleanColumnStatisticsImpl(const proto::ColumnStatistics& stats, bool correctStats);
-    virtual ~BooleanColumnStatisticsImpl();
-
-    bool hasCount() const override {
-      return _hasCount;
-    }
-
-    uint64_t getNumberOfValues() const override {
-      return valueCount;
-    }
-
-    uint64_t getFalseCount() const override {
-      if(_hasCount){
-        return valueCount - trueCount;
-      }else{
-        throw ParseError("False count is not defined.");
-      }
-    }
-
-    uint64_t getTrueCount() const override {
-      if(_hasCount){
-        return trueCount;
-      }else{
-        throw ParseError("True count is not defined.");
-      }
-    }
-
-    std::string toString() const override {
-      std::ostringstream buffer;
-      buffer << "Data type: Boolean" << std::endl
-             << "Values: " << valueCount << std::endl;
-      if(_hasCount){
-        buffer << "(true: " << trueCount << "; false: "
-               << valueCount - trueCount << ")" << std::endl;
-      } else {
-        buffer << "(true: not defined; false: not defined)" << std::endl;
-        buffer << "True and false count are not defined" << std::endl;
-      }
-      return buffer.str();
-    }
-  };
-
-  class DateColumnStatisticsImpl: public DateColumnStatistics {
-  private:
-    bool _hasMinimum;
-    bool _hasMaximum;
-    uint64_t valueCount;
-    int32_t minimum;
-    int32_t maximum;
-
-  public:
-    DateColumnStatisticsImpl(const proto::ColumnStatistics& stats, bool correctStats);
-    virtual ~DateColumnStatisticsImpl();
-
-    bool hasMinimum() const override {
-      return _hasMinimum;
-    }
-
-    bool hasMaximum() const override {
-      return _hasMaximum;
-    }
-
-    uint64_t getNumberOfValues() const override {
-      return valueCount;
-    }
-
-    int32_t getMinimum() const override {
-      if(_hasMinimum){
-        return minimum;
-      }else{
-        throw ParseError("Minimum is not defined.");
-      }
-    }
-
-    int32_t getMaximum() const override {
-      if(_hasMaximum){
-        return maximum;
-      }else{
-        throw ParseError("Maximum is not defined.");
-      }
-    }
-
-    std::string toString() const override {
-      std::ostringstream buffer;
-      buffer << "Data type: Date" << std::endl
-             << "Values: " << valueCount << std::endl;
-      if(_hasMinimum){
-        buffer << "Minimum: " << minimum << std::endl;
-      }else{
-        buffer << "Minimum: not defined" << std::endl;
-      }
-
-      if(_hasMaximum){
-        buffer << "Maximum: " << maximum << std::endl;
-      }else{
-        buffer << "Maximum: not defined" << std::endl;
-      }
-      return buffer.str();
-    }
-  };
-
-  class DecimalColumnStatisticsImpl: public DecimalColumnStatistics {
-  private:
-    bool _hasMinimum;
-    bool _hasMaximum;
-    bool _hasSum;
-    uint64_t valueCount;
-    std::string minimum;
-    std::string maximum;
-    std::string sum;
-
-  public:
-    DecimalColumnStatisticsImpl(const proto::ColumnStatistics& stats, bool correctStats);
-    virtual ~DecimalColumnStatisticsImpl();
-
-    bool hasMinimum() const override {
-      return _hasMinimum;
-    }
-
-    bool hasMaximum() const override {
-      return _hasMaximum;
-    }
-
-    bool hasSum() const override {
-      return _hasSum;
-    }
-
-    uint64_t getNumberOfValues() const override {
-      return valueCount;
-    }
-
-    Decimal getMinimum() const override {
-      if(_hasMinimum){
-        return Decimal(minimum);
-      }else{
-        throw ParseError("Minimum is not defined.");
-      }
-    }
-
-    Decimal getMaximum() const override {
-      if(_hasMaximum){
-        return Decimal(maximum);
-      }else{
-        throw ParseError("Maximum is not defined.");
-      }
-    }
-
-    Decimal getSum() const override {
-      if(_hasSum){
-        return Decimal(sum);
-      }else{
-        throw ParseError("Sum is not defined.");
-      }
-    }
-
-    std::string toString() const override {
-      std::ostringstream buffer;
-      buffer << "Data type: Decimal" << std::endl
-          << "Values: " << valueCount << std::endl;
-      if(_hasMinimum){
-        buffer << "Minimum: " << minimum << std::endl;
-      }else{
-        buffer << "Minimum: not defined" << std::endl;
-      }
-
-      if(_hasMaximum){
-        buffer << "Maximum: " << maximum << std::endl;
-      }else{
-        buffer << "Maximum: not defined" << std::endl;
-      }
-
-      if(_hasSum){
-        buffer << "Sum: " << sum << std::endl;
-      }else{
-        buffer << "Sum: not defined" << std::endl;
-      }
-
-      return buffer.str();
-    }
-  };
-
-  class DoubleColumnStatisticsImpl: public DoubleColumnStatistics {
-  private:
-    bool _hasMinimum;
-    bool _hasMaximum;
-    bool _hasSum;
-    uint64_t valueCount;
-    double minimum;
-    double maximum;
-    double sum;
-
-  public:
-    DoubleColumnStatisticsImpl(const proto::ColumnStatistics& stats);
-    virtual ~DoubleColumnStatisticsImpl();
-
-    bool hasMinimum() const override {
-      return _hasMinimum;
-    }
-
-    bool hasMaximum() const override {
-      return _hasMaximum;
-    }
-
-    bool hasSum() const override {
-      return _hasSum;
-    }
-
-    uint64_t getNumberOfValues() const override {
-      return valueCount;
-    }
-
-    double getMinimum() const override {
-      if(_hasMinimum){
-        return minimum;
-      }else{
-        throw ParseError("Minimum is not defined.");
-      }
-    }
-
-    double getMaximum() const override {
-      if(_hasMaximum){
-        return maximum;
-      }else{
-        throw ParseError("Maximum is not defined.");
-      }
-    }
-
-    double getSum() const override {
-      if(_hasSum){
-        return sum;
-      }else{
-        throw ParseError("Sum is not defined.");
-      }
-    }
-
-    std::string toString() const override {
-      std::ostringstream buffer;
-      buffer << "Data type: Double" << std::endl
-          << "Values: " << valueCount << std::endl;
-      if(_hasMinimum){
-        buffer << "Minimum: " << minimum << std::endl;
-      }else{
-        buffer << "Minimum: not defined" << std::endl;
-      }
-
-      if(_hasMaximum){
-        buffer << "Maximum: " << maximum << std::endl;
-      }else{
-        buffer << "Maximum: not defined" << std::endl;
-      }
-
-      if(_hasSum){
-        buffer << "Sum: " << sum << std::endl;
-      }else{
-        buffer << "Sum: not defined" << std::endl;
-      }
-      return buffer.str();
-    }
-  };
-
-  class IntegerColumnStatisticsImpl: public IntegerColumnStatistics {
-  private:
-    bool _hasMinimum;
-    bool _hasMaximum;
-    bool _hasSum;
-    uint64_t valueCount;
-    int64_t minimum;
-    int64_t maximum;
-    int64_t sum;
-
-  public:
-    IntegerColumnStatisticsImpl(const proto::ColumnStatistics& stats);
-    virtual ~IntegerColumnStatisticsImpl();
-
-    bool hasMinimum() const override {
-      return _hasMinimum;
-    }
-
-    bool hasMaximum() const override {
-      return _hasMaximum;
-    }
-
-    bool hasSum() const override {
-      return _hasSum;
-    }
-
-    uint64_t getNumberOfValues() const override {
-      return valueCount;
-    }
-
-    int64_t getMinimum() const override {
-      if(_hasMinimum){
-        return minimum;
-      }else{
-        throw ParseError("Minimum is not defined.");
-      }
-    }
-
-    int64_t getMaximum() const override {
-      if(_hasMaximum){
-        return maximum;
-      }else{
-        throw ParseError("Maximum is not defined.");
-      }
-    }
-
-    int64_t getSum() const override {
-      if(_hasSum){
-        return sum;
-      }else{
-        throw ParseError("Sum is not defined.");
-      }
-    }
-
-    std::string toString() const override {
-      std::ostringstream buffer;
-      buffer << "Data type: Integer" << std::endl
-          << "Values: " << valueCount << std::endl;
-      if(_hasMinimum){
-        buffer << "Minimum: " << minimum << std::endl;
-      }else{
-        buffer << "Minimum: not defined" << std::endl;
-      }
-
-      if(_hasMaximum){
-        buffer << "Maximum: " << maximum << std::endl;
-      }else{
-        buffer << "Maximum: not defined" << std::endl;
-      }
-
-      if(_hasSum){
-        buffer << "Sum: " << sum << std::endl;
-      }else{
-        buffer << "Sum: not defined" << std::endl;
-      }
-      return buffer.str();
-    }
-  };
-
-  class StringColumnStatisticsImpl: public StringColumnStatistics {
-  private:
-    bool _hasMinimum;
-    bool _hasMaximum;
-    bool _hasTotalLength;
-    uint64_t valueCount;
-    std::string minimum;
-    std::string maximum;
-    uint64_t totalLength;
-
-  public:
-    StringColumnStatisticsImpl(const proto::ColumnStatistics& stats, bool correctStats);
-    virtual ~StringColumnStatisticsImpl();
-
-    bool hasMinimum() const override {
-      return _hasMinimum;
-    }
-
-    bool hasMaximum() const override {
-      return _hasMaximum;
-    }
-
-    bool hasTotalLength() const override {
-      return _hasTotalLength;
-    }
-
-    uint64_t getNumberOfValues() const override {
-      return valueCount;
-    }
-
-    std::string getMinimum() const override {
-      if(_hasMinimum){
-        return minimum;
-      }else{
-        throw ParseError("Minimum is not defined.");
-      }
-    }
-
-    std::string getMaximum() const override {
-      if(_hasMaximum){
-        return maximum;
-      }else{
-        throw ParseError("Maximum is not defined.");
-      }
-    }
-
-    uint64_t getTotalLength() const override {
-      if(_hasTotalLength){
-        return totalLength;
-      }else{
-        throw ParseError("Total length is not defined.");
-      }
-    }
-
-    std::string toString() const override {
-      std::ostringstream buffer;
-      buffer << "Data type: String" << std::endl
-          << "Values: " << valueCount << std::endl;
-      if(_hasMinimum){
-        buffer << "Minimum: " << minimum << std::endl;
-      }else{
-        buffer << "Minimum is not defined" << std::endl;
-      }
-
-      if(_hasMaximum){
-        buffer << "Maximum: " << maximum << std::endl;
-      }else{
-        buffer << "Maximum is not defined" << std::endl;
-      }
-
-      if(_hasTotalLength){
-        buffer << "Total length: " << totalLength << std::endl;
-      }else{
-        buffer << "Total length is not defined" << std::endl;
-      }
-      return buffer.str();
-    }
-  };
-
-  class TimestampColumnStatisticsImpl: public TimestampColumnStatistics {
-  private:
-    bool _hasMinimum;
-    bool _hasMaximum;
-    uint64_t valueCount;
-    int64_t minimum;
-    int64_t maximum;
-
-  public:
-    TimestampColumnStatisticsImpl(const proto::ColumnStatistics& stats,
-                                  bool correctStats);
-    virtual ~TimestampColumnStatisticsImpl();
-
-    bool hasMinimum() const override {
-      return _hasMinimum;
-    }
-
-    bool hasMaximum() const override {
-      return _hasMaximum;
-    }
-
-    uint64_t getNumberOfValues() const override {
-      return valueCount;
-    }
-
-    int64_t getMinimum() const override {
-      if(_hasMinimum){
-        return minimum;
-      }else{
-        throw ParseError("Minimum is not defined.");
-      }
-    }
-
-    int64_t getMaximum() const override {
-      if(_hasMaximum){
-        return maximum;
-      }else{
-        throw ParseError("Maximum is not defined.");
-      }
-    }
-
-    std::string toString() const override {
-      std::ostringstream buffer;
-      buffer << "Data type: Timestamp" << std::endl
-          << "Values: " << valueCount << std::endl;
-      if(_hasMinimum){
-        buffer << "Minimum: " << minimum << std::endl;
-      }else{
-        buffer << "Minimum is not defined" << std::endl;
-      }
-
-      if(_hasMaximum){
-        buffer << "Maximum: " << maximum << std::endl;
-      }else{
-        buffer << "Maximum is not defined" << std::endl;
-      }
-      return buffer.str();
-    }
-  };
-
-  class StripeInformationImpl : public StripeInformation {
-    uint64_t offset;
-    uint64_t indexLength;
-    uint64_t dataLength;
-    uint64_t footerLength;
-    uint64_t numRows;
-
-  public:
-
-    StripeInformationImpl(uint64_t _offset,
-                          uint64_t _indexLength,
-                          uint64_t _dataLength,
-                          uint64_t _footerLength,
-                          uint64_t _numRows) :
-      offset(_offset),
-      indexLength(_indexLength),
-      dataLength(_dataLength),
-      footerLength(_footerLength),
-      numRows(_numRows)
-    {}
-
-    virtual ~StripeInformationImpl();
-
-    uint64_t getOffset() const override {
-      return offset;
-    }
-
-    uint64_t getLength() const override {
-      return indexLength + dataLength + footerLength;
-    }
-    uint64_t getIndexLength() const override {
-      return indexLength;
-    }
-
-    uint64_t getDataLength()const override {
-      return dataLength;
-    }
-
-    uint64_t getFooterLength() const override {
-      return footerLength;
-    }
-
-    uint64_t getNumberOfRows() const override {
-      return numRows;
-    }
-  };
-
-  ColumnStatistics* convertColumnStatistics(const proto::ColumnStatistics& s,
-                                            bool correctStats) {
-    if (s.has_intstatistics()) {
-      return new IntegerColumnStatisticsImpl(s);
-    } else if (s.has_doublestatistics()) {
-      return new DoubleColumnStatisticsImpl(s);
-    } else if (s.has_stringstatistics()) {
-      return new StringColumnStatisticsImpl(s, correctStats);
-    } else if (s.has_bucketstatistics()) {
-      return new BooleanColumnStatisticsImpl(s, correctStats);
-    } else if (s.has_decimalstatistics()) {
-      return new DecimalColumnStatisticsImpl(s, correctStats);
-    } else if (s.has_timestampstatistics()) {
-      return new TimestampColumnStatisticsImpl(s, correctStats);
-    } else if (s.has_datestatistics()) {
-      return new DateColumnStatisticsImpl(s, correctStats);
-    } else if (s.has_binarystatistics()) {
-      return new BinaryColumnStatisticsImpl(s, correctStats);
-    } else {
-      return new ColumnStatisticsImpl(s);
-    }
-  }
-
-  Statistics::~Statistics() {
-    // PASS
-  }
-
-  class StatisticsImpl: public Statistics {
-  private:
-    std::list<ColumnStatistics*> colStats;
-
-    // DELIBERATELY NOT IMPLEMENTED
-    StatisticsImpl(const StatisticsImpl&);
-    StatisticsImpl& operator=(const StatisticsImpl&);
-
-  public:
-    StatisticsImpl(const proto::StripeStatistics& stripeStats, bool correctStats) {
-      for(int i = 0; i < stripeStats.colstats_size(); i++) {
-        colStats.push_back(convertColumnStatistics
-                           (stripeStats.colstats(i), correctStats));
-      }
-    }
-
-    StatisticsImpl(const proto::Footer& footer, bool correctStats) {
-      for(int i = 0; i < footer.statistics_size(); i++) {
-        colStats.push_back(convertColumnStatistics
-                           (footer.statistics(i), correctStats));
-      }
-    }
-
-    virtual const ColumnStatistics* getColumnStatistics(uint32_t columnId
-                                                        ) const override {
-      std::list<ColumnStatistics*>::const_iterator it = colStats.begin();
-      std::advance(it, static_cast<int64_t>(columnId));
-      return *it;
-    }
-
-    virtual ~StatisticsImpl();
-
-    uint32_t getNumberOfColumns() const override {
-      return static_cast<uint32_t>(colStats.size());
-    }
-  };
-
-  StatisticsImpl::~StatisticsImpl() {
-    for(std::list<ColumnStatistics*>::iterator ptr = colStats.begin();
-        ptr != colStats.end();
-        ++ptr) {
-      delete *ptr;
-    }
-  }
-
-  Reader::~Reader() {
-    // PASS
-  }
-
-  StripeInformationImpl::~StripeInformationImpl() {
-    // PASS
-  }
-
-  static const uint64_t DIRECTORY_SIZE_GUESS = 16 * 1024;
-
-  class ReaderImpl : public Reader {
-  private:
-    // inputs
-    std::unique_ptr<InputStream> stream;
-    ReaderOptions options;
-    const uint64_t footerStart;
-    std::vector<bool> selectedColumns;
-
-    // custom memory pool
-    MemoryPool& memoryPool;
-
-    // postscript
-    std::unique_ptr<proto::PostScript> postscript;
-    const uint64_t blockSize;
-    const CompressionKind compression;
-
-    // footer
-    std::unique_ptr<proto::Footer> footer;
-    DataBuffer<uint64_t> firstRowOfStripe;
-    uint64_t numberOfStripes;
-    std::unique_ptr<Type> schema;
-
-    // metadata
-    mutable std::unique_ptr<proto::Metadata> metadata;
-    mutable bool isMetadataLoaded;
-
-    // reading state
-    uint64_t previousRow;
-    uint64_t firstStripe;
-    uint64_t currentStripe;
-    uint64_t lastStripe; // the stripe AFTER the last one
-    uint64_t currentRowInStripe;
-    uint64_t rowsInCurrentStripe;
-    proto::StripeInformation currentStripeInfo;
-    proto::StripeFooter currentStripeFooter;
-    std::unique_ptr<ColumnReader> reader;
-
-    // internal methods
-    proto::StripeFooter getStripeFooter(const proto::StripeInformation& info);
-    void startNextStripe();
-    void checkOrcVersion();
-    void selectTypeParent(size_t columnId);
-    void selectTypeChildren(size_t columnId);
-    void readMetadata() const;
-    std::unique_ptr<ColumnVectorBatch> createRowBatch(const Type& type,
-                                                      uint64_t capacity
-                                                      ) const;
-
-  public:
-    /**
-     * Constructor that lets the user specify additional options.
-     * @param stream the stream to read from
-     * @param options options for reading
-     * @param postscript the postscript for the file
-     * @param footer the footer for the file
-     * @param footerStart the byte offset of the start of the footer
-     */
-    ReaderImpl(std::unique_ptr<InputStream> stream,
-               const ReaderOptions& options,
-               std::unique_ptr<proto::PostScript> postscript,
-               std::unique_ptr<proto::Footer> footer,
-               uint64_t footerStart);
-
-    const ReaderOptions& getReaderOptions() const;
-
-    CompressionKind getCompression() const override;
-
-    std::string getFormatVersion() const override;
-
-    uint64_t getNumberOfRows() const override;
-
-    uint64_t getRowIndexStride() const override;
-
-    const std::string& getStreamName() const override;
-
-    std::list<std::string> getMetadataKeys() const override;
-
-    std::string getMetadataValue(const std::string& key) const override;
-
-    bool hasMetadataValue(const std::string& key) const override;
-
-    uint64_t getCompressionSize() const override;
-
-    uint64_t getNumberOfStripes() const override;
-
-    std::unique_ptr<StripeInformation> getStripe(uint64_t
-                                                 ) const override;
-
-    uint64_t getNumberOfStripeStatistics() const override;
-
-    std::unique_ptr<Statistics>
-    getStripeStatistics(uint64_t stripeIndex) const override;
-
-
-    uint64_t getContentLength() const override;
-
-    std::unique_ptr<Statistics> getStatistics() const override;
-
-    std::unique_ptr<ColumnStatistics> getColumnStatistics(uint32_t columnId
-                                                          ) const override;
-
-    const Type& getType() const override;
-
-    const std::vector<bool> getSelectedColumns() const override;
-
-    std::unique_ptr<ColumnVectorBatch> createRowBatch(uint64_t size
-                                                      ) const override;
-
-    bool next(ColumnVectorBatch& data) override;
-
-    uint64_t getRowNumber() const override;
-
-    void seekToRow(uint64_t rowNumber) override;
-
-    MemoryPool* getMemoryPool() const ;
-
-    bool hasCorrectStatistics() const override;
-
-    std::string getSerializedFileTail() const override;
-  };
-
-  InputStream::~InputStream() {
-    // PASS
-  };
-
-  uint64_t getCompressionBlockSize(const proto::PostScript& ps) {
-    if (ps.has_compressionblocksize()) {
-      return ps.compressionblocksize();
-    } else {
-      return 256 * 1024;
-    }
-  }
-
-  CompressionKind convertCompressionKind(const proto::PostScript& ps) {
-    if (ps.has_compression()) {
-      return static_cast<CompressionKind>(ps.compression());
-    } else {
-      throw ParseError("Unknown compression type");
-    }
-  }
-
-  ReaderImpl::ReaderImpl(std::unique_ptr<InputStream> input,
-                         const ReaderOptions& opts,
-                         std::unique_ptr<proto::PostScript> _postscript,
-                         std::unique_ptr<proto::Footer> _footer,
-                         uint64_t _footerStart
-                         ): stream(std::move(input)),
-                            options(opts),
-                            footerStart(_footerStart),
-                            memoryPool(*opts.getMemoryPool()),
-                            postscript(std::move(_postscript)),
-                            blockSize(getCompressionBlockSize(*postscript)),
-                            compression(convertCompressionKind(*postscript)),
-                            footer(std::move(_footer)),
-                            firstRowOfStripe(memoryPool, 0) {
-    isMetadataLoaded = false;
-    checkOrcVersion();
-    numberOfStripes = static_cast<uint64_t>(footer->stripes_size());
-    currentStripe = static_cast<uint64_t>(footer->stripes_size());
-    lastStripe = 0;
-    currentRowInStripe = 0;
-    uint64_t rowTotal = 0;
-
-    firstRowOfStripe.resize(static_cast<uint64_t>(footer->stripes_size()));
-    for(size_t i=0; i < static_cast<size_t>(footer->stripes_size()); ++i) {
-      firstRowOfStripe[i] = rowTotal;
-      proto::StripeInformation stripeInfo =
-        footer->stripes(static_cast<int>(i));
-      rowTotal += stripeInfo.numberofrows();
-      bool isStripeInRange = stripeInfo.offset() >= opts.getOffset() &&
-        stripeInfo.offset() < opts.getOffset() + opts.getLength();
-      if (isStripeInRange) {
-        if (i < currentStripe) {
-          currentStripe = i;
-        }
-        if (i >= lastStripe) {
-          lastStripe = i + 1;
-        }
-      }
-    }
-    firstStripe = currentStripe;
-
-    if (currentStripe == 0) {
-      previousRow = (std::numeric_limits<uint64_t>::max)();
-    } else if (currentStripe ==
-               static_cast<uint64_t>(footer->stripes_size())) {
-      previousRow = footer->numberofrows();
-    } else {
-      previousRow = firstRowOfStripe[firstStripe]-1;
-    }
-
-    schema = convertType(footer->types(0), *footer);
-    schema->assignIds(0);
-    previousRow = (std::numeric_limits<uint64_t>::max)();
-
-    selectedColumns.assign(static_cast<size_t>(footer->types_size()), false);
-
-    const std::list<int64_t>& included = options.getInclude();
-    for(std::list<int64_t>::const_iterator columnId = included.begin();
-        columnId != included.end(); ++columnId) {
-      if (*columnId <= static_cast<int64_t>(schema->getSubtypeCount())) {
-        selectTypeParent(static_cast<size_t>(*columnId));
-        selectTypeChildren(static_cast<size_t>(*columnId));
-      }
-    }
-  }
-
-  std::string ReaderImpl::getSerializedFileTail() const {
-    proto::FileTail tail;
-    proto::PostScript *mutable_ps = tail.mutable_postscript();
-    mutable_ps->CopyFrom(*postscript);
-    proto::Footer *mutableFooter = tail.mutable_footer();
-    mutableFooter->CopyFrom(*footer);
-    tail.set_footerstart(footerStart);
-    std::string result;
-    if (!tail.SerializeToString(&result)) {
-      throw ParseError("Failed to serialize file tail");
-    }
-    return result;
-  }
-
-  const ReaderOptions& ReaderImpl::getReaderOptions() const {
-    return options;
-  }
-
-  CompressionKind ReaderImpl::getCompression() const {
-    return compression;
-  }
-
-  uint64_t ReaderImpl::getCompressionSize() const {
-    return blockSize;
-  }
-
-  uint64_t ReaderImpl::getNumberOfStripes() const {
-    return numberOfStripes;
-  }
-
-  uint64_t ReaderImpl::getNumberOfStripeStatistics() const {
-    if (!isMetadataLoaded) {
-      readMetadata();
-    }
-    return metadata.get() == nullptr ? 0 :
-      static_cast<uint64_t>(metadata->stripestats_size());
-  }
-
-  std::unique_ptr<StripeInformation>
-  ReaderImpl::getStripe(uint64_t stripeIndex) const {
-    if (stripeIndex > getNumberOfStripes()) {
-      throw std::logic_error("stripe index out of range");
-    }
-    proto::StripeInformation stripeInfo =
-      footer->stripes(static_cast<int>(stripeIndex));
-
-    return std::unique_ptr<StripeInformation>
-      (new StripeInformationImpl
-       (stripeInfo.offset(),
-        stripeInfo.indexlength(),
-        stripeInfo.datalength(),
-        stripeInfo.footerlength(),
-        stripeInfo.numberofrows()));
-  }
-
-  std::string ReaderImpl::getFormatVersion() const {
-    std::stringstream result;
-    for(int i=0; i < postscript->version_size(); ++i) {
-      if (i != 0) {
-        result << ".";
-      }
-      result << postscript->version(i);
-    }
-    return result.str();
-  }
-
-  uint64_t ReaderImpl::getNumberOfRows() const {
-    return footer->numberofrows();
-  }
-
-  uint64_t ReaderImpl::getContentLength() const {
-    return footer->contentlength();
-  }
-
-  uint64_t ReaderImpl::getRowIndexStride() const {
-    return footer->rowindexstride();
-  }
-
-  const std::string& ReaderImpl::getStreamName() const {
-    return stream->getName();
-  }
-
-  std::list<std::string> ReaderImpl::getMetadataKeys() const {
-    std::list<std::string> result;
-    for(int i=0; i < footer->metadata_size(); ++i) {
-      result.push_back(footer->metadata(i).name());
-    }
-    return result;
-  }
-
-  std::string ReaderImpl::getMetadataValue(const std::string& key) const {
-    for(int i=0; i < footer->metadata_size(); ++i) {
-      if (footer->metadata(i).name() == key) {
-        return footer->metadata(i).value();
-      }
-    }
-    throw std::range_error("key not found");
-  }
-
-  bool ReaderImpl::hasMetadataValue(const std::string& key) const {
-    for(int i=0; i < footer->metadata_size(); ++i) {
-      if (footer->metadata(i).name() == key) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  void ReaderImpl::selectTypeParent(size_t columnId) {
-    for(size_t parent=0; parent < columnId; ++parent) {
-      const proto::Type& parentType = footer->types(static_cast<int>(parent));
-      for(int idx=0; idx < parentType.subtypes_size(); ++idx) {
-        uint64_t child = parentType.subtypes(idx);
-        if (child == columnId) {
-          if (!selectedColumns[parent]) {
-            selectedColumns[parent] = true;
-            selectTypeParent(parent);
-            return;
-          }
-        }
-      }
-    }
-  }
-
-  void ReaderImpl::selectTypeChildren(size_t columnId) {
-    if (!selectedColumns[columnId]) {
-      selectedColumns[columnId] = true;
-      const proto::Type& parentType =
-        footer->types(static_cast<int>(columnId));
-      for(int idx=0; idx < parentType.subtypes_size(); ++idx) {
-        uint64_t child = parentType.subtypes(idx);
-        selectTypeChildren(child);
-      }
-    }
-  }
-
-  const std::vector<bool> ReaderImpl::getSelectedColumns() const {
-    return selectedColumns;
-  }
-
-  const Type& ReaderImpl::getType() const {
-    return *(schema.get());
-  }
-
-  uint64_t ReaderImpl::getRowNumber() const {
-    return previousRow;
-  }
-
-  std::unique_ptr<Statistics> ReaderImpl::getStatistics() const {
-    return std::unique_ptr<Statistics>
-      (new StatisticsImpl(*footer,
-                          hasCorrectStatistics()));
-  }
-
-  std::unique_ptr<ColumnStatistics>
-  ReaderImpl::getColumnStatistics(uint32_t index) const {
-    if (index >= static_cast<uint64_t>(footer->statistics_size())) {
-      throw std::logic_error("column index out of range");
-    }
-    proto::ColumnStatistics col =
-      footer->statistics(static_cast<int32_t>(index));
-    return std::unique_ptr<ColumnStatistics> (convertColumnStatistics
-                                              (col, hasCorrectStatistics()));
-  }
-
-  void ReaderImpl::readMetadata() const {
-    uint64_t metadataSize = postscript->metadatalength();
-    uint64_t metadataStart = footerStart - metadataSize;
-    if (metadataSize != 0) {
-      std::unique_ptr<SeekableInputStream> pbStream =
-        createDecompressor(compression,
-                           std::unique_ptr<SeekableInputStream>
-                             (new SeekableFileInputStream(stream.get(),
-                                                          metadataStart,
-                                                          metadataSize,
-                                                          memoryPool)),
-                           blockSize,
-                           memoryPool);
-      metadata.reset(new proto::Metadata());
-      if (!metadata->ParseFromZeroCopyStream(pbStream.get())) {
-        throw ParseError("Failed to parse the metadata");
-      }
-    }
-    isMetadataLoaded = true;
-  }
-
-  std::unique_ptr<Statistics>
-  ReaderImpl::getStripeStatistics(uint64_t stripeIndex) const {
-    if (!isMetadataLoaded) {
-      readMetadata();
-    }
-    if (metadata.get() == nullptr) {
-      throw std::logic_error("No stripe statistics in file");
-    }
-    return std::unique_ptr<Statistics>
-      (new StatisticsImpl(metadata->stripestats
-                          (static_cast<int>(stripeIndex)),
-                          hasCorrectStatistics()));
-  }
-
-
-  void ReaderImpl::seekToRow(uint64_t rowNumber) {
-    // Empty file
-    if (lastStripe == 0) {
-      return;
-    }
-
-    // If we are reading only a portion of the file
-    // (bounded by firstStripe and lastStripe),
-    // seeking before or after the portion of interest should return no data.
-    // Implement this by setting previousRow to the number of rows in the file.
-
-    // seeking past lastStripe
-    if ( (lastStripe == static_cast<uint64_t>(footer->stripes_size())
-            && rowNumber >= footer->numberofrows())  ||
-         (lastStripe < static_cast<uint64_t>(footer->stripes_size())
-            && rowNumber >= firstRowOfStripe[lastStripe])   ) {
-      currentStripe = static_cast<uint64_t>(footer->stripes_size());
-      previousRow = footer->numberofrows();
-      return;
-    }
-
-    uint64_t seekToStripe = 0;
-    while (seekToStripe+1 < lastStripe &&
-                  firstRowOfStripe[seekToStripe+1] <= rowNumber) {
-      seekToStripe++;
-    }
-
-    // seeking before the first stripe
-    if (seekToStripe < firstStripe) {
-      currentStripe = static_cast<uint64_t>(footer->stripes_size());
-      previousRow = footer->numberofrows();
-      return;
-    }
-
-    currentStripe = seekToStripe;
-    currentRowInStripe = 0;
-    std::unique_ptr<orc::ColumnVectorBatch> batch =
-        createRowBatch(rowNumber-firstRowOfStripe[currentStripe]);
-    next(*batch);
-  }
-
-  bool ReaderImpl::hasCorrectStatistics() const {
-    return postscript->has_writerversion() && postscript->writerversion();
-  }
-
-  proto::StripeFooter ReaderImpl::getStripeFooter
-       (const proto::StripeInformation& info) {
-    uint64_t stripeFooterStart = info.offset() + info.indexlength() +
-      info.datalength();
-    uint64_t stripeFooterLength = info.footerlength();
-    std::unique_ptr<SeekableInputStream> pbStream =
-      createDecompressor(compression,
-                         std::unique_ptr<SeekableInputStream>
-                         (new SeekableFileInputStream(stream.get(),
-                                                      stripeFooterStart,
-                                                      stripeFooterLength,
-                                                      memoryPool,
-                                                      static_cast<int64_t>
-                                                      (blockSize)
-                                                      )),
-                         blockSize,
-                         memoryPool);
-    proto::StripeFooter result;
-    if (!result.ParseFromZeroCopyStream(pbStream.get())) {
-      throw ParseError(std::string("bad StripeFooter from ") +
-                       pbStream->getName());
-    }
-    return result;
-  }
-
-  class StripeStreamsImpl: public StripeStreams {
-  private:
-    const ReaderImpl& reader;
-    const proto::StripeFooter& footer;
-    const uint64_t stripeStart;
-    InputStream& input;
-    MemoryPool& memoryPool;
-
-  public:
-    StripeStreamsImpl(const ReaderImpl& reader,
-                      const proto::StripeFooter& footer,
-                      uint64_t stripeStart,
-                      InputStream& input,
-                      MemoryPool& memoryPool);
-
-    virtual ~StripeStreamsImpl();
-
-    virtual const ReaderOptions& getReaderOptions() const override;
-
-    virtual const std::vector<bool> getSelectedColumns() const override;
-
-    virtual proto::ColumnEncoding getEncoding(int64_t columnId) const override;
-
-    virtual std::unique_ptr<SeekableInputStream>
-    getStream(int64_t columnId,
-              proto::Stream_Kind kind,
-              bool shouldStream) const override;
-
-    MemoryPool& getMemoryPool() const override;
-  };
-
-  StripeStreamsImpl::StripeStreamsImpl(const ReaderImpl& _reader,
-                                       const proto::StripeFooter& _footer,
-                                       uint64_t _stripeStart,
-                                       InputStream& _input,
-                                       MemoryPool& _memoryPool
-                                       ): reader(_reader),
-                                          footer(_footer),
-                                          stripeStart(_stripeStart),
-                                          input(_input),
-                                          memoryPool(_memoryPool) {
-    // PASS
-  }
-
-  StripeStreamsImpl::~StripeStreamsImpl() {
-    // PASS
-  }
-
-  const ReaderOptions& StripeStreamsImpl::getReaderOptions() const {
-    return reader.getReaderOptions();
-  }
-
-  const std::vector<bool> StripeStreamsImpl::getSelectedColumns() const {
-    return reader.getSelectedColumns();
-  }
-
-  proto::ColumnEncoding StripeStreamsImpl::getEncoding(int64_t columnId) const {
-    return footer.columns(static_cast<int>(columnId));
-  }
-
-  std::unique_ptr<SeekableInputStream>
-  StripeStreamsImpl::getStream(int64_t columnId,
-                               proto::Stream_Kind kind,
-                               bool shouldStream) const {
-    uint64_t offset = stripeStart;
-    for(int i = 0; i < footer.streams_size(); ++i) {
-      const proto::Stream& stream = footer.streams(i);
-      if (stream.has_kind() &&
-          stream.kind() == kind &&
-          stream.column() == static_cast<uint64_t>(columnId)) {
-        int64_t myBlock = static_cast<int64_t>(shouldStream ?
-                                         1024 * 1024 :
-                                         stream.length());
-        return createDecompressor(reader.getCompression(),
-                                  std::unique_ptr<SeekableInputStream>
-                                  (new SeekableFileInputStream
-                                   (&input,
-                                    offset,
-                                    stream.length(),
-                                    memoryPool,
-                                    myBlock)),
-                                  reader.getCompressionSize(),
-                                  memoryPool);
-      }
-      offset += stream.length();
-    }
-    return std::unique_ptr<SeekableInputStream>();
-  }
-
-  MemoryPool& StripeStreamsImpl::getMemoryPool() const {
-    return memoryPool;
-  }
-
-  void ReaderImpl::startNextStripe() {
-    currentStripeInfo = footer->stripes(static_cast<int>(currentStripe));
-    currentStripeFooter = getStripeFooter(currentStripeInfo);
-    rowsInCurrentStripe = currentStripeInfo.numberofrows();
-    StripeStreamsImpl stripeStreams(*this, currentStripeFooter,
-                                    currentStripeInfo.offset(),
-                                    *(stream.get()),
-                                    memoryPool);
-    reader = buildReader(*(schema.get()), stripeStreams);
-  }
-
-  void ReaderImpl::checkOrcVersion() {
-    std::string version = getFormatVersion();
-    if (version != "0.11" && version != "0.12") {
-      *(options.getErrorStream())
-        << "Warning: ORC file " << stream->getName()
-        << " was written in an unknown format version "
-        << version << "\n";
-    }
-  }
-
-  bool ReaderImpl::next(ColumnVectorBatch& data) {
-    if (currentStripe >= lastStripe) {
-      data.numElements = 0;
-      if (lastStripe > 0) {
-        previousRow = firstRowOfStripe[lastStripe - 1] +
-          footer->stripes(static_cast<int>(lastStripe - 1)).numberofrows();
-      } else {
-        previousRow = 0;
-      }
-      return false;
-    }
-    if (currentRowInStripe == 0) {
-      startNextStripe();
-    }
-    uint64_t rowsToRead =
-      std::min(static_cast<uint64_t>(data.capacity),
-               rowsInCurrentStripe - currentRowInStripe);
-    data.numElements = rowsToRead;
-    reader->next(data, rowsToRead, 0);
-    // update row number
-    previousRow = firstRowOfStripe[currentStripe] + currentRowInStripe;
-    currentRowInStripe += rowsToRead;
-    if (currentRowInStripe >= rowsInCurrentStripe) {
-      currentStripe += 1;
-      currentRowInStripe = 0;
-    }
-    return rowsToRead != 0;
-  }
-
-  std::unique_ptr<ColumnVectorBatch> ReaderImpl::createRowBatch
-  (const Type& type, uint64_t capacity) const {
-    ColumnVectorBatch* result = nullptr;
-    const Type* subtype;
-    switch (static_cast<int64_t>(type.getKind())) {
-    case BOOLEAN:
-    case BYTE:
-    case SHORT:
-    case INT:
-    case LONG:
-    case TIMESTAMP:
-    case DATE:
-      result = new LongVectorBatch(capacity, memoryPool);
-      break;
-    case FLOAT:
-    case DOUBLE:
-      result = new DoubleVectorBatch(capacity, memoryPool);
-      break;
-    case STRING:
-    case BINARY:
-    case CHAR:
-    case VARCHAR:
-      result = new StringVectorBatch(capacity, memoryPool);
-      break;
-    case STRUCT:
-      result = new StructVectorBatch(capacity, memoryPool);
-      for(uint64_t i=0; i < type.getSubtypeCount(); ++i) {
-        subtype = &(type.getSubtype(i));
-        if (selectedColumns[static_cast<size_t>(subtype->getColumnId())]) {
-          dynamic_cast<StructVectorBatch*>(result)->fields.push_back
-            (createRowBatch(*subtype, capacity).release());
-        }
-      }
-      break;
-    case LIST:
-      result = new ListVectorBatch(capacity, memoryPool);
-      subtype = &(type.getSubtype(0));
-      if (selectedColumns[static_cast<size_t>(subtype->getColumnId())]) {
-        dynamic_cast<ListVectorBatch*>(result)->elements =
-          createRowBatch(*subtype, capacity);
-      }
-      break;
-    case MAP:
-      result = new MapVectorBatch(capacity, memoryPool);
-      subtype = &(type.getSubtype(0));
-      if (selectedColumns[static_cast<size_t>(subtype->getColumnId())]) {
-        dynamic_cast<MapVectorBatch*>(result)->keys =
-          createRowBatch(*subtype, capacity);
-      }
-      subtype = &(type.getSubtype(1));
-      if (selectedColumns[static_cast<size_t>(subtype->getColumnId())]) {
-        dynamic_cast<MapVectorBatch*>(result)->elements =
-          createRowBatch(*subtype, capacity);
-      }
-      break;
-    case DECIMAL:
-      if (type.getPrecision() == 0 || type.getPrecision() > 18) {
-        result = new Decimal128VectorBatch(capacity, memoryPool);
-      } else {
-        result = new Decimal64VectorBatch(capacity, memoryPool);
-      }
-      break;
-    case UNION:
-      result = new UnionVectorBatch(capacity, memoryPool);
-      for(uint64_t i=0; i < type.getSubtypeCount(); ++i) {
-        subtype = &(type.getSubtype(i));
-        if (selectedColumns[static_cast<size_t>(subtype->getColumnId())]) {
-          dynamic_cast<UnionVectorBatch*>(result)->children.push_back
-            (createRowBatch(*subtype, capacity).release());
-        }
-      }
-      break;
-    default:
-      throw NotImplementedYet("not supported yet");
-    }
-    return std::unique_ptr<ColumnVectorBatch>(result);
-  }
-
-  std::unique_ptr<ColumnVectorBatch> ReaderImpl::createRowBatch
-                                              (uint64_t capacity) const {
-    return createRowBatch(*(schema.get()), capacity);
-  }
-
-  void ensureOrcFooter(InputStream* stream,
-                       DataBuffer<char> *buffer,
-                       uint64_t postscriptLength) {
-
-    const std::string MAGIC("ORC");
-    const uint64_t magicLength = MAGIC.length();
-    const char * const bufferStart = buffer->data();
-    const uint64_t bufferLength = buffer->size();
-
-    if (postscriptLength < magicLength || bufferLength < magicLength) {
-      throw ParseError("Invalid ORC postscript length");
-    }
-    const char* magicStart = bufferStart + bufferLength - 1 - magicLength;
-
-    // Look for the magic string at the end of the postscript.
-    if (memcmp(magicStart, MAGIC.c_str(), magicLength) != 0) {
-      // If there is no magic string at the end, check the beginning.
-      // Only files written by Hive 0.11.0 don't have the tail ORC string.
-      char *frontBuffer = new char[magicLength];
-      stream->read(frontBuffer, magicLength, 0);
-      bool foundMatch = memcmp(frontBuffer, MAGIC.c_str(), magicLength) == 0;
-      delete[] frontBuffer;
-      if (!foundMatch) {
-        throw ParseError("Not an ORC file");
-      }
-    }
-  }
-
-  /**
-   * Read the file's postscript from the given buffer.
-   * @param stream the file stream
-   * @param buffer the buffer with the tail of the file.
-   * @param postscriptSize the length of postscript in bytes
-   */
-  std::unique_ptr<proto::PostScript> readPostscript(InputStream *stream,
-                                                    DataBuffer<char> *buffer,
-                                                    uint64_t postscriptSize) {
-    char *ptr = buffer->data();
-    uint64_t readSize = buffer->size();
-
-    ensureOrcFooter(stream, buffer, postscriptSize);
-
-    std::unique_ptr<proto::PostScript> postscript =
-      std::unique_ptr<proto::PostScript>(new proto::PostScript());
-    if (!postscript->ParseFromArray(ptr + readSize - 1 - postscriptSize,
-                                   static_cast<int>(postscriptSize))) {
-      throw ParseError("Failed to parse the postscript from " +
-                       stream->getName());
-    }
-    return std::move(postscript);
-  }
-
-  /**
-   * Parse the footer from the given buffer.
-   * @param stream the file's stream
-   * @param buffer the buffer to parse the footer from
-   * @param footerOffset the offset within the buffer that contains the footer
-   * @param ps the file's postscript
-   * @param memoryPool the memory pool to use
-   */
-  std::unique_ptr<proto::Footer> readFooter(InputStream* stream,
-                                            DataBuffer<char> *&buffer,
-                                            uint64_t footerOffset,
-                                            const proto::PostScript& ps,
-                                            MemoryPool& memoryPool) {
-    char *footerPtr = buffer->data() + footerOffset;
-
-    std::unique_ptr<SeekableInputStream> pbStream =
-      createDecompressor(convertCompressionKind(ps),
-                         std::unique_ptr<SeekableInputStream>
-                         (new SeekableArrayInputStream(footerPtr,
-                                                       ps.footerlength())),
-                         getCompressionBlockSize(ps),
-                         memoryPool);
-
-    std::unique_ptr<proto::Footer> footer =
-      std::unique_ptr<proto::Footer>(new proto::Footer());
-    if (!footer->ParseFromZeroCopyStream(pbStream.get())) {
-      throw ParseError("Failed to parse the footer from " +
-                       stream->getName());
-    }
-    return std::move(footer);
-  }
-
-  std::unique_ptr<Reader> createReader(std::unique_ptr<InputStream> stream,
-                                       const ReaderOptions& options) {
-    MemoryPool *memoryPool = options.getMemoryPool();
-    std::unique_ptr<proto::PostScript> ps;
-    std::unique_ptr<proto::Footer> footer;
-    uint64_t footerStart;
-    std::string serializedFooter = options.getSerializedFileTail();
-    if (serializedFooter.length() != 0) {
-      // Parse the file tail from the serialized one.
-      proto::FileTail tail;
-      if (!tail.ParseFromString(serializedFooter)) {
-        throw ParseError("Failed to parse the file tail from string");
-      }
-      ps.reset(new proto::PostScript(tail.postscript()));
-      footer.reset(new proto::Footer(tail.footer()));
-      footerStart = tail.footerstart();
-    } else {
-      // figure out the size of the file using the option or filesystem
-      uint64_t size = std::min(options.getTailLocation(),
-                               static_cast<uint64_t>(stream->getLength()));
-
-      //read last bytes into buffer to get PostScript
-      uint64_t readSize = std::min(size, DIRECTORY_SIZE_GUESS);
-      if (readSize < 4) {
-        throw ParseError("File size too small");
-      }
-      DataBuffer<char> *buffer = new DataBuffer<char>(*memoryPool, readSize);
-      stream->read(buffer->data(), readSize, size - readSize);
-
-      uint64_t postscriptSize = buffer->data()[readSize - 1] & 0xff;
-      ps = readPostscript(stream.get(), buffer, postscriptSize);
-      uint64_t footerSize = ps->footerlength();
-      uint64_t tailSize = 1 + postscriptSize + footerSize;
-      footerStart = size - tailSize;
-      uint64_t footerOffset;
-
-      if (tailSize > readSize) {
-        buffer->resize(footerSize);
-        stream->read(buffer->data(), footerSize, size - tailSize);
-        footerOffset = 0;
-      } else {
-        footerOffset = readSize - tailSize;
-      }
-
-      footer = readFooter(stream.get(), buffer, footerOffset, *ps,
-                          *memoryPool);
-      delete buffer;
-    }
-    return std::unique_ptr<Reader>(new ReaderImpl(std::move(stream),
-                                                  options,
-                                                  std::move(ps),
-                                                  std::move(footer),
-                                                  footerStart));
-  }
-
-  ColumnStatistics::~ColumnStatistics() {
-    // PASS
-  }
-
-  BinaryColumnStatistics::~BinaryColumnStatistics() {
-    // PASS
-  }
-
-  BooleanColumnStatistics::~BooleanColumnStatistics() {
-    // PASS
-  }
-
-  DateColumnStatistics::~DateColumnStatistics() {
-    // PASS
-  }
-
-  DecimalColumnStatistics::~DecimalColumnStatistics() {
-    // PASS
-  }
-
-  DoubleColumnStatistics::~DoubleColumnStatistics() {
-    // PASS
-  }
-
-  IntegerColumnStatistics::~IntegerColumnStatistics() {
-    // PASS
-  }
-
-  StringColumnStatistics::~StringColumnStatistics() {
-    // PASS
-  }
-
-  TimestampColumnStatistics::~TimestampColumnStatistics() {
-    // PASS
-  }
-
-  ColumnStatisticsImpl::~ColumnStatisticsImpl() {
-    // PASS
-  }
-
-  BinaryColumnStatisticsImpl::~BinaryColumnStatisticsImpl() {
-    // PASS
-  }
-
-  BooleanColumnStatisticsImpl::~BooleanColumnStatisticsImpl() {
-    // PASS
-  }
-
-  DateColumnStatisticsImpl::~DateColumnStatisticsImpl() {
-    // PASS
-  }
-
-  DecimalColumnStatisticsImpl::~DecimalColumnStatisticsImpl() {
-    // PASS
-  }
-
-  DoubleColumnStatisticsImpl::~DoubleColumnStatisticsImpl() {
-    // PASS
-  }
-
-  IntegerColumnStatisticsImpl::~IntegerColumnStatisticsImpl() {
-    // PASS
-  }
-
-  StringColumnStatisticsImpl::~StringColumnStatisticsImpl() {
-    // PASS
-  }
-
-  TimestampColumnStatisticsImpl::~TimestampColumnStatisticsImpl() {
-    // PASS
-  }
-
-  ColumnStatisticsImpl::ColumnStatisticsImpl
-  (const proto::ColumnStatistics& pb) {
-    valueCount = pb.numberofvalues();
-  }
-
-  BinaryColumnStatisticsImpl::BinaryColumnStatisticsImpl
-  (const proto::ColumnStatistics& pb, bool correctStats){
-    valueCount = pb.numberofvalues();
-    if (!pb.has_binarystatistics() || !correctStats) {
-      _hasTotalLength = false;
-    }else{
-      _hasTotalLength = pb.binarystatistics().has_sum();
-      totalLength = static_cast<uint64_t>(pb.binarystatistics().sum());
-    }
-  }
-
-  BooleanColumnStatisticsImpl::BooleanColumnStatisticsImpl
-  (const proto::ColumnStatistics& pb, bool correctStats){
-    valueCount = pb.numberofvalues();
-    if (!pb.has_bucketstatistics() || !correctStats) {
-      _hasCount = false;
-    }else{
-      _hasCount = true;
-      trueCount = pb.bucketstatistics().count(0);
-    }
-  }
-
-  DateColumnStatisticsImpl::DateColumnStatisticsImpl
-  (const proto::ColumnStatistics& pb, bool correctStats){
-    valueCount = pb.numberofvalues();
-    if (!pb.has_datestatistics() || !correctStats) {
-      _hasMinimum = false;
-      _hasMaximum = false;
-    }else{
-        _hasMinimum = pb.datestatistics().has_minimum();
-        _hasMaximum = pb.datestatistics().has_maximum();
-        minimum = pb.datestatistics().minimum();
-        maximum = pb.datestatistics().maximum();
-    }
-  }
-
-  DecimalColumnStatisticsImpl::DecimalColumnStatisticsImpl
-  (const proto::ColumnStatistics& pb, bool correctStats){
-    valueCount = pb.numberofvalues();
-    if (!pb.has_decimalstatistics() || !correctStats) {
-      _hasMinimum = false;
-      _hasMaximum = false;
-      _hasSum = false;
-    }else{
-      const proto::DecimalStatistics& stats = pb.decimalstatistics();
-      _hasMinimum = stats.has_minimum();
-      _hasMaximum = stats.has_maximum();
-      _hasSum = stats.has_sum();
-
-      minimum = stats.minimum();
-      maximum = stats.maximum();
-      sum = stats.sum();
-    }
-  }
-
-  DoubleColumnStatisticsImpl::DoubleColumnStatisticsImpl
-  (const proto::ColumnStatistics& pb){
-    valueCount = pb.numberofvalues();
-    if (!pb.has_doublestatistics()) {
-      _hasMinimum = false;
-      _hasMaximum = false;
-      _hasSum = false;
-    }else{
-      const proto::DoubleStatistics& stats = pb.doublestatistics();
-      _hasMinimum = stats.has_minimum();
-      _hasMaximum = stats.has_maximum();
-      _hasSum = stats.has_sum();
-
-      minimum = stats.minimum();
-      maximum = stats.maximum();
-      sum = stats.sum();
-    }
-  }
-
-  IntegerColumnStatisticsImpl::IntegerColumnStatisticsImpl
-  (const proto::ColumnStatistics& pb){
-    valueCount = pb.numberofvalues();
-    if (!pb.has_intstatistics()) {
-      _hasMinimum = false;
-      _hasMaximum = false;
-      _hasSum = false;
-    }else{
-      const proto::IntegerStatistics& stats = pb.intstatistics();
-      _hasMinimum = stats.has_minimum();
-      _hasMaximum = stats.has_maximum();
-      _hasSum = stats.has_sum();
-
-      minimum = stats.minimum();
-      maximum = stats.maximum();
-      sum = stats.sum();
-    }
-  }
-
-  StringColumnStatisticsImpl::StringColumnStatisticsImpl
-  (const proto::ColumnStatistics& pb, bool correctStats){
-    valueCount = pb.numberofvalues();
-    if (!pb.has_stringstatistics() || !correctStats) {
-      _hasMinimum = false;
-      _hasMaximum = false;
-      _hasTotalLength = false;
-    }else{
-      const proto::StringStatistics& stats = pb.stringstatistics();
-      _hasMinimum = stats.has_minimum();
-      _hasMaximum = stats.has_maximum();
-      _hasTotalLength = stats.has_sum();
-
-      minimum = stats.minimum();
-      maximum = stats.maximum();
-      totalLength = static_cast<uint64_t>(stats.sum());
-    }
-  }
-
-  TimestampColumnStatisticsImpl::TimestampColumnStatisticsImpl
-  (const proto::ColumnStatistics& pb, bool correctStats){
-    valueCount = pb.numberofvalues();
-    if (!pb.has_timestampstatistics() || !correctStats) {
-      _hasMinimum = false;
-      _hasMaximum = false;
-    }else{
-      const proto::TimestampStatistics& stats = pb.timestampstatistics();
-      _hasMinimum = stats.has_minimum();
-      _hasMaximum = stats.has_maximum();
-
-      minimum = stats.minimum();
-      maximum = stats.maximum();
-    }
-  }
-
-}// namespace

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/TypeImpl.cc
----------------------------------------------------------------------
diff --git a/c++/src/orc/TypeImpl.cc b/c++/src/orc/TypeImpl.cc
deleted file mode 100644
index 8168cb3..0000000
--- a/c++/src/orc/TypeImpl.cc
+++ /dev/null
@@ -1,369 +0,0 @@
-/**
- * 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 "orc/Adaptor.hh"
-#include "Exceptions.hh"
-#include "TypeImpl.hh"
-
-#include <iostream>
-#include <sstream>
-
-namespace orc {
-
-  Type::~Type() {
-    // PASS
-  }
-
-  TypeImpl::TypeImpl(TypeKind _kind) {
-    columnId = 0;
-    kind = _kind;
-    maxLength = 0;
-    precision = 0;
-    scale = 0;
-    subtypeCount = 0;
-  }
-
-  TypeImpl::TypeImpl(TypeKind _kind, uint64_t _maxLength) {
-    columnId = 0;
-    kind = _kind;
-    maxLength = _maxLength;
-    precision = 0;
-    scale = 0;
-    subtypeCount = 0;
-  }
-
-  TypeImpl::TypeImpl(TypeKind _kind, uint64_t _precision,
-                     uint64_t _scale) {
-    columnId = 0;
-    kind = _kind;
-    maxLength = 0;
-    precision = _precision;
-    scale = _scale;
-    subtypeCount = 0;
-  }
-
-  TypeImpl::TypeImpl(TypeKind _kind,
-                     const std::vector<Type*>& types,
-                     const std::vector<std::string>& _fieldNames) {
-    columnId = 0;
-    kind = _kind;
-    maxLength = 0;
-    precision = 0;
-    scale = 0;
-    subtypeCount = static_cast<uint64_t>(types.size());
-    subTypes.assign(types.begin(), types.end());
-    fieldNames.assign(_fieldNames.begin(), _fieldNames.end());
-  }
-
-  TypeImpl::TypeImpl(TypeKind _kind, const std::vector<Type*>& types) {
-    columnId = 0;
-    kind = _kind;
-    maxLength = 0;
-    precision = 0;
-    scale = 0;
-    subtypeCount = static_cast<uint64_t>(types.size());
-    subTypes.assign(types.begin(), types.end());
-  }
-
-  int64_t TypeImpl::assignIds(int64_t root) {
-    columnId = root;
-    int64_t current = root + 1;
-    for(uint64_t i=0; i < subtypeCount; ++i) {
-      current = subTypes[i]->assignIds(current);
-    }
-    return current;
-  }
-
-  TypeImpl::~TypeImpl() {
-    for (std::vector<Type*>::iterator it = subTypes.begin();
-        it != subTypes.end(); it++) {
-      delete (*it) ;
-    }
-  }
-
-  int64_t TypeImpl::getColumnId() const {
-    return columnId;
-  }
-
-  TypeKind TypeImpl::getKind() const {
-    return kind;
-  }
-
-  uint64_t TypeImpl::getSubtypeCount() const {
-    return subtypeCount;
-  }
-
-  const Type& TypeImpl::getSubtype(uint64_t i) const {
-    return *(subTypes[i]);
-  }
-
-  const std::string& TypeImpl::getFieldName(uint64_t i) const {
-    return fieldNames[i];
-  }
-
-  uint64_t TypeImpl::getMaximumLength() const {
-    return maxLength;
-  }
-
-  uint64_t TypeImpl::getPrecision() const {
-    return precision;
-  }
-
-  uint64_t TypeImpl::getScale() const {
-    return scale;
-  }
-
-  Type& TypeImpl::addStructField(std::unique_ptr<Type> fieldType,
-                                 const std::string& fieldName) {
-    Type* result = fieldType.release();
-    subTypes.push_back(result);
-    fieldNames.push_back(fieldName);
-    subtypeCount += 1;
-    return *result;
-  }
-
-  std::string TypeImpl::toString() const {
-    switch (static_cast<int64_t>(kind)) {
-    case BOOLEAN:
-      return "boolean";
-    case BYTE:
-      return "tinyint";
-    case SHORT:
-      return "smallint";
-    case INT:
-      return "int";
-    case LONG:
-      return "bigint";
-    case FLOAT:
-      return "float";
-    case DOUBLE:
-      return "double";
-    case STRING:
-      return "string";
-    case BINARY:
-      return "binary";
-    case TIMESTAMP:
-      return "timestamp";
-    case LIST:
-      return "array<" + subTypes[0]->toString() + ">";
-    case MAP:
-      return "map<" + subTypes[0]->toString() + "," +
-        subTypes[1]->toString() +  ">";
-    case STRUCT: {
-      std::string result = "struct<";
-      for(size_t i=0; i < subTypes.size(); ++i) {
-        if (i != 0) {
-          result += ",";
-        }
-        result += fieldNames[i];
-        result += ":";
-        result += subTypes[i]->toString();
-      }
-      result += ">";
-      return result;
-    }
-    case UNION: {
-      std::string result = "uniontype<";
-      for(size_t i=0; i < subTypes.size(); ++i) {
-        if (i != 0) {
-          result += ",";
-        }
-        result += subTypes[i]->toString();
-      }
-      result += ">";
-      return result;
-    }
-    case DECIMAL: {
-      std::stringstream result;
-      result << "decimal(" << precision << "," << scale << ")";
-      return result.str();
-    }
-    case DATE:
-      return "date";
-    case VARCHAR: {
-      std::stringstream result;
-      result << "varchar(" << maxLength << ")";
-      return result.str();
-    }
-    case CHAR: {
-      std::stringstream result;
-      result << "char(" << maxLength << ")";
-      return result.str();
-    }
-    default:
-      throw NotImplementedYet("Unknown type");
-    }
-  }
-
-  std::unique_ptr<Type> createPrimitiveType(TypeKind kind) {
-    return std::unique_ptr<Type>(new TypeImpl(kind));
-  }
-
-  std::unique_ptr<Type> createCharType(TypeKind kind,
-                                       uint64_t maxLength) {
-    return std::unique_ptr<Type>(new TypeImpl(kind, maxLength));
-  }
-
-  std::unique_ptr<Type> createDecimalType(uint64_t precision,
-                                          uint64_t scale) {
-    return std::unique_ptr<Type>(new TypeImpl(DECIMAL, precision, scale));
-  }
-
-  std::unique_ptr<Type> createStructType() {
-    return std::unique_ptr<Type>(new TypeImpl(STRUCT));
-  }
-
-  std::unique_ptr<Type>
-      createStructType(std::vector<Type*> types,
-                       std::vector<std::string> fieldNames) {
-    std::vector<Type*> typeVector(types.begin(), types.end());
-    std::vector<std::string> fieldVector(fieldNames.begin(), fieldNames.end());
-
-    return std::unique_ptr<Type>(new TypeImpl(STRUCT, typeVector,
-                                              fieldVector));
-  }
-
-#ifdef ORC_CXX_HAS_INITIALIZER_LIST
-  std::unique_ptr<Type> createStructType(
-      std::initializer_list<std::unique_ptr<Type> > types,
-      std::initializer_list<std::string> fieldNames) {
-    std::vector<Type*> typeVector(types.size());
-    std::vector<std::string> fieldVector(types.size());
-    auto currentType = types.begin();
-    auto endType = types.end();
-    size_t current = 0;
-    while (currentType != endType) {
-      typeVector[current++] =
-          const_cast<std::unique_ptr<Type>*>(currentType)->release();
-      ++currentType;
-    }
-    fieldVector.insert(fieldVector.end(), fieldNames.begin(),
-        fieldNames.end());
-    return std::unique_ptr<Type>(new TypeImpl(STRUCT, typeVector,
-        fieldVector));
-  }
-#endif
-
-  std::unique_ptr<Type> createListType(std::unique_ptr<Type> elements) {
-    std::vector<Type*> subtypes(1);
-    subtypes[0] = elements.release();
-    return std::unique_ptr<Type>(new TypeImpl(LIST, subtypes));
-  }
-
-  std::unique_ptr<Type> createMapType(std::unique_ptr<Type> key,
-                                      std::unique_ptr<Type> value) {
-    std::vector<Type*> subtypes(2);
-    subtypes[0] = key.release();
-    subtypes[1] = value.release();
-    return std::unique_ptr<Type>(new TypeImpl(MAP, subtypes));
-  }
-
-  std::unique_ptr<Type>
-      createUnionType(std::vector<Type*> types) {
-    std::vector<Type*> typeVector(types.begin(), types.end());
-    return std::unique_ptr<Type>(new TypeImpl(UNION, typeVector));
-  }
-
-  std::string printProtobufMessage(const google::protobuf::Message& message);
-  std::unique_ptr<Type> convertType(const proto::Type& type,
-                                    const proto::Footer& footer) {
-    switch (static_cast<int64_t>(type.kind())) {
-
-    case proto::Type_Kind_BOOLEAN:
-    case proto::Type_Kind_BYTE:
-    case proto::Type_Kind_SHORT:
-    case proto::Type_Kind_INT:
-    case proto::Type_Kind_LONG:
-    case proto::Type_Kind_FLOAT:
-    case proto::Type_Kind_DOUBLE:
-    case proto::Type_Kind_STRING:
-    case proto::Type_Kind_BINARY:
-    case proto::Type_Kind_TIMESTAMP:
-    case proto::Type_Kind_DATE:
-      return std::unique_ptr<Type>
-        (new TypeImpl(static_cast<TypeKind>(type.kind())));
-
-    case proto::Type_Kind_CHAR:
-    case proto::Type_Kind_VARCHAR:
-      return std::unique_ptr<Type>
-        (new TypeImpl(static_cast<TypeKind>(type.kind()),
-                      type.maximumlength()));
-
-    case proto::Type_Kind_DECIMAL:
-      return std::unique_ptr<Type>
-        (new TypeImpl(DECIMAL, type.precision(), type.scale()));
-
-    case proto::Type_Kind_LIST:
-    case proto::Type_Kind_MAP:
-    case proto::Type_Kind_UNION: {
-      uint64_t size = static_cast<uint64_t>(type.subtypes_size());
-      std::vector<Type*> typeList(size);
-      for(int i=0; i < type.subtypes_size(); ++i) {
-        typeList[static_cast<uint64_t>(i)] =
-          convertType(footer.types(static_cast<int>(type.subtypes(i))),
-                      footer).release();
-      }
-      return std::unique_ptr<Type>
-        (new TypeImpl(static_cast<TypeKind>(type.kind()), typeList));
-    }
-
-    case proto::Type_Kind_STRUCT: {
-      uint64_t size = static_cast<uint64_t>(type.subtypes_size());
-      std::vector<Type*> typeList(size);
-      std::vector<std::string> fieldList(size);
-      for(int i=0; i < type.subtypes_size(); ++i) {
-        typeList[static_cast<uint64_t>(i)] =
-          convertType(footer.types(static_cast<int>(type.subtypes(i))),
-                      footer).release();
-        fieldList[static_cast<uint64_t>(i)] = type.fieldnames(i);
-      }
-      return std::unique_ptr<Type>
-        (new TypeImpl(STRUCT, typeList, fieldList));
-    }
-    default:
-      throw NotImplementedYet("Unknown type kind");
-    }
-  }
-
-  std::string kind2String(TypeKind t) {
-      std::string name ;
-      switch(static_cast<int64_t>(t)) {
-        case BOOLEAN: { name = "BOOLEAN"; break; }
-        case BYTE: { name = "TINYINT"; break; }
-        case SHORT: { name = "SMALLINT"; break; }
-        case INT: { name = "INT"; break; }
-        case LONG: { name = "BIGINT"; break; }
-        case FLOAT: { name = "FLOAT"; break; }
-        case DOUBLE: { name = "DOUBLE"; break; }
-        case STRING: { name = "STRING"; break; }
-        case BINARY: { name = "BINARY"; break; }
-        case TIMESTAMP: { name = "TIMESTAMP"; break; }
-        case LIST: { name = "LIST"; break; }
-        case MAP: { name = "MAP"; break; }
-        case STRUCT: { name = "STRUCT"; break; }
-        case UNION: { name = "UNION"; break; }
-        case DECIMAL: { name = "DECIMAL"; break; }
-        case DATE: { name = "DATE"; break; }
-        case VARCHAR: { name = "VARCHAR"; break; }
-        case CHAR: { name = "CHAR"; break; }
-        default: { name = "UNKNOWN"; break; }
-      }
-      return name ;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/TypeImpl.hh
----------------------------------------------------------------------
diff --git a/c++/src/orc/TypeImpl.hh b/c++/src/orc/TypeImpl.hh
deleted file mode 100644
index 313cbe7..0000000
--- a/c++/src/orc/TypeImpl.hh
+++ /dev/null
@@ -1,101 +0,0 @@
-/**
- * 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 TYPE_IMPL_HH
-#define TYPE_IMPL_HH
-
-#include "orc/Adaptor.hh"
-#include "orc/Vector.hh"
-
-#include "wrap/orc-proto-wrapper.hh"
-
-#include <vector>
-
-namespace orc {
-
-  class TypeImpl: public Type {
-  private:
-    int64_t columnId;
-    TypeKind kind;
-    std::vector<Type*> subTypes;
-    std::vector<std::string> fieldNames;
-    uint64_t subtypeCount;
-    uint64_t maxLength;
-    uint64_t precision;
-    uint64_t scale;
-
-  public:
-    /**
-     * Create most of the primitive types.
-     */
-    TypeImpl(TypeKind kind);
-
-    /**
-     * Create char and varchar type.
-     */
-    TypeImpl(TypeKind kind, uint64_t maxLength);
-
-    /**
-     * Create decimal type.
-     */
-    TypeImpl(TypeKind kind, uint64_t precision,
-             uint64_t scale);
-
-    /**
-     * Create struct type.
-     */
-    TypeImpl(TypeKind kind,
-             const std::vector<Type*>& types,
-             const std::vector<std::string>& fieldNames);
-
-    /**
-     * Create list, map, and union type.
-     */
-    TypeImpl(TypeKind kind, const std::vector<Type*>& types);
-
-    virtual ~TypeImpl();
-
-    int64_t assignIds(int64_t root) override;
-
-    int64_t getColumnId() const override;
-
-    TypeKind getKind() const override;
-
-    uint64_t getSubtypeCount() const override;
-
-    const Type& getSubtype(uint64_t i) const override;
-
-    const std::string& getFieldName(uint64_t i) const override;
-
-    uint64_t getMaximumLength() const override;
-
-    uint64_t getPrecision() const override;
-
-    uint64_t getScale() const override;
-
-    std::string toString() const override;
-
-    Type& addStructField(std::unique_ptr<Type> fieldType,
-                         const std::string& fieldName) override;
-  };
-
-  std::unique_ptr<Type> convertType(const proto::Type& type,
-                                    const proto::Footer& footer);
-}
-
-#endif

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/orc/Vector.cc
----------------------------------------------------------------------
diff --git a/c++/src/orc/Vector.cc b/c++/src/orc/Vector.cc
deleted file mode 100644
index d2c0a4b..0000000
--- a/c++/src/orc/Vector.cc
+++ /dev/null
@@ -1,305 +0,0 @@
-/**
- * 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 "Exceptions.hh"
-#include "orc/Vector.hh"
-#include "orc/Adaptor.hh"
-
-#include <iostream>
-#include <sstream>
-#include <cstdlib>
-
-namespace orc {
-
-  ColumnVectorBatch::ColumnVectorBatch(uint64_t cap,
-                                       MemoryPool& pool
-                                       ): capacity(cap),
-                                          numElements(0),
-                                          notNull(pool, cap),
-                                          hasNulls(false),
-                                          memoryPool(pool) {
-    // PASS
-  }
-
-  ColumnVectorBatch::~ColumnVectorBatch() {
-    // PASS
-  }
-
-  void ColumnVectorBatch::resize(uint64_t cap) {
-    if (capacity < cap) {
-      capacity = cap;
-      notNull.resize(cap);
-    }
-  }
-
-  LongVectorBatch::LongVectorBatch(uint64_t capacity, MemoryPool& pool
-                     ): ColumnVectorBatch(capacity, pool),
-                        data(pool, capacity) {
-    // PASS
-  }
-
-  LongVectorBatch::~LongVectorBatch() {
-    // PASS
-  }
-
-  std::string LongVectorBatch::toString() const {
-    std::ostringstream buffer;
-    buffer << "Long vector <" << numElements << " of " << capacity << ">";
-    return buffer.str();
-  }
-
-  void LongVectorBatch::resize(uint64_t cap) {
-    if (capacity < cap) {
-      ColumnVectorBatch::resize(cap);
-      data.resize(cap);
-    }
-  }
-
-  DoubleVectorBatch::DoubleVectorBatch(uint64_t capacity, MemoryPool& pool
-                   ): ColumnVectorBatch(capacity, pool),
-                      data(pool, capacity) {
-    // PASS
-  }
-
-  DoubleVectorBatch::~DoubleVectorBatch() {
-    // PASS
-  }
-
-  std::string DoubleVectorBatch::toString() const {
-    std::ostringstream buffer;
-    buffer << "Double vector <" << numElements << " of " << capacity << ">";
-    return buffer.str();
-  }
-
-  void DoubleVectorBatch::resize(uint64_t cap) {
-    if (capacity < cap) {
-      ColumnVectorBatch::resize(cap);
-      data.resize(cap);
-    }
-  }
-
-  StringVectorBatch::StringVectorBatch(uint64_t capacity, MemoryPool& pool
-               ): ColumnVectorBatch(capacity, pool),
-                  data(pool, capacity),
-                  length(pool, capacity) {
-    // PASS
-  }
-
-  StringVectorBatch::~StringVectorBatch() {
-    // PASS
-  }
-
-  std::string StringVectorBatch::toString() const {
-    std::ostringstream buffer;
-    buffer << "Byte vector <" << numElements << " of " << capacity << ">";
-    return buffer.str();
-  }
-
-  void StringVectorBatch::resize(uint64_t cap) {
-    if (capacity < cap) {
-      ColumnVectorBatch::resize(cap);
-      data.resize(cap);
-      length.resize(cap);
-    }
-  }
-
-  StructVectorBatch::StructVectorBatch(uint64_t cap, MemoryPool& pool
-                                        ): ColumnVectorBatch(cap, pool) {
-    // PASS
-  }
-
-  StructVectorBatch::~StructVectorBatch() {
-    for (uint64_t i=0; i<this->fields.size(); i++) {
-      delete this->fields[i];
-    }
-  }
-
-  std::string StructVectorBatch::toString() const {
-    std::ostringstream buffer;
-    buffer << "Struct vector <" << numElements << " of " << capacity
-           << "; ";
-    for(std::vector<ColumnVectorBatch*>::const_iterator ptr=fields.begin();
-        ptr != fields.end(); ++ptr) {
-      buffer << (*ptr)->toString() << "; ";
-    }
-    buffer << ">";
-    return buffer.str();
-  }
-
-
-  void StructVectorBatch::resize(uint64_t cap) {
-    ColumnVectorBatch::resize(cap);
-  }
-
-  ListVectorBatch::ListVectorBatch(uint64_t cap, MemoryPool& pool
-                   ): ColumnVectorBatch(cap, pool),
-                      offsets(pool, cap+1) {
-    // PASS
-  }
-
-  ListVectorBatch::~ListVectorBatch() {
-    // PASS
-  }
-
-  std::string ListVectorBatch::toString() const {
-    std::ostringstream buffer;
-    buffer << "List vector <" << elements->toString() << " with "
-           << numElements << " of " << capacity << ">";
-    return buffer.str();
-  }
-
-  void ListVectorBatch::resize(uint64_t cap) {
-    if (capacity < cap) {
-      ColumnVectorBatch::resize(cap);
-      offsets.resize(cap + 1);
-    }
-  }
-
-  MapVectorBatch::MapVectorBatch(uint64_t cap, MemoryPool& pool
-                 ): ColumnVectorBatch(cap, pool),
-                    offsets(pool, cap+1) {
-    // PASS
-  }
-
-  MapVectorBatch::~MapVectorBatch() {
-    // PASS
-  }
-
-  std::string MapVectorBatch::toString() const {
-    std::ostringstream buffer;
-    buffer << "Map vector <" << keys->toString() << ", "
-           << elements->toString() << " with "
-           << numElements << " of " << capacity << ">";
-    return buffer.str();
-  }
-
-  void MapVectorBatch::resize(uint64_t cap) {
-    if (capacity < cap) {
-      ColumnVectorBatch::resize(cap);
-      offsets.resize(cap + 1);
-    }
-  }
-
-  UnionVectorBatch::UnionVectorBatch(uint64_t cap, MemoryPool& pool
-                                     ): ColumnVectorBatch(cap, pool),
-                                        tags(pool, cap),
-                                        offsets(pool, cap) {
-    // PASS
-  }
-
-  UnionVectorBatch::~UnionVectorBatch() {
-    for (uint64_t i=0; i < children.size(); i++) {
-      delete children[i];
-    }
-  }
-
-  std::string UnionVectorBatch::toString() const {
-    std::ostringstream buffer;
-    buffer << "Union vector <";
-    for(size_t i=0; i < children.size(); ++i) {
-      if (i != 0) {
-        buffer << ", ";
-      }
-      buffer << children[i]->toString();
-    }
-    buffer << "; with " << numElements << " of " << capacity << ">";
-    return buffer.str();
-  }
-
-  void UnionVectorBatch::resize(uint64_t cap) {
-    if (capacity < cap) {
-      ColumnVectorBatch::resize(cap);
-      tags.resize(cap);
-      offsets.resize(cap);
-    }
-  }
-
-  Decimal64VectorBatch::Decimal64VectorBatch(uint64_t cap, MemoryPool& pool
-                 ): ColumnVectorBatch(cap, pool),
-                    values(pool, cap),
-                    readScales(pool, cap) {
-    // PASS
-  }
-
-  Decimal64VectorBatch::~Decimal64VectorBatch() {
-    // PASS
-  }
-
-  std::string Decimal64VectorBatch::toString() const {
-    std::ostringstream buffer;
-    buffer << "Decimal64 vector  with "
-           << numElements << " of " << capacity << ">";
-    return buffer.str();
-  }
-
-  void Decimal64VectorBatch::resize(uint64_t cap) {
-    if (capacity < cap) {
-      ColumnVectorBatch::resize(cap);
-      values.resize(cap);
-      readScales.resize(cap);
-    }
-  }
-
-  Decimal128VectorBatch::Decimal128VectorBatch(uint64_t cap, MemoryPool& pool
-               ): ColumnVectorBatch(cap, pool),
-                  values(pool, cap),
-                  readScales(pool, cap) {
-    // PASS
-  }
-
-  Decimal128VectorBatch::~Decimal128VectorBatch() {
-    // PASS
-  }
-
-  std::string Decimal128VectorBatch::toString() const {
-    std::ostringstream buffer;
-    buffer << "Decimal128 vector  with "
-           << numElements << " of " << capacity << ">";
-    return buffer.str();
-  }
-
-  void Decimal128VectorBatch::resize(uint64_t cap) {
-    if (capacity < cap) {
-      ColumnVectorBatch::resize(cap);
-      values.resize(cap);
-      readScales.resize(cap);
-    }
-  }
-
-  Decimal::Decimal(const Int128& _value,
-                   int32_t _scale): value(_value), scale(_scale) {
-    // PASS
-  }
-
-  Decimal::Decimal(const std::string& str) {
-    std::size_t foundPoint = str.find(".");
-    // no decimal point, it is int
-    if(foundPoint == std::string::npos){
-      value = Int128(str);
-      scale = 0;
-    }else{
-      std::string copy(str);
-      scale = static_cast<int32_t>(str.length() - foundPoint);
-      value = Int128(copy.replace(foundPoint, 1, ""));
-    }
-  }
-
-  std::string Decimal::toString() const {
-    return value.toDecimalString(scale);
-  }
-}

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/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 8fc3ec4..1155dd7 100644
--- a/c++/src/wrap/coded-stream-wrapper.h
+++ b/c++/src/wrap/coded-stream-wrapper.h
@@ -15,7 +15,7 @@
 #ifndef CODED_STREAM_WRAPPER_HH
 #define CODED_STREAM_WRAPPER_HH
 
-#include "orc/Adaptor.hh"
+#include "Adaptor.hh"
 
 DIAGNOSTIC_PUSH
 

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/wrap/gmock.h
----------------------------------------------------------------------
diff --git a/c++/src/wrap/gmock.h b/c++/src/wrap/gmock.h
index e7675db..4a807ee 100644
--- a/c++/src/wrap/gmock.h
+++ b/c++/src/wrap/gmock.h
@@ -18,7 +18,7 @@
 // we need to disable a whole set of warnings as we include gtest.h
 // restore most of the warnings after the file is loaded.
 
-#include "orc/Adaptor.hh"
+#include "Adaptor.hh"
 
 DIAGNOSTIC_PUSH
 DIAGNOSTIC_IGNORE("-Wdeprecated")

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/wrap/gtest-wrapper.h
----------------------------------------------------------------------
diff --git a/c++/src/wrap/gtest-wrapper.h b/c++/src/wrap/gtest-wrapper.h
index 72b1867..fb95a1d 100644
--- a/c++/src/wrap/gtest-wrapper.h
+++ b/c++/src/wrap/gtest-wrapper.h
@@ -15,7 +15,7 @@
 #ifndef GTEST_WRAPPER_H
 #define GTEST_WRAPPER_H
 
-#include "orc/Adaptor.hh"
+#include "Adaptor.hh"
 
 // we need to disable a whole set of warnings as we include gtest.h
 // restore most of the warnings after the file is loaded.

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/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 7de57b0..7234b7f 100644
--- a/c++/src/wrap/orc-proto-wrapper.cc
+++ b/c++/src/wrap/orc-proto-wrapper.cc
@@ -12,7 +12,7 @@
  * limitations under the License.
  */
 
-#include "orc/Adaptor.hh"
+#include "Adaptor.hh"
 
 DIAGNOSTIC_IGNORE("-Wconversion")
 DIAGNOSTIC_IGNORE("-Wdeprecated")

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/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 35ea0e9..2f2304d 100644
--- a/c++/src/wrap/orc-proto-wrapper.hh
+++ b/c++/src/wrap/orc-proto-wrapper.hh
@@ -15,7 +15,7 @@
 #ifndef ORC_PROTO_WRAPPER_HH
 #define ORC_PROTO_WRAPPER_HH
 
-#include "orc/Adaptor.hh"
+#include "Adaptor.hh"
 
 DIAGNOSTIC_PUSH
 DIAGNOSTIC_IGNORE("-Wconversion")

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/src/wrap/snappy-wrapper.h
----------------------------------------------------------------------
diff --git a/c++/src/wrap/snappy-wrapper.h b/c++/src/wrap/snappy-wrapper.h
index f0cf018..aeab0f0 100644
--- a/c++/src/wrap/snappy-wrapper.h
+++ b/c++/src/wrap/snappy-wrapper.h
@@ -15,7 +15,7 @@
 #ifndef SNAPPY_WRAPPER_HH
 #define SNAPPY_WRAPPER_HH
 
-#include "orc/Adaptor.hh"
+#include "Adaptor.hh"
 
 DIAGNOSTIC_PUSH
 

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/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 45373e9..6f03113 100644
--- a/c++/src/wrap/zero-copy-stream-wrapper.h
+++ b/c++/src/wrap/zero-copy-stream-wrapper.h
@@ -15,7 +15,7 @@
 #ifndef ZERO_COPY_STREAM_WRAPPER_HH
 #define ZERO_COPY_STREAM_WRAPPER_HH
 
-#include "orc/Adaptor.hh"
+#include "Adaptor.hh"
 
 DIAGNOSTIC_PUSH
 

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/test/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/c++/test/CMakeLists.txt b/c++/test/CMakeLists.txt
index 0486771..5a2105a 100644
--- a/c++/test/CMakeLists.txt
+++ b/c++/test/CMakeLists.txt
@@ -23,13 +23,13 @@ include_directories(
 set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${CXX11_FLAGS} ${WARN_FLAGS}")
 
 add_executable (test-orc
-  orc/TestByteRle.cc
-  orc/TestColumnPrinter.cc
-  orc/TestColumnReader.cc
-  orc/TestCompression.cc
-  orc/TestDriver.cc
-  orc/TestInt128.cc
-  orc/TestRle.cc
+  TestByteRle.cc
+  TestColumnPrinter.cc
+  TestColumnReader.cc
+  TestCompression.cc
+  TestDriver.cc
+  TestInt128.cc
+  TestRle.cc
 )
 
 target_link_libraries (test-orc

http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/test/OrcTest.hh
----------------------------------------------------------------------
diff --git a/c++/test/OrcTest.hh b/c++/test/OrcTest.hh
new file mode 100644
index 0000000..adef344
--- /dev/null
+++ b/c++/test/OrcTest.hh
@@ -0,0 +1,25 @@
+/**
+ * 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 ORC_TEST_HH
+#define ORC_TEST_HH
+
+// determine the size of an array
+#define ARRAY_SIZE(array) (sizeof(array)/sizeof(*array))
+
+#endif


[08/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/test/orc/TestColumnReader.cc
----------------------------------------------------------------------
diff --git a/c++/test/orc/TestColumnReader.cc b/c++/test/orc/TestColumnReader.cc
deleted file mode 100644
index aad73e5..0000000
--- a/c++/test/orc/TestColumnReader.cc
+++ /dev/null
@@ -1,4309 +0,0 @@
-/**
- * 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 "orc/Adaptor.hh"
-#include "orc/ColumnReader.hh"
-#include "orc/Exceptions.hh"
-
-#include "OrcTest.hh"
-
-#include "wrap/orc-proto-wrapper.hh"
-#include "wrap/gtest-wrapper.h"
-#include "wrap/gmock.h"
-
-#include <iostream>
-#include <vector>
-
-#ifdef __clang__
-  DIAGNOSTIC_IGNORE("-Winconsistent-missing-override")
-#endif
-
-namespace orc {
-
-class MockStripeStreams: public StripeStreams {
-public:
-  ~MockStripeStreams();
-  std::unique_ptr<SeekableInputStream> getStream(int64_t columnId,
-                                                 proto::Stream_Kind kind,
-                                                 bool stream) const override;
-  MOCK_CONST_METHOD0(getReaderOptions, const ReaderOptions&());
-  MOCK_CONST_METHOD0(getSelectedColumns, const std::vector<bool>());
-  MOCK_CONST_METHOD1(getEncoding, proto::ColumnEncoding (int64_t));
-  MOCK_CONST_METHOD3(getStreamProxy, SeekableInputStream*
-                     (int64_t, proto::Stream_Kind, bool));
-  MemoryPool& getMemoryPool() const {
-    return *getDefaultPool();
-  }
-};
-
-MockStripeStreams::~MockStripeStreams() {
-  // PASS
-}
-
-std::unique_ptr<SeekableInputStream>
-MockStripeStreams::getStream(int64_t columnId,
-                             proto::Stream_Kind kind,
-                             bool shouldStream) const {
-  return std::unique_ptr < SeekableInputStream >
-    (getStreamProxy(columnId, kind, shouldStream));
-}
-
-TEST(TestColumnReader, testBooleanWithNulls) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(2, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-  // alternate 4 non-null and 4 null via [0xf0 for x in range(512 / 8)]
-  const unsigned char buffer1[] = { 0x3d, 0xf0 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-
-  // [0x0f for x in range(256 / 8)]
-  const unsigned char buffer2[] = {0x1d, 0x0f};
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer2, ARRAY_SIZE(buffer2))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createPrimitiveType(BOOLEAN), "col0");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
-  LongVectorBatch *longBatch = new LongVectorBatch(1024, *getDefaultPool());
-  StructVectorBatch batch(1024, *getDefaultPool());
-  batch.fields.push_back(longBatch);
-  reader->next(batch, 512, 0);
-  ASSERT_EQ(512, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(512, longBatch->numElements);
-  ASSERT_EQ(true, longBatch->hasNulls);
-  unsigned int next = 0;
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    if (i & 4) {
-      EXPECT_EQ(0, longBatch->notNull[i]) << "Wrong value at " << i;
-    } else {
-      EXPECT_EQ(1, longBatch->notNull[i]) << "Wrong value at " << i;
-      EXPECT_EQ((next++ & 4) != 0, longBatch->data[i])
-          << "Wrong value at " << i;
-    }
-  }
-}
-
-TEST(TestColumnReader, testBooleanSkipsWithNulls) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(2, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-  // alternate 4 non-null and 4 null via [0xf0 for x in range(512 / 8)]
-  const unsigned char buffer1[] = {0x3d, 0xf0};
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-  // [0x0f for x in range(128 / 8)]
-  const unsigned char buffer2[] = {0x1d, 0x0f};
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer2, ARRAY_SIZE(buffer2))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createPrimitiveType(BOOLEAN), "col0");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
-  LongVectorBatch *longBatch = new LongVectorBatch(1024, *getDefaultPool());
-  StructVectorBatch batch(1024, *getDefaultPool());
-  batch.fields.push_back(longBatch);
-  reader->next(batch, 1, 0);
-  ASSERT_EQ(1, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(1, longBatch->numElements);
-  ASSERT_EQ(true, !longBatch->hasNulls);
-  EXPECT_EQ(0, longBatch->data[0]);
-  reader->skip(506);
-  reader->next(batch, 5, 0);
-  ASSERT_EQ(5, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(5, longBatch->numElements);
-  ASSERT_EQ(true, longBatch->hasNulls);
-  EXPECT_EQ(1, longBatch->data[0]);
-  EXPECT_EQ(true, !longBatch->notNull[1]);
-  EXPECT_EQ(true, !longBatch->notNull[2]);
-  EXPECT_EQ(true, !longBatch->notNull[3]);
-  EXPECT_EQ(true, !longBatch->notNull[4]);
-}
-
-TEST(TestColumnReader, testByteWithNulls) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(2, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-  // alternate 4 non-null and 4 null via [0xf0 for x in range(512 / 8)]
-  const unsigned char buffer1[] = {0x3d, 0xf0};
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-
-  // range(256)
-  char buffer[258];
-  buffer[0] = static_cast<char>(0x80);
-  for (unsigned int i = 0; i < 128; ++i) {
-    buffer[i + 1] = static_cast<char>(i);
-  }
-  buffer[129] = static_cast<char>(0x80);
-  for (unsigned int i = 128; i < 256; ++i) {
-    buffer[i + 2] = static_cast<char>(i);
-  }
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer, ARRAY_SIZE(buffer))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createPrimitiveType(BYTE), "col0");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader =
-      buildReader(*rowType, streams);
-  LongVectorBatch *longBatch = new LongVectorBatch(1024, *getDefaultPool());
-  StructVectorBatch batch(1024, *getDefaultPool());
-  batch.fields.push_back(longBatch);
-  reader->next(batch, 512, 0);
-  ASSERT_EQ(512, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(512, longBatch->numElements);
-  ASSERT_EQ(true, longBatch->hasNulls);
-  unsigned int next = 0;
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    if (i & 4) {
-      EXPECT_EQ(0, longBatch->notNull[i]) << "Wrong value at " << i;
-    } else {
-      EXPECT_EQ(1, longBatch->notNull[i]) << "Wrong value at " << i;
-      EXPECT_EQ(static_cast<char>(next++), longBatch->data[i])
-          << "Wrong value at " << i;
-    }
-  }
-}
-
-TEST(TestColumnReader, testByteSkipsWithNulls) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(2, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-  // alternate 4 non-null and 4 null via [0xf0 for x in range(512 / 8)]
-  const unsigned char buffer1[] = {0x3d, 0xf0};
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-
-  // range(256)
-  char buffer[258];
-  buffer[0] = static_cast<char>(0x80);
-  for (unsigned int i = 0; i < 128; ++i) {
-    buffer[i + 1] = static_cast<char>(i);
-  }
-  buffer[129] = static_cast<char>(0x80);
-  for (unsigned int i = 128; i < 256; ++i) {
-    buffer[i + 2] = static_cast<char>(i);
-  }
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer, ARRAY_SIZE(buffer))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createPrimitiveType(BYTE), "col0");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader =
-      buildReader(*rowType, streams);
-  LongVectorBatch *longBatch = new LongVectorBatch(1024, *getDefaultPool());
-  StructVectorBatch batch(1024, *getDefaultPool());
-  batch.fields.push_back(longBatch);
-  reader->next(batch, 1, 0);
-  ASSERT_EQ(1, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(1, longBatch->numElements);
-  ASSERT_EQ(true, !longBatch->hasNulls);
-  EXPECT_EQ(0, longBatch->data[0]);
-  reader->skip(506);
-  reader->next(batch, 5, 0);
-  ASSERT_EQ(5, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(5, longBatch->numElements);
-  ASSERT_EQ(true, longBatch->hasNulls);
-  EXPECT_EQ(static_cast<char>(-1), longBatch->data[0]);
-  EXPECT_EQ(true, !longBatch->notNull[1]);
-  EXPECT_EQ(true, !longBatch->notNull[2]);
-  EXPECT_EQ(true, !longBatch->notNull[3]);
-  EXPECT_EQ(true, !longBatch->notNull[4]);
-}
-
-TEST(TestColumnReader, testIntegerWithNulls) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(2, true);
-  
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-  const unsigned char buffer1[] = {0x16, 0xf0};
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-  const unsigned char buffer2[] = {0x64, 0x01, 0x00};
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer2, ARRAY_SIZE(buffer2))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createPrimitiveType(INT), "myInt" );
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader =
-      buildReader(*rowType, streams);
-  LongVectorBatch *longBatch = new LongVectorBatch(1024, *getDefaultPool());
-  StructVectorBatch batch(1024, *getDefaultPool());
-  batch.fields.push_back(longBatch);
-  reader->next(batch, 200, 0);
-  ASSERT_EQ(200, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(200, longBatch->numElements);
-  ASSERT_EQ(true, longBatch->hasNulls);
-  long next = 0;
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    if (i & 4) {
-      EXPECT_EQ(0, longBatch->notNull[i]);
-    } else {
-      EXPECT_EQ(1, longBatch->notNull[i]);
-      EXPECT_EQ(next++, longBatch->data[i]);
-    }
-  }
-}
-
-TEST(TestColumnReader, testDictionaryWithNulls) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(2, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(0))
-      .WillRepeatedly(testing::Return(directEncoding));
-  proto::ColumnEncoding dictionaryEncoding;
-  dictionaryEncoding.set_kind(proto::ColumnEncoding_Kind_DICTIONARY);
-  dictionaryEncoding.set_dictionarysize(2);
-  EXPECT_CALL(streams, getEncoding(1))
-      .WillRepeatedly(testing::Return(dictionaryEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-  const unsigned char buffer1[] =  { 0x19, 0xf0 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-  const unsigned char buffer2[] = { 0x2f, 0x00, 0x00, 0x2f, 0x00, 0x01 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer2, ARRAY_SIZE(buffer2))));
-  const unsigned char buffer3[] = { 0x4f, 0x52, 0x43, 0x4f, 0x77, 0x65, 0x6e };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DICTIONARY_DATA,
-                                      false))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer3, ARRAY_SIZE(buffer3))));
-  const unsigned char buffer4[] =  { 0x02, 0x01, 0x03 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, false))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer4, ARRAY_SIZE(buffer4))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createPrimitiveType(STRING), "myString");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader =
-      buildReader(*rowType, streams);
-  StringVectorBatch *stringBatch = new StringVectorBatch(1024,
-                                                         *getDefaultPool());
-  StructVectorBatch batch(1024, *getDefaultPool());
-  batch.fields.push_back(stringBatch);
-  reader->next(batch, 200, 0);
-  ASSERT_EQ(200, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(200, stringBatch->numElements);
-  ASSERT_EQ(true, stringBatch->hasNulls);
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    if (i & 4) {
-      EXPECT_EQ(0, stringBatch->notNull[i]);
-    } else {
-      EXPECT_EQ(1, stringBatch->notNull[i]);
-      const char* expected = i < 98 ? "ORC" : "Owen";
-      ASSERT_EQ(strlen(expected), stringBatch->length[i])
-      << "Wrong length at " << i;
-      for (size_t letter = 0; letter < strlen(expected); ++letter) {
-        EXPECT_EQ(expected[letter], stringBatch->data[i][letter])
-            << "Wrong contents at " << i << ", " << letter;
-      }
-    }
-  }
-}
-
-TEST(TestColumnReader, testVarcharDictionaryWithNulls) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(3, true);
-  selectedColumns.push_back(false);
-
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(0))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  proto::ColumnEncoding dictionary2Encoding;
-  dictionary2Encoding.set_kind(proto::ColumnEncoding_Kind_DICTIONARY);
-  dictionary2Encoding.set_dictionarysize(2);
-  EXPECT_CALL(streams, getEncoding(1))
-      .WillRepeatedly(testing::Return(dictionary2Encoding));
-
-  proto::ColumnEncoding dictionary0Encoding;
-  dictionary0Encoding.set_kind(proto::ColumnEncoding_Kind_DICTIONARY);
-  dictionary0Encoding.set_dictionarysize(0);
-  EXPECT_CALL(streams, getEncoding(testing::Ge(2)))
-      .WillRepeatedly(testing::Return(dictionary0Encoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  const unsigned char buffer1[] = { 0x16, 0xff };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-  const unsigned char buffer2[] = { 0x61, 0x00, 0x01, 0x61, 0x00, 0x00 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer2, ARRAY_SIZE(buffer2))));
-  const unsigned char buffer3[] = { 0x4f, 0x52, 0x43, 0x4f, 0x77,
-          0x65, 0x6e };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DICTIONARY_DATA,
-                                      false))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer3, ARRAY_SIZE(buffer3))));
-  const unsigned char buffer4[] = { 0x02, 0x01, 0x03 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, false))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer4, ARRAY_SIZE(buffer4))));
-
-  const unsigned char buffer5[] = {0x16, 0x00};
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer5, ARRAY_SIZE(buffer5))));
-
-  // all three return an empty stream
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer5, 0)));
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DICTIONARY_DATA,
-                                      false))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer5, 0)));
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_LENGTH, false))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer5, 0)));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createPrimitiveType(VARCHAR), "col0");
-  rowType->addStructField(createPrimitiveType(CHAR), "col1");
-  rowType->addStructField(createPrimitiveType(STRING), "col2");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader =
-      buildReader(*rowType, streams);
-  StructVectorBatch batch(1024, *getDefaultPool());
-  StringVectorBatch *stringBatch = new StringVectorBatch(1024,
-                                                         *getDefaultPool());
-  StringVectorBatch *nullBatch = new StringVectorBatch(1024,
-                                                       *getDefaultPool());
-  batch.fields.push_back(stringBatch);
-  batch.fields.push_back(nullBatch);
-  reader->next(batch, 200, 0);
-  ASSERT_EQ(200, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(200, stringBatch->numElements);
-  ASSERT_EQ(true, !stringBatch->hasNulls);
-  ASSERT_EQ(200, nullBatch->numElements);
-  ASSERT_EQ(true, nullBatch->hasNulls);
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    EXPECT_EQ(true, stringBatch->notNull[i]);
-    EXPECT_EQ(true, !nullBatch->notNull[i]);
-    const char* expected = i < 100 ? "Owen" : "ORC";
-    ASSERT_EQ(strlen(expected), stringBatch->length[i])
-    << "Wrong length at " << i;
-    for (size_t letter = 0; letter < strlen(expected); ++letter) {
-      EXPECT_EQ(expected[letter], stringBatch->data[i][letter])
-          << "Wrong contents at " << i << ", " << letter;
-    }
-  }
-}
-
-TEST(TestColumnReader, testSubstructsWithNulls) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(4, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  const unsigned char buffer1[] = {0x16, 0x0f};
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-
-  const unsigned char buffer2[] = { 0x0a, 0x55 };
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer2, ARRAY_SIZE(buffer2))));
-
-  const unsigned char buffer3[] = { 0x04, 0xf0 };
-  EXPECT_CALL(streams, getStreamProxy(3, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer3, ARRAY_SIZE(buffer3))));
-
-  const unsigned char buffer4[] =  { 0x17, 0x01, 0x00 };
-  EXPECT_CALL(streams, getStreamProxy(3, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer4, ARRAY_SIZE(buffer4))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createStructType(), "col0")
-    .addStructField(createStructType(), "col1")
-    .addStructField(createPrimitiveType(LONG), "col2");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
-
-  StructVectorBatch batch(1024, *getDefaultPool());
-  StructVectorBatch *middle = new StructVectorBatch(1024, *getDefaultPool());
-  StructVectorBatch *inner = new StructVectorBatch(1024, *getDefaultPool());
-  LongVectorBatch *longs = new LongVectorBatch(1024, *getDefaultPool());
-  batch.fields.push_back(middle);
-  middle->fields.push_back(inner);
-  inner->fields.push_back(longs);
-  reader->next(batch, 200, 0);
-  ASSERT_EQ(200, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(200, middle->numElements);
-  ASSERT_EQ(true, middle->hasNulls);
-  ASSERT_EQ(200, inner->numElements);
-  ASSERT_EQ(true, inner->hasNulls);
-  ASSERT_EQ(200, longs->numElements);
-  ASSERT_EQ(true, longs->hasNulls);
-  long middleCount = 0;
-  long innerCount = 0;
-  long longCount = 0;
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    if (i & 4) {
-      EXPECT_EQ(true, middle->notNull[i]) << "Wrong at " << i;
-      if (middleCount++ & 1) {
-        EXPECT_EQ(true, inner->notNull[i]) << "Wrong at " << i;
-        if (innerCount++ & 4) {
-          EXPECT_EQ(true, !longs->notNull[i]) << "Wrong at " << i;
-        } else {
-          EXPECT_EQ(true, longs->notNull[i]) << "Wrong at " << i;
-          EXPECT_EQ(longCount++, longs->data[i]) << "Wrong at " << i;
-        }
-      } else {
-        EXPECT_EQ(true, !inner->notNull[i]) << "Wrong at " << i;
-        EXPECT_EQ(true, !longs->notNull[i]) << "Wrong at " << i;
-      }
-    } else {
-      EXPECT_EQ(true, !middle->notNull[i]) << "Wrong at " << i;
-      EXPECT_EQ(true, !inner->notNull[i]) << "Wrong at " << i;
-      EXPECT_EQ(true, !longs->notNull[i]) << "Wrong at " << i;
-    }
-  }
-}
-
-TEST(TestColumnReader, testSkipWithNulls) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(3, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-  proto::ColumnEncoding dictionaryEncoding;
-  dictionaryEncoding.set_kind(proto::ColumnEncoding_Kind_DICTIONARY);
-  dictionaryEncoding.set_dictionarysize(100);
-  EXPECT_CALL(streams, getEncoding(2))
-      .WillRepeatedly(testing::Return(dictionaryEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-  const unsigned char buffer1[] =  { 0x03, 0x00, 0xff, 0x3f, 0x08, 0xff,
-                                     0xff, 0xfc, 0x03, 0x00 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-
-  const unsigned char buffer2[] = { 0x61, 0x01, 0x00 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer2, ARRAY_SIZE(buffer2))));
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer2, ARRAY_SIZE(buffer2))));
-
-  // fill the dictionary with '00' to '99'
-  char digits[200];
-  for (int i = 0; i < 10; ++i) {
-    for (int j = 0; j < 10; ++j) {
-      digits[2 * (10 * i + j)] = static_cast<char>('0' + i);
-      digits[2 * (10 * i + j) + 1] = static_cast<char>('0' + j);
-    }
-  }
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DICTIONARY_DATA,
-                                      false))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (digits, ARRAY_SIZE(digits))));
-  const unsigned char buffer3[] = {0x61, 0x00, 0x02};
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_LENGTH, false))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer3, ARRAY_SIZE(buffer3))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createPrimitiveType(INT), "myInt");
-  rowType->addStructField(createPrimitiveType(STRING), "myString");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader =
-      buildReader(*rowType, streams);
-  StructVectorBatch batch(100, *getDefaultPool());
-  LongVectorBatch *longBatch = new LongVectorBatch(100, *getDefaultPool());
-  StringVectorBatch *stringBatch =
-    new StringVectorBatch(100, *getDefaultPool());
-  batch.fields.push_back(longBatch);
-  batch.fields.push_back(stringBatch);
-  reader->next(batch, 20, 0);
-  ASSERT_EQ(20, batch.numElements);
-  ASSERT_EQ(20, longBatch->numElements);
-  ASSERT_EQ(20, stringBatch->numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(true, longBatch->hasNulls);
-  ASSERT_EQ(true, stringBatch->hasNulls);
-  for (size_t i = 0; i < 20; ++i) {
-    EXPECT_EQ(true, !longBatch->notNull[i]) << "Wrong at " << i;
-    EXPECT_EQ(true, !stringBatch->notNull[i]) << "Wrong at " << i;
-  }
-  reader->skip(30);
-  reader->next(batch, 100, 0);
-  ASSERT_EQ(100, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(true, !longBatch->hasNulls);
-  ASSERT_EQ(true, !stringBatch->hasNulls);
-  for (size_t i = 0; i < 10; ++i) {
-    for (size_t j = 0; j < 10; ++j) {
-      size_t k = 10 * i + j;
-      EXPECT_EQ(1, longBatch->notNull[k]) << "Wrong at " << k;
-      ASSERT_EQ(2, stringBatch->length[k])<< "Wrong at " << k;
-      EXPECT_EQ('0' + static_cast<char>(i), stringBatch->data[k][0])
-          << "Wrong at " << k;
-      EXPECT_EQ('0' + static_cast<char>(j), stringBatch->data[k][1])
-          << "Wrong at " << k;
-    }
-  }
-  reader->skip(50);
-}
-
-TEST(TestColumnReader, testBinaryDirect) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(2, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  char blob[200];
-  for (size_t i = 0; i < 10; ++i) {
-    for (size_t j = 0; j < 10; ++j) {
-      blob[2 * (10 * i + j)] = static_cast<char>(i);
-      blob[2 * (10 * i + j) + 1] = static_cast<char>(j);
-    }
-  }
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (blob, ARRAY_SIZE(blob))));
-
-  const unsigned char buffer[] = { 0x61, 0x00, 0x02 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer, ARRAY_SIZE(buffer))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createPrimitiveType(BINARY), "col0");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
-
-  StructVectorBatch batch(1024, *getDefaultPool());
-  StringVectorBatch *strings = new StringVectorBatch(1024, *getDefaultPool());
-  batch.fields.push_back(strings);
-  for (size_t i = 0; i < 2; ++i) {
-    reader->next(batch, 50, 0);
-    ASSERT_EQ(50, batch.numElements);
-    ASSERT_EQ(true, !batch.hasNulls);
-    ASSERT_EQ(50, strings->numElements);
-    ASSERT_EQ(true, !strings->hasNulls);
-    for (size_t j = 0; j < batch.numElements; ++j) {
-      ASSERT_EQ(2, strings->length[j]);
-      EXPECT_EQ((50 * i + j) / 10, strings->data[j][0]);
-      EXPECT_EQ((50 * i + j) % 10, strings->data[j][1]);
-    }
-  }
-}
-
-TEST(TestColumnReader, testBinaryDirectWithNulls) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(2, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  const unsigned char buffer1[] = {0x1d, 0xf0};
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-
-  char blob[256];
-  for (size_t i = 0; i < 8; ++i) {
-    for (size_t j = 0; j < 16; ++j) {
-      blob[2 * (16 * i + j)] = static_cast<char>('A' + i);
-      blob[2 * (16 * i + j) + 1] = static_cast<char>('A' + j);
-    }
-  }
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (blob, ARRAY_SIZE(blob))));
-
-  const unsigned char buffer2[] = {0x7d, 0x00, 0x02};
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer2, ARRAY_SIZE(buffer2))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createPrimitiveType(BINARY), "col0");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
-
-  StructVectorBatch batch(1024, *getDefaultPool());
-  StringVectorBatch *strings = new StringVectorBatch(1024, *getDefaultPool());
-  batch.fields.push_back(strings);
-  size_t next = 0;
-  for (size_t i = 0; i < 2; ++i) {
-    reader->next(batch, 128, 0);
-    ASSERT_EQ(128, batch.numElements);
-    ASSERT_EQ(true, !batch.hasNulls);
-    ASSERT_EQ(128, strings->numElements);
-    ASSERT_EQ(true, strings->hasNulls);
-    for (size_t j = 0; j < batch.numElements; ++j) {
-      ASSERT_EQ(((128 * i + j) & 4) == 0, strings->notNull[j]);
-      if (strings->notNull[j]) {
-        ASSERT_EQ(2, strings->length[j]);
-        EXPECT_EQ('A' + static_cast<char>(next / 16), strings->data[j][0]);
-        EXPECT_EQ('A' + static_cast<char>(next % 16), strings->data[j][1]);
-        next += 1;
-      }
-    }
-  }
-}
-
-TEST(TestColumnReader, testShortBlobError) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(2, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  char blob[100];
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (blob, ARRAY_SIZE(blob))));
-  
-  const unsigned char buffer1[] = {0x61, 0x00, 0x02};
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createPrimitiveType(STRING), "col0");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
-
-  StructVectorBatch batch(1024, *getDefaultPool());
-  StringVectorBatch *strings = new StringVectorBatch(1024, *getDefaultPool());
-  batch.fields.push_back(strings);
-  EXPECT_THROW(reader->next(batch, 100, 0), ParseError);
-}
-
-TEST(TestColumnReader, testStringDirectShortBuffer) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(2, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  char blob[200];
-  for (size_t i = 0; i < 10; ++i) {
-    for (size_t j = 0; j < 10; ++j) {
-      blob[2 * (10 * i + j)] = static_cast<char>(i);
-      blob[2 * (10 * i + j) + 1] = static_cast<char>(j);
-    }
-  }
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (blob, ARRAY_SIZE(blob), 3)));
-
-  const unsigned char buffer1[] = {0x61, 0x00, 0x02};
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createPrimitiveType(STRING), "col0");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
-
-  StructVectorBatch batch(25, *getDefaultPool());
-  StringVectorBatch *strings = new StringVectorBatch(25, *getDefaultPool());
-  batch.fields.push_back(strings);
-  for (size_t i = 0; i < 4; ++i) {
-    reader->next(batch, 25, 0);
-    ASSERT_EQ(25, batch.numElements);
-    ASSERT_EQ(true, !batch.hasNulls);
-    ASSERT_EQ(25, strings->numElements);
-    ASSERT_EQ(true, !strings->hasNulls);
-    for (size_t j = 0; j < batch.numElements; ++j) {
-      ASSERT_EQ(2, strings->length[j]);
-      EXPECT_EQ((25 * i + j) / 10, strings->data[j][0]);
-      EXPECT_EQ((25 * i + j) % 10, strings->data[j][1]);
-    }
-  }
-}
-
-TEST(TestColumnReader, testStringDirectShortBufferWithNulls) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(2, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  const unsigned char buffer1[] = {0x3d, 0xf0};
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-
-  char blob[512];
-  for (size_t i = 0; i < 16; ++i) {
-    for (size_t j = 0; j < 16; ++j) {
-      blob[2 * (16 * i + j)] = static_cast<char>('A' + i);
-      blob[2 * (16 * i + j) + 1] = static_cast<char>('A' + j);
-    }
-  }
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (blob, ARRAY_SIZE(blob), 30)));
-
-  const unsigned char buffer2[] =  { 0x7d, 0x00, 0x02, 0x7d, 0x00, 0x02 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer2, ARRAY_SIZE(buffer2))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createPrimitiveType(STRING), "col0");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
-
-  StructVectorBatch batch(64, *getDefaultPool());
-  StringVectorBatch *strings = new StringVectorBatch(64, *getDefaultPool());
-  batch.fields.push_back(strings);
-  size_t next = 0;
-  for (size_t i = 0; i < 8; ++i) {
-    reader->next(batch, 64, 0);
-    ASSERT_EQ(64, batch.numElements);
-    ASSERT_EQ(true, !batch.hasNulls);
-    ASSERT_EQ(64, strings->numElements);
-    ASSERT_EQ(true, strings->hasNulls);
-    for (size_t j = 0; j < batch.numElements; ++j) {
-      ASSERT_EQ((j & 4) == 0, strings->notNull[j]);
-      if (strings->notNull[j]) {
-        ASSERT_EQ(2, strings->length[j]);
-        EXPECT_EQ('A' + next / 16, strings->data[j][0]);
-        EXPECT_EQ('A' + next % 16, strings->data[j][1]);
-        next += 1;
-      }
-    }
-  }
-}
-
-TEST(TestColumnReader, testStringDirectSkip) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(2, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  // sum(0 to 1199)
-  const size_t BLOB_SIZE = 719400;
-  char blob[BLOB_SIZE];
-  size_t posn = 0;
-  for (size_t item = 0; item < 1200; ++item) {
-    for (size_t ch = 0; ch < item; ++ch) {
-      blob[posn++] = static_cast<char>(ch);
-    }
-  }
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-      (blob, BLOB_SIZE, 200)));
-
-  // the stream of 0 to 1199
-  const unsigned char buffer1[] = 
-    { 0x7f, 0x01, 0x00,
-      0x7f, 0x01, 0x82, 0x01,
-      0x7f, 0x01, 0x84, 0x02,
-      0x7f, 0x01, 0x86, 0x03,
-      0x7f, 0x01, 0x88, 0x04,
-      0x7f, 0x01, 0x8a, 0x05,
-      0x7f, 0x01, 0x8c, 0x06,
-      0x7f, 0x01, 0x8e, 0x07,
-      0x7f, 0x01, 0x90, 0x08,
-      0x1b, 0x01, 0x92, 0x09 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createPrimitiveType(STRING), "col0");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
-
-  StructVectorBatch batch(2, *getDefaultPool());
-  StringVectorBatch *strings = new StringVectorBatch(2, *getDefaultPool());
-  batch.fields.push_back(strings);
-  reader->next(batch, 2, 0);
-  ASSERT_EQ(2, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(2, strings->numElements);
-  ASSERT_EQ(true, !strings->hasNulls);
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    ASSERT_EQ(i, strings->length[i]);
-    for (size_t j = 0; j < i; ++j) {
-      EXPECT_EQ(static_cast<char>(j), strings->data[i][j]);
-    }
-  }
-  reader->skip(14);
-  reader->next(batch, 2, 0);
-  ASSERT_EQ(2, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(2, strings->numElements);
-  ASSERT_EQ(true, !strings->hasNulls);
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    ASSERT_EQ(16 + i, strings->length[i]);
-    for (size_t j = 0; j < 16 + i; ++j) {
-      EXPECT_EQ(static_cast<char>(j), strings->data[i][j]);
-    }
-  }
-  reader->skip(1180);
-  reader->next(batch, 2, 0);
-  ASSERT_EQ(2, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(2, strings->numElements);
-  ASSERT_EQ(true, !strings->hasNulls);
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    ASSERT_EQ(1198 + i, strings->length[i]);
-    for (size_t j = 0; j < 1198 + i; ++j) {
-      EXPECT_EQ(static_cast<char>(j), strings->data[i][j]);
-    }
-  }
-}
-
-TEST(TestColumnReader, testStringDirectSkipWithNulls) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(2, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  // alternate 4 non-null and 4 null via [0xf0 for x in range(2400 / 8)]
-  const unsigned char buffer1[] =  { 0x7f, 0xf0, 0x7f, 0xf0, 0x25, 0xf0 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-
-  // sum(range(1200))
-  const size_t BLOB_SIZE = 719400;
-
-  // each string is [x % 256 for x in range(r)]
-  char blob[BLOB_SIZE];
-  size_t posn = 0;
-  for (size_t item = 0; item < 1200; ++item) {
-    for (size_t ch = 0; ch < item; ++ch) {
-      blob[posn++] = static_cast<char>(ch);
-    }
-  }
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-      (blob, BLOB_SIZE, 200)));
-
-  // range(1200)
-  const unsigned char buffer2[] =  { 0x7f, 0x01, 0x00,
-                                     0x7f, 0x01, 0x82, 0x01,
-                                     0x7f, 0x01, 0x84, 0x02,
-                                     0x7f, 0x01, 0x86, 0x03,
-                                     0x7f, 0x01, 0x88, 0x04,
-                                     0x7f, 0x01, 0x8a, 0x05,
-                                     0x7f, 0x01, 0x8c, 0x06,
-                                     0x7f, 0x01, 0x8e, 0x07,
-                                     0x7f, 0x01, 0x90, 0x08,
-                                     0x1b, 0x01, 0x92, 0x09 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer2, ARRAY_SIZE(buffer2))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createPrimitiveType(STRING), "col0");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
-
-  StructVectorBatch batch(2, *getDefaultPool());
-  StringVectorBatch *strings = new StringVectorBatch(2, *getDefaultPool());
-  batch.fields.push_back(strings);
-  reader->next(batch, 2, 0);
-  ASSERT_EQ(2, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(2, strings->numElements);
-  ASSERT_EQ(true, !strings->hasNulls);
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    ASSERT_EQ(i, strings->length[i]);
-    for (size_t j = 0; j < i; ++j) {
-      EXPECT_EQ(static_cast<char>(j), strings->data[i][j]);
-    }
-  }
-  reader->skip(30);
-  reader->next(batch, 2, 0);
-  ASSERT_EQ(2, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(2, strings->numElements);
-  ASSERT_EQ(true, !strings->hasNulls);
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    ASSERT_EQ(16 + i, strings->length[i]);
-    for (size_t j = 0; j < 16 + i; ++j) {
-      EXPECT_EQ(static_cast<char>(j), strings->data[i][j]);
-    }
-  }
-  reader->skip(2364);
-  reader->next(batch, 2, 0);
-  ASSERT_EQ(2, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(2, strings->numElements);
-  ASSERT_EQ(true, strings->hasNulls);
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    EXPECT_EQ(true, !strings->notNull[i]);
-  }
-}
-
-TEST(TestColumnReader, testList) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(3, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(testing::_,
-          proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  // [2 for x in range(600)]
-  const unsigned char buffer1[] =  { 0x7f, 0x00, 0x02,
-                                     0x7f, 0x00, 0x02,
-                                     0x7f, 0x00, 0x02,
-                                     0x7f, 0x00, 0x02,
-                                     0x4d, 0x00, 0x02 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-
-  // range(1200)
-  const unsigned char buffer2[] = { 0x7f, 0x01, 0x00,
-                                    0x7f, 0x01, 0x84, 0x02,
-                                    0x7f, 0x01, 0x88, 0x04,
-                                    0x7f, 0x01, 0x8c, 0x06,
-                                    0x7f, 0x01, 0x90, 0x08,
-                                    0x7f, 0x01, 0x94, 0x0a,
-                                    0x7f, 0x01, 0x98, 0x0c,
-                                    0x7f, 0x01, 0x9c, 0x0e,
-                                    0x7f, 0x01, 0xa0, 0x10,
-                                    0x1b, 0x01, 0xa4, 0x12 };
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer2, ARRAY_SIZE(buffer2))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createListType(createPrimitiveType(LONG)), "col0");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
-
-  StructVectorBatch batch(512, *getDefaultPool());
-  ListVectorBatch *lists = new ListVectorBatch(512, *getDefaultPool());
-  LongVectorBatch *longs = new LongVectorBatch(512, *getDefaultPool());
-  batch.fields.push_back(lists);
-  lists->elements = std::unique_ptr < ColumnVectorBatch > (longs);
-  reader->next(batch, 512, 0);
-  ASSERT_EQ(512, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(512, lists->numElements);
-  ASSERT_EQ(true, !lists->hasNulls);
-  ASSERT_EQ(1024, longs->numElements);
-  ASSERT_EQ(true, !longs->hasNulls);
-  for (size_t i = 0; i <= batch.numElements; ++i) {
-    EXPECT_EQ(2 * i, lists->offsets[i]);
-  }
-  for (size_t i = 0; i < longs->numElements; ++i) {
-    EXPECT_EQ(i, longs->data[i]);
-  }
-}
-
-TEST(TestColumnReader, testListPropagateNulls) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(4, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createStructType(), "col0")
-    .addStructField(createListType(createPrimitiveType(LONG)), "col0_0");
-  rowType->assignIds(0);
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(testing::_,
-          proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  // set getStream
-  const unsigned char buffer[] = {0xff, 0x00};
-  EXPECT_CALL(streams, getStreamProxy(1,proto::Stream_Kind_PRESENT, true))
-    .WillRepeatedly(testing::Return
-                    (new SeekableArrayInputStream(buffer,
-                                                  ARRAY_SIZE(buffer))));
-
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_LENGTH, true))
-                    .WillRepeatedly(testing::Return
-                                    (new SeekableArrayInputStream(buffer, 0)));
-
-  EXPECT_CALL(streams, getStreamProxy(3, proto::Stream_Kind_DATA, true))
-                    .WillRepeatedly(testing::Return
-                                    (new SeekableArrayInputStream(buffer, 0)));
-
-  // create the row type
-  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
-
-  StructVectorBatch batch(512, *getDefaultPool());
-  StructVectorBatch *structs = new StructVectorBatch(512, *getDefaultPool());
-  ListVectorBatch *lists = new ListVectorBatch(512, *getDefaultPool());
-  LongVectorBatch *longs = new LongVectorBatch(512, *getDefaultPool());
-  batch.fields.push_back(structs);
-  structs->fields.push_back(lists);
-  lists->elements = std::unique_ptr < ColumnVectorBatch > (longs);
-  reader->next(batch, 8, 0);
-  ASSERT_EQ(8, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(8, structs->numElements);
-  ASSERT_EQ(true, structs->hasNulls);
-  ASSERT_EQ(8, lists->numElements);
-  ASSERT_EQ(true, lists->hasNulls);
-  ASSERT_EQ(0, longs->numElements);
-  ASSERT_EQ(true, !longs->hasNulls);
-  for(size_t i=0; i < 8; ++i) {
-    EXPECT_EQ(true, !structs->notNull[i]);
-  }
-}
-
-TEST(TestColumnReader, testListWithNulls) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(3, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  // [0xaa for x in range(2048/8)]
-  const unsigned char buffer1[] =  { 0x7f, 0xaa, 0x7b, 0xaa };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  // [1 for x in range(260)] +
-  // [4 for x in range(260)] +
-  // [0 for x in range(260)] +
-  // [3 for x in range(243)] +
-  // [19]
-  const unsigned char buffer2[] =  { 0x7f, 0x00, 0x01,
-                                     0x7f, 0x00, 0x01,
-                                     0x7f, 0x00, 0x04,
-                                     0x7f, 0x00, 0x04,
-                                     0x7f, 0x00, 0x00,
-                                     0x7f, 0x00, 0x00,
-                                     0x7f, 0x00, 0x03,
-                                     0x6e, 0x00, 0x03,
-                                     0xff, 0x13 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer2, ARRAY_SIZE(buffer2))));
-
-  // range(2048)
-  const unsigned char buffer3[] = { 0x7f, 0x01, 0x00,
-                                    0x7f, 0x01, 0x84, 0x02,
-                                    0x7f, 0x01, 0x88, 0x04,
-                                    0x7f, 0x01, 0x8c, 0x06,
-                                    0x7f, 0x01, 0x90, 0x08,
-                                    0x7f, 0x01, 0x94, 0x0a,
-                                    0x7f, 0x01, 0x98, 0x0c,
-                                    0x7f, 0x01, 0x9c, 0x0e,
-                                    0x7f, 0x01, 0xa0, 0x10,
-                                    0x7f, 0x01, 0xa4, 0x12,
-                                    0x7f, 0x01, 0xa8, 0x14,
-                                    0x7f, 0x01, 0xac, 0x16,
-                                    0x7f, 0x01, 0xb0, 0x18,
-                                    0x7f, 0x01, 0xb4, 0x1a,
-                                    0x7f, 0x01, 0xb8, 0x1c,
-                                    0x5f, 0x01, 0xbc, 0x1e };
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer3, ARRAY_SIZE(buffer3))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createListType(createPrimitiveType(LONG)), "col0");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
-
-  StructVectorBatch batch(512, *getDefaultPool());
-  ListVectorBatch *lists = new ListVectorBatch(512, *getDefaultPool());
-  LongVectorBatch *longs = new LongVectorBatch(512, *getDefaultPool());
-  batch.fields.push_back(lists);
-  lists->elements = std::unique_ptr < ColumnVectorBatch > (longs);
-  reader->next(batch, 512, 0);
-  ASSERT_EQ(512, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(512, lists->numElements);
-  ASSERT_EQ(true, lists->hasNulls);
-  ASSERT_EQ(256, longs->numElements);
-  ASSERT_EQ(true, !longs->hasNulls);
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    EXPECT_EQ(i % 2 == 0, lists->notNull[i]) << "Wrong value at " << i;
-    EXPECT_EQ((i + 1) / 2, lists->offsets[i]) << "Wrong value at " << i;
-  }
-  EXPECT_EQ(256, lists->offsets[512]);
-  for (size_t i = 0; i < longs->numElements; ++i) {
-    EXPECT_EQ(i, longs->data[i]);
-  }
-
-  reader->next(batch, 512, 0);
-  ASSERT_EQ(512, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(512, lists->numElements);
-  ASSERT_EQ(true, lists->hasNulls);
-  ASSERT_EQ(1012, longs->numElements);
-  ASSERT_EQ(true, !longs->hasNulls);
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    EXPECT_EQ(i % 2 == 0, lists->notNull[i]) << "Wrong value at " << i;
-    if (i < 8) {
-      EXPECT_EQ((i + 1) / 2, lists->offsets[i])
-          << "Wrong value at " << i;
-    } else {
-      EXPECT_EQ(4 * ((i + 1) / 2) - 12, lists->offsets[i])
-          << "Wrong value at " << i;
-    }
-  }
-  EXPECT_EQ(1012, lists->offsets[512]);
-  for (size_t i = 0; i < longs->numElements; ++i) {
-    EXPECT_EQ(256 + i, longs->data[i]);
-  }
-
-  reader->next(batch, 512, 0);
-  ASSERT_EQ(512, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(512, lists->numElements);
-  ASSERT_EQ(true, lists->hasNulls);
-  ASSERT_EQ(32, longs->numElements);
-  ASSERT_EQ(true, !longs->hasNulls);
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    EXPECT_EQ(i % 2 == 0, lists->notNull[i]) << "Wrong value at " << i;
-    if (i < 16) {
-      EXPECT_EQ(4 * ((i + 1) / 2), lists->offsets[i])
-          << "Wrong value at " << i;
-    } else {
-      EXPECT_EQ(32, lists->offsets[i]) << "Wrong value at " << i;
-    }
-  }
-  EXPECT_EQ(32, lists->offsets[512]);
-  for (size_t i = 0; i < longs->numElements; ++i) {
-    EXPECT_EQ(1268 + i, longs->data[i]);
-  }
-
-  reader->next(batch, 512, 0);
-  ASSERT_EQ(512, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(512, lists->numElements);
-  ASSERT_EQ(true, lists->hasNulls);
-  ASSERT_EQ(748, longs->numElements);
-  ASSERT_EQ(true, !longs->hasNulls);
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    EXPECT_EQ(i % 2 == 0, lists->notNull[i]) << "Wrong value at " << i;
-    if (i < 24) {
-      EXPECT_EQ(0, lists->offsets[i]) << "Wrong value at " << i;
-    } else if (i < 510) {
-      EXPECT_EQ(3 * ((i - 23) / 2), lists->offsets[i])
-          << "Wrong value at " << i;
-    } else if (i < 511) {
-      EXPECT_EQ(729, lists->offsets[i]) << "Wrong value at " << i;
-    } else {
-      EXPECT_EQ(748, lists->offsets[i]) << "Wrong value at " << i;
-    }
-  }
-  EXPECT_EQ(748, lists->offsets[512]);
-  for (size_t i = 0; i < longs->numElements; ++i) {
-    EXPECT_EQ(1300 + i, longs->data[i]);
-  }
-}
-
-TEST(TestColumnReader, testListSkipWithNulls) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(3, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  // [0xaa for x in range(2048/8)]
-  const unsigned char buffer1[] =  { 0x7f, 0xaa, 0x7b, 0xaa };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  // [1 for x in range(260)] +
-  // [4 for x in range(260)] +
-  // [0 for x in range(260)] +
-  // [3 for x in range(243)] +
-  // [19]
-  const unsigned char buffer2[] = { 0x7f, 0x00, 0x01,
-                                    0x7f, 0x00, 0x01,
-                                    0x7f, 0x00, 0x04,
-                                    0x7f, 0x00, 0x04,
-                                    0x7f, 0x00, 0x00,
-                                    0x7f, 0x00, 0x00,
-                                    0x7f, 0x00, 0x03,
-                                    0x6e, 0x00, 0x03,
-                                    0xff, 0x13 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer2, ARRAY_SIZE(buffer2))));
-
-  // range(2048)
-  const unsigned char buffer3[] = { 0x7f, 0x01, 0x00,
-                                    0x7f, 0x01, 0x84, 0x02,
-                                    0x7f, 0x01, 0x88, 0x04,
-                                    0x7f, 0x01, 0x8c, 0x06,
-                                    0x7f, 0x01, 0x90, 0x08,
-                                    0x7f, 0x01, 0x94, 0x0a,
-                                    0x7f, 0x01, 0x98, 0x0c,
-                                    0x7f, 0x01, 0x9c, 0x0e,
-                                    0x7f, 0x01, 0xa0, 0x10,
-                                    0x7f, 0x01, 0xa4, 0x12,
-                                    0x7f, 0x01, 0xa8, 0x14,
-                                    0x7f, 0x01, 0xac, 0x16,
-                                    0x7f, 0x01, 0xb0, 0x18,
-                                    0x7f, 0x01, 0xb4, 0x1a,
-                                    0x7f, 0x01, 0xb8, 0x1c,
-                                    0x5f, 0x01, 0xbc, 0x1e };
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer3, ARRAY_SIZE(buffer3))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createListType(createPrimitiveType(LONG)), "col0");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
-
-  StructVectorBatch batch(1, *getDefaultPool());
-  ListVectorBatch *lists = new ListVectorBatch(1, *getDefaultPool());
-  LongVectorBatch *longs = new LongVectorBatch(1, *getDefaultPool());
-  batch.fields.push_back(lists);
-  lists->elements = std::unique_ptr < ColumnVectorBatch > (longs);
-
-  reader->next(batch, 1, 0);
-  ASSERT_EQ(1, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(1, lists->numElements);
-  ASSERT_EQ(true, !lists->hasNulls);
-  ASSERT_EQ(1, longs->numElements);
-  ASSERT_EQ(true, !longs->hasNulls);
-  EXPECT_EQ(0, lists->offsets[0]);
-  EXPECT_EQ(1, lists->offsets[1]);
-  EXPECT_EQ(0, longs->data[0]);
-
-  reader->skip(13);
-  reader->next(batch, 1, 0);
-  ASSERT_EQ(1, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(1, lists->numElements);
-  ASSERT_EQ(true, !lists->hasNulls);
-  ASSERT_EQ(1, longs->numElements);
-  ASSERT_EQ(true, !longs->hasNulls);
-  EXPECT_EQ(0, lists->offsets[0]);
-  EXPECT_EQ(1, lists->offsets[1]);
-  EXPECT_EQ(7, longs->data[0]);
-
-  reader->skip(2031);
-  reader->next(batch, 2, 0);
-  ASSERT_EQ(2, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(2, lists->numElements);
-  ASSERT_EQ(true, lists->hasNulls);
-  ASSERT_EQ(19, longs->numElements);
-  ASSERT_EQ(true, !longs->hasNulls);
-  EXPECT_EQ(0, lists->offsets[0]);
-  EXPECT_EQ(19, lists->offsets[1]);
-  EXPECT_EQ(19, lists->offsets[2]);
-  for (size_t i = 0; i < longs->numElements; ++i) {
-    EXPECT_EQ(2029 + i, longs->data[i]);
-  }
-}
-
-TEST(TestColumnReader, testListSkipWithNullsNoData) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(2, true);
-  selectedColumns.push_back(false);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  // [0xaa for x in range(2048/8)]
-  const unsigned char buffer1[] = { 0x7f, 0xaa, 0x7b, 0xaa };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  // [1 for x in range(260)] +
-  // [4 for x in range(260)] +
-  // [0 for x in range(260)] +
-  // [3 for x in range(243)] +
-  // [19]
-  const unsigned char buffer2[] =  { 0x7f, 0x00, 0x01,
-                                     0x7f, 0x00, 0x01,
-                                     0x7f, 0x00, 0x04,
-                                     0x7f, 0x00, 0x04,
-                                     0x7f, 0x00, 0x00,
-                                     0x7f, 0x00, 0x00,
-                                     0x7f, 0x00, 0x03,
-                                     0x6e, 0x00, 0x03,
-                                     0xff, 0x13 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer2, ARRAY_SIZE(buffer2))));
-
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createListType(createPrimitiveType(LONG)), "col0");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
-
-  StructVectorBatch batch(1, *getDefaultPool());
-  ListVectorBatch *lists = new ListVectorBatch(1, *getDefaultPool());
-  batch.fields.push_back(lists);
-
-  reader->next(batch, 1, 0);
-  ASSERT_EQ(1, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(1, lists->numElements);
-  ASSERT_EQ(true, !lists->hasNulls);
-  EXPECT_EQ(0, lists->offsets[0]);
-  EXPECT_EQ(1, lists->offsets[1]);
-
-  reader->skip(13);
-  reader->next(batch, 1, 0);
-  ASSERT_EQ(1, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(1, lists->numElements);
-  ASSERT_EQ(true, !lists->hasNulls);
-  EXPECT_EQ(0, lists->offsets[0]);
-  EXPECT_EQ(1, lists->offsets[1]);
-
-  reader->skip(2031);
-  reader->next(batch, 2, 0);
-  ASSERT_EQ(2, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(2, lists->numElements);
-  ASSERT_EQ(true, lists->hasNulls);
-  EXPECT_EQ(0, lists->offsets[0]);
-  EXPECT_EQ(19, lists->offsets[1]);
-  EXPECT_EQ(19, lists->offsets[2]);
-}
-
-TEST(TestColumnReader, testMap) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(4, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(testing::_,
-          proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  // [2 for x in range(600)]
-  const unsigned char buffer1[] = { 0x7f, 0x00, 0x02,
-                                    0x7f, 0x00, 0x02,
-                                    0x7f, 0x00, 0x02,
-                                    0x7f, 0x00, 0x02,
-                                    0x4d, 0x00, 0x02 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-
-  // range(1200)
-  const unsigned char buffer2[] =  { 0x7f, 0x01, 0x00,
-                                     0x7f, 0x01, 0x84, 0x02,
-                                     0x7f, 0x01, 0x88, 0x04,
-                                     0x7f, 0x01, 0x8c, 0x06,
-                                     0x7f, 0x01, 0x90, 0x08,
-                                     0x7f, 0x01, 0x94, 0x0a,
-                                     0x7f, 0x01, 0x98, 0x0c,
-                                     0x7f, 0x01, 0x9c, 0x0e,
-                                     0x7f, 0x01, 0xa0, 0x10,
-                                     0x1b, 0x01, 0xa4, 0x12 };
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer2, ARRAY_SIZE(buffer2))));
-
-  // range(8, 1208)
-  const unsigned char buffer3[] =  { 0x7f, 0x01, 0x10,
-                                     0x7f, 0x01, 0x94, 0x02,
-                                     0x7f, 0x01, 0x98, 0x04,
-                                     0x7f, 0x01, 0x9c, 0x06,
-                                     0x7f, 0x01, 0xa0, 0x08,
-                                     0x7f, 0x01, 0xa4, 0x0a,
-                                     0x7f, 0x01, 0xa8, 0x0c,
-                                     0x7f, 0x01, 0xac, 0x0e,
-                                     0x7f, 0x01, 0xb0, 0x10,
-                                     0x1b, 0x01, 0xb4, 0x12 };
-  EXPECT_CALL(streams, getStreamProxy(3, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer3, ARRAY_SIZE(buffer3))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createMapType(createPrimitiveType(LONG),
-                                        createPrimitiveType(LONG)),
-                          "col0");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
-
-  StructVectorBatch batch(512, *getDefaultPool());
-  MapVectorBatch *maps = new MapVectorBatch(512, *getDefaultPool());
-  LongVectorBatch *keys = new LongVectorBatch(512, *getDefaultPool());
-  LongVectorBatch *elements = new LongVectorBatch(512, *getDefaultPool());
-  batch.fields.push_back(maps);
-  maps->keys = std::unique_ptr < ColumnVectorBatch > (keys);
-  maps->elements = std::unique_ptr < ColumnVectorBatch > (elements);
-  reader->next(batch, 512, 0);
-  ASSERT_EQ(512, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(512, maps->numElements);
-  ASSERT_EQ(true, !maps->hasNulls);
-  ASSERT_EQ(1024, keys->numElements);
-  ASSERT_EQ(true, !keys->hasNulls);
-  ASSERT_EQ(1024, elements->numElements);
-  ASSERT_EQ(true, !elements->hasNulls);
-  for (size_t i = 0; i <= batch.numElements; ++i) {
-    EXPECT_EQ(2 * i, maps->offsets[i]);
-  }
-  for (size_t i = 0; i < keys->numElements; ++i) {
-    EXPECT_EQ(i, keys->data[i]);
-    EXPECT_EQ(i + 8, elements->data[i]);
-  }
-}
-
-TEST(TestColumnReader, testMapWithNulls) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(4, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(0, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  // [0xaa for x in range(2048/8)]
-  const unsigned char buffer1[] = { 0x7f, 0xaa, 0x7b, 0xaa };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  // [0x55 for x in range(2048/8)]
-  const unsigned char buffer2[] =  { 0x7f, 0x55, 0x7b, 0x55 };
-  EXPECT_CALL(streams, getStreamProxy(3, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer2, ARRAY_SIZE(buffer2))));
-
-  // [1 for x in range(260)] +
-  // [4 for x in range(260)] +
-  // [0 for x in range(260)] +
-  // [3 for x in range(243)] +
-  // [19]
-  const unsigned char buffer3[] = { 0x7f, 0x00, 0x01,
-                                    0x7f, 0x00, 0x01,
-                                    0x7f, 0x00, 0x04,
-                                    0x7f, 0x00, 0x04,
-                                    0x7f, 0x00, 0x00,
-                                    0x7f, 0x00, 0x00,
-                                    0x7f, 0x00, 0x03,
-                                    0x6e, 0x00, 0x03,
-                                    0xff, 0x13 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer3, ARRAY_SIZE(buffer3))));
-
-  // range(2048)
-  const unsigned char buffer4[] =  { 0x7f, 0x01, 0x00,
-                                     0x7f, 0x01, 0x84, 0x02,
-                                     0x7f, 0x01, 0x88, 0x04,
-                                     0x7f, 0x01, 0x8c, 0x06,
-                                     0x7f, 0x01, 0x90, 0x08,
-                                     0x7f, 0x01, 0x94, 0x0a,
-                                     0x7f, 0x01, 0x98, 0x0c,
-                                     0x7f, 0x01, 0x9c, 0x0e,
-                                     0x7f, 0x01, 0xa0, 0x10,
-                                     0x7f, 0x01, 0xa4, 0x12,
-                                     0x7f, 0x01, 0xa8, 0x14,
-                                     0x7f, 0x01, 0xac, 0x16,
-                                     0x7f, 0x01, 0xb0, 0x18,
-                                     0x7f, 0x01, 0xb4, 0x1a,
-                                     0x7f, 0x01, 0xb8, 0x1c,
-                                     0x5f, 0x01, 0xbc, 0x1e };
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer4, ARRAY_SIZE(buffer4))));
-
-  // range(8, 1032)
-  const unsigned char buffer5[] = { 0x7f, 0x01, 0x10,
-                                    0x7f, 0x01, 0x94, 0x02,
-                                    0x7f, 0x01, 0x98, 0x04,
-                                    0x7f, 0x01, 0x9c, 0x06,
-                                    0x7f, 0x01, 0xa0, 0x08,
-                                    0x7f, 0x01, 0xa4, 0x0a,
-                                    0x7f, 0x01, 0xa8, 0x0c,
-                                    0x6f, 0x01, 0xac, 0x0e };
-  EXPECT_CALL(streams, getStreamProxy(3, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer5, ARRAY_SIZE(buffer5))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createMapType(createPrimitiveType(LONG),
-                                        createPrimitiveType(LONG)),
-                          "col0");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
-
-  StructVectorBatch batch(512, *getDefaultPool());
-  MapVectorBatch *maps = new MapVectorBatch(512, *getDefaultPool());
-  LongVectorBatch *keys = new LongVectorBatch(512, *getDefaultPool());
-  LongVectorBatch *elements = new LongVectorBatch(512, *getDefaultPool());
-  batch.fields.push_back(maps);
-  maps->keys = std::unique_ptr < ColumnVectorBatch > (keys);
-  maps->elements = std::unique_ptr < ColumnVectorBatch > (elements);
-  reader->next(batch, 512, 0);
-  ASSERT_EQ(512, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(512, maps->numElements);
-  ASSERT_EQ(true, maps->hasNulls);
-  ASSERT_EQ(256, keys->numElements);
-  ASSERT_EQ(true, !keys->hasNulls);
-  ASSERT_EQ(256, elements->numElements);
-  ASSERT_EQ(true, elements->hasNulls);
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    EXPECT_EQ(i % 2 == 0, maps->notNull[i]) << "Wrong value at " << i;
-    EXPECT_EQ((i + 1) / 2, maps->offsets[i]) << "Wrong value at " << i;
-  }
-  EXPECT_EQ(256, maps->offsets[512]);
-  for (size_t i = 0; i < keys->numElements; ++i) {
-    EXPECT_EQ(i, keys->data[i]);
-    EXPECT_EQ(i & 1, elements->notNull[i]);
-    if (elements->notNull[i]) {
-      EXPECT_EQ(i / 2 + 8, elements->data[i]);
-    }
-  }
-
-  reader->next(batch, 512, 0);
-  ASSERT_EQ(512, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(512, maps->numElements);
-  ASSERT_EQ(true, maps->hasNulls);
-  ASSERT_EQ(1012, keys->numElements);
-  ASSERT_EQ(true, !keys->hasNulls);
-  ASSERT_EQ(1012, elements->numElements);
-  ASSERT_EQ(true, elements->hasNulls);
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    EXPECT_EQ(i % 2 == 0, maps->notNull[i]) << "Wrong value at " << i;
-    if (i < 8) {
-      EXPECT_EQ((i + 1) / 2, maps->offsets[i])
-          << "Wrong value at " << i;
-    } else {
-      EXPECT_EQ(4 * ((i + 1) / 2) - 12, maps->offsets[i])
-          << "Wrong value at " << i;
-    }
-  }
-  EXPECT_EQ(1012, maps->offsets[512]);
-  for (size_t i = 0; i < keys->numElements; ++i) {
-    EXPECT_EQ(256 + i, keys->data[i]);
-    EXPECT_EQ(i & 1, elements->notNull[i]);
-    if (elements->notNull[i]) {
-      EXPECT_EQ(128 + 8 + i / 2, elements->data[i]);
-    }
-  }
-
-  reader->next(batch, 512, 0);
-  ASSERT_EQ(512, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(512, maps->numElements);
-  ASSERT_EQ(true, maps->hasNulls);
-  ASSERT_EQ(32, keys->numElements);
-  ASSERT_EQ(true, !keys->hasNulls);
-  ASSERT_EQ(32, elements->numElements);
-  ASSERT_EQ(true, elements->hasNulls);
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    EXPECT_EQ(i % 2 == 0, maps->notNull[i]) << "Wrong value at " << i;
-    if (i < 16) {
-      EXPECT_EQ(4 * ((i + 1) / 2), maps->offsets[i])
-          << "Wrong value at " << i;
-    } else {
-      EXPECT_EQ(32, maps->offsets[i]) << "Wrong value at " << i;
-    }
-  }
-  EXPECT_EQ(32, maps->offsets[512]);
-  for (size_t i = 0; i < keys->numElements; ++i) {
-    EXPECT_EQ(1268 + i, keys->data[i]);
-    EXPECT_EQ(i & 1, elements->notNull[i]);
-    if (elements->notNull[i]) {
-      EXPECT_EQ(634 + 8 + i / 2, elements->data[i]);
-    }
-  }
-
-  reader->next(batch, 512, 0);
-  ASSERT_EQ(512, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(512, maps->numElements);
-  ASSERT_EQ(true, maps->hasNulls);
-  ASSERT_EQ(748, keys->numElements);
-  ASSERT_EQ(true, !keys->hasNulls);
-  ASSERT_EQ(748, elements->numElements);
-  ASSERT_EQ(true, elements->hasNulls);
-  for (size_t i = 0; i < batch.numElements; ++i) {
-    EXPECT_EQ(i % 2 == 0, maps->notNull[i]) << "Wrong value at " << i;
-    if (i < 24) {
-      EXPECT_EQ(0, maps->offsets[i]) << "Wrong value at " << i;
-    } else if (i < 510) {
-      EXPECT_EQ(3 * ((i - 23) / 2), maps->offsets[i])
-          << "Wrong value at " << i;
-    } else if (i < 511) {
-      EXPECT_EQ(729, maps->offsets[i]) << "Wrong value at " << i;
-    } else {
-      EXPECT_EQ(748, maps->offsets[i]) << "Wrong value at " << i;
-    }
-  }
-  EXPECT_EQ(748, maps->offsets[512]);
-  for (size_t i = 0; i < keys->numElements; ++i) {
-    EXPECT_EQ(1300 + i, keys->data[i]);
-    EXPECT_EQ(i & 1, elements->notNull[i]);
-    if (elements->notNull[i]) {
-      EXPECT_EQ(650 + 8 + i / 2, elements->data[i]);
-    }
-  }
-}
-
-TEST(TestColumnReader, testMapSkipWithNulls) {
-  MockStripeStreams streams;
-
-  // set getSelectedColumns()
-  std::vector<bool> selectedColumns(4, true);
-  EXPECT_CALL(streams, getSelectedColumns())
-      .WillRepeatedly(testing::Return(selectedColumns));
-
-  // set getEncoding
-  proto::ColumnEncoding directEncoding;
-  directEncoding.set_kind(proto::ColumnEncoding_Kind_DIRECT);
-  EXPECT_CALL(streams, getEncoding(testing::_))
-      .WillRepeatedly(testing::Return(directEncoding));
-
-  // set getStream
-  EXPECT_CALL(streams, getStreamProxy(testing::_,proto::Stream_Kind_PRESENT,
-                                      true))
-      .WillRepeatedly(testing::Return(nullptr));
-
-  // [0xaa for x in range(2048/8)]
-  const unsigned char buffer1[] = { 0x7f, 0xaa, 0x7b, 0xaa };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_PRESENT, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer1, ARRAY_SIZE(buffer1))));
-
-  // [1 for x in range(260)] +
-  // [4 for x in range(260)] +
-  // [0 for x in range(260)] +
-  // [3 for x in range(243)] +
-  // [19]
-  const unsigned char buffer2[] = { 0x7f, 0x00, 0x01,
-                                    0x7f, 0x00, 0x01,
-                                    0x7f, 0x00, 0x04,
-                                    0x7f, 0x00, 0x04,
-                                    0x7f, 0x00, 0x00,
-                                    0x7f, 0x00, 0x00,
-                                    0x7f, 0x00, 0x03,
-                                    0x6e, 0x00, 0x03,
-                                    0xff, 0x13 };
-  EXPECT_CALL(streams, getStreamProxy(1, proto::Stream_Kind_LENGTH, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer2, ARRAY_SIZE(buffer2))));
-
-  // range(2048)
-  const unsigned char buffer3[] =  { 0x7f, 0x01, 0x00,
-                                     0x7f, 0x01, 0x84, 0x02,
-                                     0x7f, 0x01, 0x88, 0x04,
-                                     0x7f, 0x01, 0x8c, 0x06,
-                                     0x7f, 0x01, 0x90, 0x08,
-                                     0x7f, 0x01, 0x94, 0x0a,
-                                     0x7f, 0x01, 0x98, 0x0c,
-                                     0x7f, 0x01, 0x9c, 0x0e,
-                                     0x7f, 0x01, 0xa0, 0x10,
-                                     0x7f, 0x01, 0xa4, 0x12,
-                                     0x7f, 0x01, 0xa8, 0x14,
-                                     0x7f, 0x01, 0xac, 0x16,
-                                     0x7f, 0x01, 0xb0, 0x18,
-                                     0x7f, 0x01, 0xb4, 0x1a,
-                                     0x7f, 0x01, 0xb8, 0x1c,
-                                     0x5f, 0x01, 0xbc, 0x1e };
-  EXPECT_CALL(streams, getStreamProxy(2, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer3, ARRAY_SIZE(buffer3))));
-
-  // range(8, 2056)
-  const unsigned char buffer4[] =  { 0x7f, 0x01, 0x10,
-                                     0x7f, 0x01, 0x94, 0x02,
-                                     0x7f, 0x01, 0x98, 0x04,
-                                     0x7f, 0x01, 0x9c, 0x06,
-                                     0x7f, 0x01, 0xa0, 0x08,
-                                     0x7f, 0x01, 0xa4, 0x0a,
-                                     0x7f, 0x01, 0xa8, 0x0c,
-                                     0x7f, 0x01, 0xac, 0x0e,
-                                     0x7f, 0x01, 0xb0, 0x10,
-                                     0x7f, 0x01, 0xb4, 0x12,
-                                     0x7f, 0x01, 0xb8, 0x14,
-                                     0x7f, 0x01, 0xbc, 0x16,
-                                     0x7f, 0x01, 0xc0, 0x18,
-                                     0x7f, 0x01, 0xc4, 0x1a,
-                                     0x7f, 0x01, 0xc8, 0x1c,
-                                     0x5f, 0x01, 0xcc, 0x1e };
-  EXPECT_CALL(streams, getStreamProxy(3, proto::Stream_Kind_DATA, true))
-      .WillRepeatedly(testing::Return(new SeekableArrayInputStream
-                                      (buffer4, ARRAY_SIZE(buffer4))));
-
-  // create the row type
-  std::unique_ptr<Type> rowType = createStructType();
-  rowType->addStructField(createMapType(createPrimitiveType(LONG),
-                                        createPrimitiveType(LONG)),
-                          "col0");
-  rowType->assignIds(0);
-
-  std::unique_ptr<ColumnReader> reader = buildReader(*rowType, streams);
-
-  StructVectorBatch batch(1, *getDefaultPool());
-  MapVectorBatch *maps = new MapVectorBatch(1, *getDefaultPool());
-  LongVectorBatch *keys = new LongVectorBatch(1, *getDefaultPool());
-  LongVectorBatch *elements = new LongVectorBatch(1, *getDefaultPool());
-  batch.fields.push_back(maps);
-  maps->keys = std::unique_ptr < ColumnVectorBatch > (keys);
-  maps->elements = std::unique_ptr < ColumnVectorBatch > (elements);
-
-  reader->next(batch, 1, 0);
-  ASSERT_EQ(1, batch.numElements);
-  ASSERT_EQ(true, !batch.hasNulls);
-  ASSERT_EQ(1, maps->numElements);
-  ASSERT_EQ(true, !maps->hasNulls);
-  ASSERT_EQ(1, keys->numElements);
-  ASSERT_EQ(true, !keys->hasNulls);
-  ASSERT_EQ(1, elements->numElements);
-  ASSERT_EQ(true, !elements->hasNulls);
-  EXPECT_EQ(0, maps->offsets[0]);
-  EXPECT_EQ(1, maps->offsets[1]);
-  EXPECT_EQ(0, keys->data[0]);
-  EXPECT_EQ(8, elements->data[

<TRUNCATED>

[02/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/tools/test/TestReader.cc
----------------------------------------------------------------------
diff --git a/tools/test/TestReader.cc b/tools/test/TestReader.cc
new file mode 100644
index 0000000..72aa5fd
--- /dev/null
+++ b/tools/test/TestReader.cc
@@ -0,0 +1,2950 @@
+/**
+ * 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 "orc/ColumnPrinter.hh"
+#include "orc/OrcFile.hh"
+
+#include "Adaptor.hh"
+#include "gzip.hh"
+#include "ToolTest.hh"
+
+#include "wrap/gmock.h"
+#include "wrap/gtest-wrapper.h"
+
+#include <sstream>
+
+#ifdef __clang__
+  DIAGNOSTIC_IGNORE("-Wmissing-variable-declarations")
+#endif
+
+namespace orc {
+
+  class OrcFileDescription {
+  public:
+    std::string filename;
+    std::string json;
+    std::string typeString;
+    std::string formatVersion;
+    uint64_t rowCount;
+    uint64_t contentLength;
+    uint64_t stripeCount;
+    CompressionKind compression;
+    size_t compressionSize;
+    uint64_t rowIndexStride;
+    std::map<std::string, std::string> userMeta;
+
+    OrcFileDescription(const std::string& _filename,
+                       const std::string& _json,
+                       const std::string& _typeString,
+                       const std::string& _version,
+                       uint64_t _rowCount,
+                       uint64_t _contentLength,
+                       uint64_t _stripeCount,
+                       CompressionKind _compression,
+                       size_t _compressionSize,
+                       uint64_t _rowIndexStride,
+                       const std::map<std::string, std::string>& _meta
+                       ): filename(_filename),
+                          json(_json),
+                          typeString(_typeString),
+                          formatVersion(_version),
+                          rowCount(_rowCount),
+                          contentLength(_contentLength),
+                          stripeCount(_stripeCount),
+                          compression(_compression),
+                          compressionSize(_compressionSize),
+                          rowIndexStride(_rowIndexStride),
+                          userMeta(_meta) {
+      // PASS
+    }
+
+    friend std::ostream& operator<< (std::ostream& stream,
+                                     OrcFileDescription const& obj);
+  };
+
+  std::ostream& operator<< (std::ostream& stream,
+                            OrcFileDescription const& obj) {
+    stream << obj.filename;
+    return stream;
+  }
+
+  class MatchTest: public testing::TestWithParam<OrcFileDescription> {
+  public:
+    virtual ~MatchTest();
+
+    std::string getFilename() {
+      std::ostringstream filename;
+      filename << exampleDirectory << "/" << GetParam().filename;
+      return filename.str();
+    }
+
+    std::string getJsonFilename() {
+      std::ostringstream filename;
+      filename << exampleDirectory << "/expected/" << GetParam().json;
+      return filename.str();
+    }
+  };
+
+  MatchTest::~MatchTest() {
+    // PASS
+  }
+
+  TEST_P(MatchTest, Metadata) {
+    orc::ReaderOptions opts;
+    std::unique_ptr<Reader> reader =
+      createReader(readLocalFile(getFilename()), opts);
+
+    EXPECT_EQ(GetParam().compression, reader->getCompression());
+    EXPECT_EQ(GetParam().compressionSize, reader->getCompressionSize());
+    EXPECT_EQ(GetParam().stripeCount, reader->getNumberOfStripes());
+    EXPECT_EQ(GetParam().rowCount, reader->getNumberOfRows());
+    EXPECT_EQ(GetParam().rowIndexStride, reader->getRowIndexStride());
+    EXPECT_EQ(GetParam().contentLength, reader->getContentLength());
+    EXPECT_EQ(GetParam().formatVersion, reader->getFormatVersion());
+    EXPECT_EQ(getFilename(), reader->getStreamName());
+    EXPECT_EQ(GetParam().userMeta.size(), reader->getMetadataKeys().size());
+    for(std::map<std::string, std::string>::const_iterator itr =
+          GetParam().userMeta.begin();
+        itr != GetParam().userMeta.end();
+        ++itr) {
+      ASSERT_EQ(true, reader->hasMetadataValue(itr->first));
+      std::string val = reader->getMetadataValue(itr->first);
+      EXPECT_EQ(itr->second, val);
+    }
+    EXPECT_EQ(true, !reader->hasMetadataValue("foo"));
+    EXPECT_EQ(18446744073709551615UL, reader->getRowNumber());
+
+    EXPECT_EQ(GetParam().typeString, reader->getType().toString());
+  }
+
+  std::string getOutput(FILE* outputFile) {
+    size_t posn = static_cast<size_t>(ftell(outputFile));
+    rewind(outputFile);
+    char *buffer = new char[posn];
+    size_t sizeRead = fread(buffer, 1, posn, outputFile);
+    if (sizeRead != posn) {
+      throw std::runtime_error("Bad read");
+    }
+    rewind(outputFile);
+    return std::string(buffer, posn);
+  }
+
+  TEST_P(MatchTest, Contents) {
+    orc::ReaderOptions opts;
+    std::unique_ptr<Reader> reader =
+      createReader(readLocalFile(getFilename()), opts);
+    unsigned long rowCount = 0;
+    std::unique_ptr<ColumnVectorBatch> batch = reader->createRowBatch(1024);
+    std::string line;
+    std::unique_ptr<orc::ColumnPrinter> printer =
+      orc::createColumnPrinter(line, reader->getType());
+    GzipTextReader expected(getJsonFilename());
+    std::string expectedLine;
+    while (reader->next(*batch)) {
+      EXPECT_EQ(rowCount, reader->getRowNumber());
+      printer->reset(*batch);
+      for(size_t i=0; i < batch->numElements; ++i) {
+        ASSERT_EQ(true, expected.nextLine(expectedLine));
+        line.clear();
+        printer->printRow(i);
+        EXPECT_EQ(expectedLine, line)
+          << "wrong output at row " << (rowCount + i);
+      }
+      rowCount += batch->numElements;
+    }
+    EXPECT_EQ(GetParam().rowCount, rowCount);
+    EXPECT_EQ(GetParam().rowCount, reader->getRowNumber());
+  }
+
+  std::map<std::string, std::string> makeMetadata();
+
+  INSTANTIATE_TEST_CASE_P(TestReader, MatchTest,
+    testing::Values(
+                    OrcFileDescription("TestOrcFile.columnProjection.orc",
+                                       "TestOrcFile.columnProjection.jsn.gz",
+                                       "struct<int1:int,string1:string>",
+                                       "0.12",
+                                       21000,
+                                       428406,
+                                       5,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       1000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.emptyFile.orc",
+                                       "TestOrcFile.emptyFile.jsn.gz",
+                                       "struct<boolean1:boolean,byte1:tinyint,"
+                                       "short1:smallint,int1:int,long1:bigint,"
+                                       "float1:float,double1:double,"
+                                       "bytes1:binary,string1:string,"
+                                       "middle:struct<list:array<struct<"
+                                       "int1:int,string1:string>>>,"
+                                       "list:array<struct<int1:int,string1:"
+                                       "string>>,map:map<string,struct<int1:"
+                                       "int,string1:string>>>",
+                                       "0.12",
+                                       0,
+                                       3,
+                                       0,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.metaData.orc",
+                                       "TestOrcFile.metaData.jsn.gz",
+                                       "struct<boolean1:boolean,byte1:tinyint,"
+                                       "short1:smallint,int1:int,long1:bigint,"
+                                       "float1:float,double1:double,"
+                                       "bytes1:binary,string1:string,"
+                                       "middle:struct<list:array<struct<"
+                                       "int1:int,string1:string>>>,"
+                                       "list:array<struct<int1:int,string1:"
+                                       "string>>,map:map<string,struct<int1:"
+                                       "int,string1:string>>>",
+                                       "0.12",
+                                       1,
+                                       980,
+                                       1,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       10000,
+                                       makeMetadata()),
+                    OrcFileDescription("TestOrcFile.test1.orc",
+                                       "TestOrcFile.test1.jsn.gz",
+                                       "struct<boolean1:boolean,byte1:tinyint,"
+                                       "short1:smallint,int1:int,long1:bigint,"
+                                       "float1:float,double1:double,"
+                                       "bytes1:binary,string1:string,"
+                                       "middle:struct<list:array<struct<"
+                                       "int1:int,string1:string>>>,"
+                                       "list:array<struct<int1:int,string1:"
+                                       "string>>,map:map<string,struct<int1:"
+                                       "int,string1:string>>>",
+                                       "0.12",
+                                       2,
+                                       1015,
+                                       1,
+                                       CompressionKind_ZLIB,
+                                       10000,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testMemoryManagementV11"
+                                       ".orc",
+                                       "TestOrcFile.testMemoryManagementV11"
+                                       ".jsn.gz",
+                                       "struct<int1:int,string1:string>",
+                                       "0.11",
+                                       2500,
+                                       18779,
+                                       25,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       0,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testMemoryManagementV12"
+                                       ".orc",
+                                       "TestOrcFile.testMemoryManagementV12"
+                                       ".jsn.gz",
+                                       "struct<int1:int,string1:string>",
+                                       "0.12",
+                                       2500,
+                                       10618,
+                                       4,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       0,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testPredicatePushdown.orc",
+                                       "TestOrcFile.testPredicatePushdown"
+                                       ".jsn.gz",
+                                       "struct<int1:int,string1:string>",
+                                       "0.12",
+                                       3500,
+                                       15529,
+                                       1,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       1000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testSeek.orc",
+                                       "TestOrcFile.testSeek.jsn.gz",
+                                       "struct<boolean1:boolean,byte1:tinyint,"
+                                       "short1:smallint,int1:int,long1:bigint,"
+                                       "float1:float,double1:double,bytes1:"
+                                       "binary,string1:string,middle:struct<"
+                                       "list:array<struct<int1:int,string1:"
+                                       "string>>>,list:array<struct<int1:int,"
+                                       "string1:string>>,map:map<string,"
+                                       "struct<int1:int,string1:string>>>",
+                                       "0.12",
+                                       32768,
+                                       1896379,
+                                       7,
+                                       CompressionKind_ZLIB,
+                                       65536,
+                                       1000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testSnappy.orc",
+                                       "TestOrcFile.testSnappy.jsn.gz",
+                                       "struct<int1:int,string1:string>",
+                                       "0.12",
+                                       10000,
+                                       126061,
+                                       2,
+                                       CompressionKind_SNAPPY,
+                                       100,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testStringAndBinaryStat"
+                                       "istics.orc",
+                                       "TestOrcFile.testStringAndBinaryStat"
+                                       "istics.jsn.gz",
+                                       "struct<bytes1:binary,string1:string>",
+                                       "0.12",
+                                       4,
+                                       185,
+                                       1,
+                                       CompressionKind_ZLIB,
+                                       10000,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testStripeLevelStats.orc",
+                                       "TestOrcFile.testStripeLevelStats"
+                                       ".jsn.gz",
+                                       "struct<int1:int,string1:string>",
+                                       "0.12",
+                                       11000,
+                                       597,
+                                       3,
+                                       CompressionKind_ZLIB,
+                                       10000,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testTimestamp.orc",
+                                       "TestOrcFile.testTimestamp.jsn.gz",
+                                       "timestamp",
+                                       "0.11",
+                                       12,
+                                       188,
+                                       1,
+                                       CompressionKind_ZLIB,
+                                       10000,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testUnionAndTimestamp.orc",
+                                       "TestOrcFile.testUnionAndTimestamp"
+                                       ".jsn.gz",
+                                       "struct<time:timestamp,union:uniontype"
+                                       "<int,string>,decimal:decimal(38,18)>",
+                                       "0.12",
+                                       5077,
+                                       20906,
+                                       2,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("TestOrcFile.testWithoutIndex.orc",
+                                       "TestOrcFile.testWithoutIndex.jsn.gz",
+                                       "struct<int1:int,string1:string>",
+                                       "0.12",
+                                       50000,
+                                       214643,
+                                       10,
+                                       CompressionKind_SNAPPY,
+                                       1000,
+                                       0,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("decimal.orc",
+                                       "decimal.jsn.gz",
+                                       "struct<_col0:decimal(10,5)>",
+                                       "0.12",
+                                       6000,
+                                       16186,
+                                       1,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("demo-11-none.orc",
+                                       "demo-12-zlib.jsn.gz",
+                                       ("struct<_col0:int,_col1:string,"
+                                        "_col2:string,_col3:string,_col4:int,"
+                                        "_col5:string,_col6:int,_col7:int,"
+                                        "_col8:int>"),
+                                       "0.11",
+                                       1920800,
+                                       5069718,
+                                       385,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("demo-11-zlib.orc",
+                                       "demo-12-zlib.jsn.gz",
+                                       ("struct<_col0:int,_col1:string,"
+                                        "_col2:string,_col3:string,_col4:int,"
+                                        "_col5:string,_col6:int,_col7:int,"
+                                        "_col8:int>"),
+                                       "0.11",
+                                       1920800,
+                                       396823,
+                                       385,
+                                       CompressionKind_ZLIB,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("demo-12-zlib.orc",
+                                       "demo-12-zlib.jsn.gz",
+                                       ("struct<_col0:int,_col1:string,"
+                                        "_col2:string,_col3:string,_col4:int,"
+                                        "_col5:string,_col6:int,_col7:int,"
+                                        "_col8:int>"),
+                                       "0.12",
+                                       1920800,
+                                       45592,
+                                       1,
+                                       CompressionKind_ZLIB,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("nulls-at-end-snappy.orc",
+                                       "nulls-at-end-snappy.jsn.gz",
+                                       ("struct<_col0:tinyint,_col1:smallint,"
+                                        "_col2:int,_col3:bigint,_col4:float,"
+                                        "_col5:double,_col6:boolean>"),
+                                       "0.12",
+                                       70000,
+                                       366347,
+                                       1,
+                                       CompressionKind_SNAPPY,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("orc-file-11-format.orc",
+                                       "orc-file-11-format.jsn.gz",
+                                       ("struct<boolean1:boolean,"
+                                        "byte1:tinyint,short1:smallint,"
+                                        "int1:int,long1:bigint,float1:float,"
+                                        "double1:double,bytes1:binary,"
+                                        "string1:string,middle:struct<list:"
+                                        "array<struct<int1:int,"
+                                        "string1:string>>>,list:array<struct"
+                                        "<int1:int,string1:string>>,map:map"
+                                        "<string,struct<int1:int,string1:"
+                                        "string>>,ts:timestamp,"
+                                        "decimal1:decimal(0,0)>"),
+                                       "0.11",
+                                       7500,
+                                       372542,
+                                       2,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("orc_split_elim.orc",
+                                       "orc_split_elim.jsn.gz",
+                                       ("struct<userid:bigint,string1:string,"
+                                        "subtype:double,decimal1:decimal(0,0),"
+                                        "ts:timestamp>"),
+                                       "0.12",
+                                       25000,
+                                       245568,
+                                       5,
+                                       CompressionKind_NONE,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>()),
+                    OrcFileDescription("over1k_bloom.orc",
+                                       "over1k_bloom.jsn.gz",
+                                       "struct<_col0:tinyint,_col1:smallint,"
+                                       "_col2:int,_col3:bigint,_col4:float,"
+                                       "_col5:double,_col6:boolean,"
+                                       "_col7:string,_col8:timestamp,"
+                                       "_col9:decimal(4,2),_col10:binary>",
+                                       "0.12",
+                                       2098,
+                                       41780,
+                                       2,
+                                       CompressionKind_ZLIB,
+                                       262144,
+                                       10000,
+                                       std::map<std::string, std::string>())
+                    ));
+
+#ifdef HAS_PRE_1970
+INSTANTIATE_TEST_CASE_P(TestReader1900, MatchTest,
+    testing::Values(
+                    OrcFileDescription("TestOrcFile.testDate1900.orc",
+                                       "TestOrcFile.testDate1900.jsn.gz",
+                                       "struct<time:timestamp,date:date>",
+                                       "0.12",
+                                       70000,
+                                       30478,
+                                       8,
+                                       CompressionKind_ZLIB,
+                                       10000,
+                                       10000,
+                                       std::map<std::string, std::string>())
+		    ));
+#endif
+
+#ifdef HAS_POST_2038
+  INSTANTIATE_TEST_CASE_P(TestReader2038, MatchTest,
+    testing::Values(
+                    OrcFileDescription("TestOrcFile.testDate2038.orc",
+                                       "TestOrcFile.testDate2038.jsn.gz",
+                                       "struct<time:timestamp,date:date>",
+                                       "0.12",
+                                       212000,
+                                       94762,
+                                       28,
+                                       CompressionKind_ZLIB,
+                                       10000,
+                                       10000,
+                                       std::map<std::string, std::string>())
+		    ));
+#endif
+
+  TEST(Reader, columnSelectionTest) {
+    ReaderOptions opts;
+    std::list<int64_t> includes;
+    for(int i=1; i < 10; i += 2) {
+      includes.push_back(i);
+    }
+    opts.include(includes);
+    std::ostringstream filename;
+    filename << exampleDirectory << "/demo-11-none.orc";
+    std::unique_ptr<Reader> reader =
+      createReader(readLocalFile(filename.str()), opts);
+
+    EXPECT_EQ(CompressionKind_NONE, reader->getCompression());
+    EXPECT_EQ(256 * 1024, reader->getCompressionSize());
+    EXPECT_EQ(385, reader->getNumberOfStripes());
+    EXPECT_EQ(1920800, reader->getNumberOfRows());
+    EXPECT_EQ(10000, reader->getRowIndexStride());
+    EXPECT_EQ(5069718, reader->getContentLength());
+    EXPECT_EQ(filename.str(), reader->getStreamName());
+    EXPECT_THAT(reader->getMetadataKeys(), testing::IsEmpty());
+    EXPECT_FALSE(reader->hasMetadataValue("foo"));
+    EXPECT_EQ(18446744073709551615UL, reader->getRowNumber());
+
+    const Type& rootType = reader->getType();
+    EXPECT_EQ(0, rootType.getColumnId());
+    EXPECT_EQ(STRUCT, rootType.getKind());
+    ASSERT_EQ(9, rootType.getSubtypeCount());
+    EXPECT_EQ("_col0", rootType.getFieldName(0));
+    EXPECT_EQ("_col1", rootType.getFieldName(1));
+    EXPECT_EQ("_col2", rootType.getFieldName(2));
+    EXPECT_EQ("_col3", rootType.getFieldName(3));
+    EXPECT_EQ("_col4", rootType.getFieldName(4));
+    EXPECT_EQ("_col5", rootType.getFieldName(5));
+    EXPECT_EQ("_col6", rootType.getFieldName(6));
+    EXPECT_EQ("_col7", rootType.getFieldName(7));
+    EXPECT_EQ("_col8", rootType.getFieldName(8));
+    EXPECT_EQ(INT, rootType.getSubtype(0).getKind());
+    EXPECT_EQ(STRING, rootType.getSubtype(1).getKind());
+    EXPECT_EQ(STRING, rootType.getSubtype(2).getKind());
+    EXPECT_EQ(STRING, rootType.getSubtype(3).getKind());
+    EXPECT_EQ(INT, rootType.getSubtype(4).getKind());
+    EXPECT_EQ(STRING, rootType.getSubtype(5).getKind());
+    EXPECT_EQ(INT, rootType.getSubtype(6).getKind());
+    EXPECT_EQ(INT, rootType.getSubtype(7).getKind());
+    EXPECT_EQ(INT, rootType.getSubtype(8).getKind());
+    for(unsigned int i=0; i < 9; ++i) {
+      EXPECT_EQ(i + 1, rootType.getSubtype(i).getColumnId())
+        << "fail on " << i;
+    }
+
+    const std::vector<bool> selected = reader->getSelectedColumns();
+    EXPECT_EQ(true, selected[0]) << "fail on " << 0;
+    for (size_t i = 1; i < 10; ++i) {
+      EXPECT_EQ(i%2==1?true:false, selected[i]) << "fail on " << i;
+    }
+
+    unsigned long rowCount = 0;
+    std::unique_ptr<ColumnVectorBatch> batch = reader->createRowBatch(1024);
+    LongVectorBatch* longVector =
+      dynamic_cast<LongVectorBatch*>
+      (dynamic_cast<StructVectorBatch&>(*batch).fields[0]);
+    int64_t* idCol = longVector->data.data();
+    while (reader->next(*batch)) {
+      EXPECT_EQ(rowCount, reader->getRowNumber());
+      for(unsigned int i=0; i < batch->numElements; ++i) {
+        EXPECT_EQ(rowCount + i + 1, idCol[i]) << "Bad id for " << i;
+      }
+      rowCount += batch->numElements;
+    }
+    EXPECT_EQ(1920800, rowCount);
+    EXPECT_EQ(1920800, reader->getRowNumber());
+  }
+
+  TEST(Reader, stripeInformationTest) {
+    ReaderOptions opts;
+    std::ostringstream filename;
+    filename << exampleDirectory << "/demo-11-none.orc";
+    std::unique_ptr<Reader> reader =
+      createReader(readLocalFile(filename.str()), opts);
+
+    EXPECT_EQ(385, reader->getNumberOfStripes());
+
+    std::unique_ptr<StripeInformation> stripeInfo = reader->getStripe(7);
+    EXPECT_EQ(92143, stripeInfo->getOffset());
+    EXPECT_EQ(13176, stripeInfo->getLength());
+    EXPECT_EQ(234, stripeInfo->getIndexLength());
+    EXPECT_EQ(12673, stripeInfo->getDataLength());
+    EXPECT_EQ(269, stripeInfo->getFooterLength());
+    EXPECT_EQ(5000, stripeInfo->getNumberOfRows());
+  }
+
+  TEST(Reader, readRangeTest) {
+    ReaderOptions fullOpts, lastOpts, oobOpts, offsetOpts;
+    // stripes[N-1]
+    lastOpts.range(5067085, 1);
+    // stripes[N]
+    oobOpts.range(5067086, 4096);
+    // stripes[7, 16]
+    offsetOpts.range(80000, 130722);
+    std::ostringstream filename;
+    filename << exampleDirectory << "/demo-11-none.orc";
+    std::unique_ptr<Reader> fullReader =
+      createReader(readLocalFile(filename.str()), fullOpts);
+    std::unique_ptr<Reader> lastReader =
+      createReader(readLocalFile(filename.str()), lastOpts);
+    std::unique_ptr<Reader> oobReader =
+      createReader(readLocalFile(filename.str()), oobOpts);
+    std::unique_ptr<Reader> offsetReader =
+      createReader(readLocalFile(filename.str()), offsetOpts);
+
+    std::unique_ptr<ColumnVectorBatch> oobBatch =
+      oobReader->createRowBatch(5000);
+    EXPECT_FALSE(oobReader->next(*oobBatch));
+
+    // advance fullReader to align with offsetReader
+    std::unique_ptr<ColumnVectorBatch> fullBatch =
+      fullReader->createRowBatch(5000);
+    for (int i=0; i < 7; ++i) {
+      EXPECT_TRUE(fullReader->next(*fullBatch));
+      EXPECT_EQ(5000, fullBatch->numElements);
+    }
+
+    std::unique_ptr<ColumnVectorBatch> offsetBatch =
+      offsetReader->createRowBatch(5000);
+    LongVectorBatch* fullLongVector =
+      dynamic_cast<LongVectorBatch*>
+      (dynamic_cast<StructVectorBatch&>(*fullBatch).fields[0]);
+    int64_t* fullId = fullLongVector->data.data();
+    LongVectorBatch* offsetLongVector =
+      dynamic_cast<LongVectorBatch*>
+      (dynamic_cast<StructVectorBatch&>(*offsetBatch).fields[0]);
+    int64_t* offsetId = offsetLongVector->data.data();
+    for (int i=7; i < 17; ++i) {
+      EXPECT_TRUE(fullReader->next(*fullBatch));
+      EXPECT_TRUE(offsetReader->next(*offsetBatch));
+      EXPECT_EQ(fullBatch->numElements, offsetBatch->numElements);
+      for (unsigned j=0; j < fullBatch->numElements; ++j) {
+        EXPECT_EQ(fullId[j], offsetId[j]);
+      }
+    }
+    EXPECT_FALSE(offsetReader->next(*offsetBatch));
+
+    // advance fullReader to align with lastReader
+    for (int i=17; i < 384; ++i) {
+      EXPECT_TRUE(fullReader->next(*fullBatch));
+      EXPECT_EQ(5000, fullBatch->numElements);
+    }
+
+    std::unique_ptr<ColumnVectorBatch> lastBatch =
+      lastReader->createRowBatch(5000);
+    LongVectorBatch* lastLongVector =
+      dynamic_cast<LongVectorBatch*>
+      (dynamic_cast<StructVectorBatch&>(*lastBatch).fields[0]);
+    int64_t* lastId = lastLongVector->data.data();
+    EXPECT_TRUE(fullReader->next(*fullBatch));
+    EXPECT_TRUE(lastReader->next(*lastBatch));
+    EXPECT_EQ(fullBatch->numElements, lastBatch->numElements);
+    for (unsigned i=0; i < fullBatch->numElements; ++i) {
+      EXPECT_EQ(fullId[i], lastId[i]);
+    }
+    EXPECT_FALSE(fullReader->next(*fullBatch));
+    EXPECT_FALSE(lastReader->next(*lastBatch));
+  }
+
+TEST(Reader, columnStatistics) {
+  orc::ReaderOptions opts;
+  std::ostringstream filename;
+  filename << exampleDirectory << "/demo-11-none.orc";
+  std::unique_ptr<orc::Reader> reader =
+    orc::createReader(orc::readLocalFile(filename.str()), opts);
+
+  // corrupt stats test
+  EXPECT_EQ(true, reader->hasCorrectStatistics());
+
+  // test column statistics
+  std::unique_ptr<orc::Statistics> stats = reader->getStatistics();
+  EXPECT_EQ(10, stats->getNumberOfColumns());
+
+  // 6th real column, start from 1
+  std::unique_ptr<orc::ColumnStatistics> col_6 =
+    reader->getColumnStatistics(6);
+  const orc::StringColumnStatistics& strStats =
+    dynamic_cast<const orc::StringColumnStatistics&> (*(col_6.get()));
+  EXPECT_EQ("Good", strStats.getMinimum());
+  EXPECT_EQ("Unknown", strStats.getMaximum());
+
+  // 7th real column
+  std::unique_ptr<orc::ColumnStatistics> col_7 =
+    reader->getColumnStatistics(7);
+  const orc::IntegerColumnStatistics& intStats =
+    dynamic_cast<const orc::IntegerColumnStatistics&> (*(col_7.get()));
+  EXPECT_EQ(0, intStats.getMinimum());
+  EXPECT_EQ(6, intStats.getMaximum());
+  EXPECT_EQ(5762400, intStats.getSum());
+}
+
+TEST(Reader, stripeStatistics) {
+  orc::ReaderOptions opts;
+  std::ostringstream filename;
+  filename << exampleDirectory << "/demo-11-none.orc";
+  std::unique_ptr<orc::Reader> reader =
+    orc::createReader(orc::readLocalFile(filename.str()), opts);
+
+  // test stripe statistics
+  EXPECT_EQ(385, reader->getNumberOfStripeStatistics());
+
+  // stripe[384]: 385th stripe, last stripe
+  unsigned long stripeIdx = 384;
+  std::unique_ptr<orc::Statistics> stripeStats =
+    reader->getStripeStatistics(stripeIdx);
+  EXPECT_EQ(10, stripeStats->getNumberOfColumns());
+
+  // 6th real column
+  const orc::StringColumnStatistics* col_6 =
+    dynamic_cast<const orc::StringColumnStatistics*>
+    (stripeStats->getColumnStatistics(6));
+  EXPECT_EQ("Unknown", col_6->getMinimum());
+  EXPECT_EQ("Unknown", col_6->getMaximum());
+
+  // 7th real column
+  const orc::IntegerColumnStatistics* col_7 =
+    dynamic_cast<const orc::IntegerColumnStatistics*>
+    (stripeStats->getColumnStatistics(7));
+  EXPECT_EQ(6, col_7->getMinimum());
+  EXPECT_EQ(6, col_7->getMaximum());
+  EXPECT_EQ(4800, col_7->getSum());
+}
+
+TEST(Reader, corruptStatistics) {
+  orc::ReaderOptions opts;
+  std::ostringstream filename;
+  // read the file has corrupt statistics
+  filename << exampleDirectory << "/orc_split_elim.orc";
+  std::unique_ptr<orc::Reader> reader =
+    orc::createReader(orc::readLocalFile(filename.str()), opts);
+
+  EXPECT_EQ(true, !reader->hasCorrectStatistics());
+
+  // 2nd real column, string
+  std::unique_ptr<orc::ColumnStatistics> col_2 =
+    reader->getColumnStatistics(2);
+  const orc::StringColumnStatistics& strStats =
+    dynamic_cast<const orc::StringColumnStatistics&> (*(col_2.get()));
+  EXPECT_EQ(true, !strStats.hasMinimum());
+  EXPECT_EQ(true, !strStats.hasMaximum());
+
+  // stripe statistics
+  unsigned long stripeIdx = 1;
+  std::unique_ptr<orc::Statistics> stripeStats =
+    reader->getStripeStatistics(stripeIdx);
+
+  // 4th real column, Decimal
+  const orc::DecimalColumnStatistics* col_4 =
+    dynamic_cast<const orc::DecimalColumnStatistics*>
+    (stripeStats->getColumnStatistics(4));
+  EXPECT_EQ(true, !col_4->hasMinimum());
+  EXPECT_EQ(true, !col_4->hasMaximum());
+}
+
+TEST(Reader, noStripeStatistics) {
+  orc::ReaderOptions opts;
+  std::ostringstream filename;
+  // read the file has no stripe statistics
+  filename << exampleDirectory << "/orc-file-11-format.orc";
+  std::unique_ptr<orc::Reader> reader =
+    orc::createReader(orc::readLocalFile(filename.str()), opts);
+
+  EXPECT_EQ(0, reader->getNumberOfStripeStatistics());
+}
+
+TEST(Reader, seekToRow) {
+  /* Test with a regular file */
+  {
+    orc::ReaderOptions opts;
+    std::ostringstream filename;
+    filename << exampleDirectory << "/demo-11-none.orc";
+    std::unique_ptr<orc::Reader> reader =
+        orc::createReader(orc::readLocalFile(filename.str()), opts);
+    EXPECT_EQ(1920800, reader->getNumberOfRows());
+
+    std::unique_ptr<orc::ColumnVectorBatch> batch =
+        reader->createRowBatch(5000); // Stripe size
+    reader->next(*batch);
+    EXPECT_EQ(5000, batch->numElements);
+    EXPECT_EQ(0, reader->getRowNumber());
+
+    // We only load data till the end of the current stripe
+    reader->seekToRow(11000);
+    reader->next(*batch);
+    EXPECT_EQ(4000, batch->numElements);
+    EXPECT_EQ(11000, reader->getRowNumber());
+
+    // We only load data till the end of the current stripe
+    reader->seekToRow(99999);
+    reader->next(*batch);
+    EXPECT_EQ(1, batch->numElements);
+    EXPECT_EQ(99999, reader->getRowNumber());
+
+    // Skip more rows than available
+    reader->seekToRow(1920800);
+    reader->next(*batch);
+    EXPECT_EQ(0, batch->numElements);
+    EXPECT_EQ(1920800, reader->getRowNumber());
+  }
+
+  /* Test with a portion of the file */
+  {
+    orc::ReaderOptions opts;
+    std::ostringstream filename;
+    filename << exampleDirectory << "/demo-11-none.orc";
+    opts.range(13126, 13145);   // Read only the second stripe (rows 5000..9999)
+
+    std::unique_ptr<orc::Reader> reader =
+        orc::createReader(orc::readLocalFile(filename.str()), opts);
+    EXPECT_EQ(1920800, reader->getNumberOfRows());
+
+    std::unique_ptr<orc::ColumnVectorBatch> batch =
+        reader->createRowBatch(5000); // Stripe size
+    reader->next(*batch);
+    EXPECT_EQ(5000, batch->numElements);
+
+    reader->seekToRow(7000);
+    reader->next(*batch);
+    EXPECT_EQ(3000, batch->numElements);
+    EXPECT_EQ(7000, reader->getRowNumber());
+
+    reader->seekToRow(1000);
+    reader->next(*batch);
+    EXPECT_EQ(0, batch->numElements);
+    EXPECT_EQ(10000, reader->getRowNumber());
+
+    reader->seekToRow(11000);
+    reader->next(*batch);
+    EXPECT_EQ(0, batch->numElements);
+    EXPECT_EQ(10000, reader->getRowNumber());
+  }
+
+  /* Test with an empty file */
+  {
+    orc::ReaderOptions opts;
+    std::ostringstream filename;
+    filename << exampleDirectory << "/TestOrcFile.emptyFile.orc";
+    std::unique_ptr<orc::Reader> reader =
+        orc::createReader(orc::readLocalFile(filename.str()), opts);
+    EXPECT_EQ(0, reader->getNumberOfRows());
+
+    std::unique_ptr<orc::ColumnVectorBatch> batch =
+        reader->createRowBatch(5000);
+    reader->next(*batch);
+    EXPECT_EQ(0, batch->numElements);
+
+    reader->seekToRow(0);
+    reader->next(*batch);
+    EXPECT_EQ(0, batch->numElements);
+    EXPECT_EQ(0, reader->getRowNumber());
+
+    reader->seekToRow(1);
+    reader->next(*batch);
+    EXPECT_EQ(0, batch->numElements);
+    EXPECT_EQ(0, reader->getRowNumber());
+  }
+}
+
+TEST(Reader, futureFormatVersion) {
+  std::ostringstream filename;
+  filename << exampleDirectory << "/version1999.orc";
+  orc::ReaderOptions opts;
+  std::ostringstream errorMsg;
+  opts.setErrorStream(errorMsg);
+  std::unique_ptr<orc::Reader> reader =
+    orc::createReader(orc::readLocalFile(filename.str()), opts);
+  EXPECT_EQ(("Warning: ORC file " + filename.str() +
+             " was written in an unknown format version 19.99\n"),
+            errorMsg.str());
+  EXPECT_EQ("19.99", reader->getFormatVersion());
+}
+
+  std::map<std::string, std::string> makeMetadata() {
+    std::map<std::string, std::string> result;
+    result["my.meta"] = "\x01\x02\x03\x04\x05\x06\x07\xff\xfe\x7f\x80";
+    result["clobber"] = "\x05\x07\x0b\x0d\x11\x13";
+    const unsigned char buffer[] = {
+96,180,32,187,56,81,217,212,122,203,147,61,190,112,57,155,246,201,45,163,
+58,240,29,79,183,112,233,140,3,37,244,29,62,186,248,152,109,167,18,200,
+43,205,77,85,75,240,181,64,35,194,155,98,77,233,239,156,47,147,30,252,
+88,15,154,251,8,27,18,225,7,177,232,5,242,180,245,240,241,208,12,45,
+15,98,99,70,112,146,28,80,88,103,255,32,246,168,51,94,152,175,135,37,
+56,85,134,180,31,239,242,5,180,224,90,0,8,35,247,139,95,143,92,2,
+67,156,232,246,122,120,29,144,203,230,191,26,231,242,188,64,164,151,9,160,
+108,14,49,73,155,240,41,105,202,66,210,3,229,102,188,198,150,222,8,250,
+1,2,160,253,46,35,48,176,150,74,187,124,68,48,32,222,28,173,9,191,
+214,56,31,251,148,218,175,187,144,196,237,145,160,97,58,209,220,75,71,3,
+175,132,193,214,59,26,135,105,33,198,213,134,157,97,204,185,142,209,58,230,
+192,154,19,252,145,225,73,34,243,1,207,139,207,147,67,21,166,4,157,47,
+7,217,131,250,169,27,143,78,114,101,236,184,21,167,203,171,193,69,12,183,
+43,60,116,16,119,23,170,36,172,66,242,91,108,103,132,118,125,14,53,70,
+196,247,37,1,145,163,182,170,162,182,77,18,110,85,131,176,76,17,50,89,
+201,72,225,208,179,155,185,86,12,213,64,155,110,202,254,219,200,172,175,238,
+167,77,183,248,90,223,148,190,154,133,161,221,75,3,170,136,131,29,210,156,
+64,120,129,11,58,40,210,45,102,128,182,79,203,177,178,55,194,68,18,52,
+206,171,191,218,216,124,49,21,72,246,121,2,116,185,46,106,89,29,58,177,
+166,11,115,64,11,196,116,197,45,60,188,242,251,174,114,182,230,212,159,176,
+177,133,19,54,250,44,84,12,223,191,120,200,219,73,44,101,231,91,1,242,
+86,10,157,196,86,254,164,3,66,134,86,158,48,134,234,100,151,36,149,156,
+68,8,146,218,235,114,76,6,229,19,58,201,170,148,16,238,186,45,84,254,
+138,253,248,80,125,33,19,226,2,106,147,122,228,57,152,43,206,121,204,36,
+14,38,74,246,205,67,204,48,37,102,107,185,179,127,129,231,20,21,103,173,
+104,201,130,73,128,219,236,205,20,219,32,168,112,88,1,113,21,182,16,82,
+57,102,75,118,131,96,156,24,178,230,82,218,235,107,179,120,187,63,120,128,
+92,187,129,182,154,249,239,253,13,66,111,145,188,18,34,248,23,137,195,144,
+219,40,101,90,30,2,29,96,158,157,25,12,222,85,168,201,35,237,85,18,
+244,72,205,0,24,151,115,7,95,78,212,28,87,207,209,123,139,190,52,2,
+21,185,209,248,226,186,91,123,35,200,54,253,59,89,143,89,220,21,119,196,
+157,151,25,184,177,147,91,128,45,32,152,163,112,27,106,65,71,222,102,86,
+183,253,36,79,11,33,22,190,57,233,40,217,234,84,4,186,183,105,85,13,
+246,140,86,48,227,219,154,2,253,245,209,49,130,27,150,90,189,48,247,209,
+133,115,52,22,177,69,12,237,253,3,174,217,74,188,227,213,6,248,240,0,
+139,40,121,189,39,22,210,78,95,141,155,182,188,127,23,136,50,15,115,77,
+90,104,55,235,130,241,252,79,85,60,247,70,138,139,90,240,208,165,207,223,
+54,19,46,197,250,49,33,156,0,163,227,139,104,148,156,232,107,6,11,38,
+177,188,99,11,39,114,53,250,170,178,143,165,54,168,3,82,2,136,46,127,
+132,245,39,53,8,50,183,129,196,69,203,125,221,220,75,133,165,11,85,51,
+102,209,201,93,140,62,231,96,186,105,70,122,120,4,111,141,220,91,55,180,
+196,21,73,55,1,233,160,82,217,81,160,70,186,207,251,2,21,167,243,244,
+173,73,119,179,108,45,221,194,204,113,207,190,145,114,114,189,43,62,119,155,
+172,71,73,123,222,80,46,60,228,51,229,145,135,85,152,99,68,212,96,24,
+130,253,85,233,201,56,209,85,202,2,218,42,157,135,181,148,16,122,0,1,
+192,42,13,152,157,54,208,67,183,89,5,15,237,166,143,228,253,63,51,193,
+49,111,45,76,60,34,122,56,16,175,204,109,163,7,81,95,98,98,3,176,
+210,97,62,97,194,56,147,104,49,69,75,174,26,166,97,90,176,204,132,43,
+164,94,248,171,60,143,223,88,243,250,139,189,116,106,229,216,246,180,249,228,
+94,165,148,214,32,29,120,148,50,95,204,0,21,223,192,130,110,177,133,10,
+141,63,221,79,208,177,227,165,69,121,76,138,241,231,3,157,67,149,29,249,
+144,163,34,5,177,90,23,157,207,59,205,105,17,141,244,6,237,108,194,224,
+175,115,99,176,73,25,78,173,104,163,90,161,171,115,1,77,71,204,93,209,
+42,92,79,248,134,238,185,189,41,78,155,81,245,102,165,161,60,245,208,105,
+215,200,156,80,249,39,109,202,174,11,23,192,253,242,228,5,151,61,178,48,
+178,91,35,105,32,63,92,31,146,225,101,237,187,27,154,182,4,86,70,58,
+62,234,219,238,252,181,158,168,17,211,164,79,12,22,202,150,251,117,143,135,
+137,184,169,5,166,127,209,45,232,222,164,137,84,95,39,29,140,34,175,77,
+103,199,22,175,142,238,38,204,148,135,22,97,80,99,131,209,10,110,169,151,
+217,77,22,13,211,196,203,240,73,64,176,65,46,195,189,136,228,13,47,11,
+191,118,213,54,140,68,243,158,192,78,111,85,155,134,217,132,158,35,35,64,
+128,51,239,49,161,95,76,188,142,106,237,81,147,97,85,23,213,114,117,58,
+133,96,185,67,8,196,113,114,31,144,76,48,181,159,167,115,30,23,58,76,
+96,47,183,19,234,37,43,194,58,195,128,33,120,49,237,11,142,230,42,181,
+195,150,8,22,31,218,88,209,166,197,104,228,0,114,22,181,229,21,222,136,
+185,163,236,240,158,167,236,21,174,18,105,188,124,184,6,136,39,236,158,185,
+97,185,201,18,108,57,229,44,103,188,64,45,200,197,71,247,94,153,43,226,
+126,159,221,223,62,247,181,89,237,101,57,238,24,83,100,252,113,212,82,2,
+149,177,109,147,207,152,105,10,6,246,175,154,40,85,251,150,130,114,234,69,
+195,0,42,61,185,54,96,131,52,128,205,92,92,127,218,241,171,148,200,158,
+68,126,190,55,105,251,67,90,197,19,234,232,175,17,21,97,215,11,245,4,
+173,94,10,192,235,149,50,70,45,84,95,166,173,12,54,171,19,56,73,242,
+10,75,178,73,237,203,77,225,40,206,97,16,39,189,165,91,52,80,236,57,
+153,127,42,236,57,110,219,35,139,189,122,217,84,219,136,154,107,83,56,173,
+5,174,77,186,194,170,21,139,112,8,202,77,40,135,137,120,197,202,23,160,
+75,201,27,31,45,183,41,9,76,159,235,57,237,128,52,122,241,222,232,63,
+152,60,185,23,134,45,12,10,144,157,235,181,97,242,249,234,35,237,35,111,
+102,44,61,28,2,194,192,209,253,239,139,149,236,194,193,154,84,226,118,213,
+190,61,24,172,239,71,191,90,69,164,55,115,196,127,160,116,138,34,53,88,
+127,217,14,187,112,14,247,68,167,236,7,143,216,246,193,190,169,191,249,242,
+170,40,199,52,36,121,132,5,111,170,71,38,234,210,164,180,106,131,157,235,
+135,101,71,54,74,177,12,176,90,244,49,71,234,7,173,234,121,117,24,120,
+192,104,16,9,255,117,216,51,230,219,245,49,113,2,236,60,228,42,117,147,
+95,52,171,205,163,61,9,247,106,65,163,38,46,180,237,84,86,53,174,234,
+50,156,96,35,214,175,158,104,90,191,232,24,42,224,166,12,245,111,215,209,
+210,219,213,190,144,251,127,171,220,34,47,75,98,151,203,109,154,251,166,62,
+196,192,221,122,192,24,69,112,6,60,96,212,62,8,196,49,95,38,31,138,
+79,6,22,154,205,131,155,149,149,79,202,223,30,96,238,152,60,190,92,33,
+128,146,215,95,114,177,108,178,58,133,107,0,196,195,152,152,142,159,131,176,
+21,252,231,249,201,37,184,182,74,190,228,38,14,36,186,17,228,27,252,246,
+100,82,220,128,34,18,136,206,4,101,253,176,91,18,28,220,8,250,1,205,
+172,178,200,244,226,221,187,184,63,232,49,140,65,194,109,87,165,129,63,171,
+82,79,82,16,25,15,115,201,132,189,106,89,185,207,66,76,50,118,89,133,
+226,229,148,205,220,163,208,244,91,210,31,73,224,174,105,177,250,84,120,93,
+201,113,34,31,217,34,21,251,182,8,95,129,95,181,94,0,100,145,189,230,
+170,154,141,156,216,141,204,42,26,119,41,53,199,241,111,89,10,33,60,152,
+44,195,245,177,252,58,73,34,171,176,77,8,200,61,174,60,169,164,145,66,
+138,83,24,22,81,58,5,119,94,133,244,213,213,161,10,104,53,225,167,56,
+166,3,123,47,66,50,93,193,136,94,35,75,206,253,197,124,161,66,100,147,
+123,127,46,98,245,59,32,43,56,171,118,79,240,72,42,95,118,152,19,231,
+234,237,17,60,94,121,128,210,210,100,214,137,25,50,200,151,86,160,221,90,
+103,23,73,227,70,108,96,79,24,33,188,70,12,113,99,214,40,68,77,138,
+101,86,184,171,136,129,41,116,198,65,236,27,218,209,66,36,12,135,133,239,
+177,67,173,116,107,38,20,224,12,177,170,11,189,176,28,218,39,50,27,94,
+148,243,246,100,218,54,132,91,198,112,215,96,27,197,67,199,76,177,184,134,
+95,18,50,161,163,9,7,1,238,86,243,75,23,246,219,8,103,165,180,191,
+160,223,109,201,6,142,215,218,53,110,116,69,105,180,100,152,194,155,193,184,
+229,15,113,192,39,19,100,107,179,161,148,48,231,24,114,74,119,209,4,38,
+114,215,181,62,231,167,45,95,44,127,15,32,170,152,225,230,162,202,16,38,
+165,199,193,164,142,49,108,86,68,131,131,150,145,249,106,214,137,92,226,178,
+211,113,61,216,240,166,104,208,233,142,211,66,88,141,22,144,170,222,199,158,
+153,30,7,60,230,247,159,125,117,11,204,227,17,48,83,106,250,24,134,117,
+96,17,192,218,153,213,79,44,246,213,242,209,117,188,224,123,8,161,185,12,
+194,241,131,199,33,93,231,66,36,244,50,168,241,158,11,215,108,82,226,88,
+80,190,182,109,46,122,93,37,82,19,190,189,254,90,172,51,163,251,87,224,
+91,179,47,148,137,140,235,74,135,183,41,186,58,176,176,251,233,47,27,4,
+99,113,93,212,169,123,208,193,28,118,105,197,159,229,93,196,154,179,36,93,
+213,188,91,219,90,70,8,40,35,119,205,58,158,166,172,69,184,27,117,44,
+115,55,142,64,136,28,63,229,152,254,93,0,244,240,232,89,250,61,200,68,
+43,1,45,198,171,59,196,61,19,17,164,204,118,211,88,30,190,42,243,165,
+124,5,136,201,184,91,53,140,136,167,122,246,165,61,11,5,244,58,181,53,
+22,138,124,158,83,191,218,235,128,178,79,42,68,186,235,11,166,240,210,168,
+23,167,234,248,8,81,196,7,41,37,134,104,20,40,146,186,31,154,241,131,
+30,4,235,121,113,37,178,2,121,209,167,46,221,196,45,37,33,91,137,182,
+189,245,41,218,207,233,36,97,82,5,197,64,127,76,52,57,135,50,247,55,
+161,96,170,49,179,205,86,36,142,13,61,147,102,55,163,207,2,230,123,139,
+73,13,247,80,156,19,243,194,144,140,185,137,191,35,29,59,152,67,116,132,
+68,146,220,248,160,40,197,139,215,213,236,118,195,33,73,94,3,11,200,105,
+154,138,57,37,43,118,116,159,46,94,188,55,10,194,174,63,43,240,68,96,
+113,111,90,196,101,158,183,233,85,44,137,66,52,44,153,145,110,11,80,135,
+60,155,8,224,251,170,179,26,137,225,12,167,100,144,51,150,54,227,77,127,
+200,39,147,89,245,37,207,106,200,65,50,108,42,223,2,171,8,103,14,216,
+129,209,3,4,56,56,61,142,35,253,52,42,165,34,106,158,245,253,62,190,
+171,68,223,116,136,37,166,237,116,66,99,235,159,122,186,99,233,82,177,171,
+124,222,190,95,203,197,67,34,82,56,136,18,62,255,141,240,135,193,244,31,
+86,50,100,78,177,241,176,135,106,83,124,209,117,39,112,238,42,156,84,127,
+173,147,94,4,219,222,84,43,126,46,6,84,26,155,5,209,75,42,38,149,
+29,158,163,43,166,126,74,92,73,193,250,95,170,85,125,10,145,101,124,249,
+102,112,206,197,0,103,21,104,68,139,163,7,92,195,30,194,140,121,217,133,
+194,223,31,38,116,167,7,99,162,140,161,162,95,4,65,171,156,108,46,230,
+55,147,182,156,244,137,173,70,6,21,196,61,192,127,108,164,54,2,232,100,
+6,247,209,72,41,97,177,254,84,37,66,80,3,141,138,74,165,201,116,146,
+216,189,228,34,135,165,38,180,163,171,0,112,231,243,11,131,172,7,147,151,
+230,179,50,44,76,84,84,23,163,21,108,87,43,233,41,225,236,191,213,51,
+111,66,63,127,35,81,204,165,112,35,251,135,223,58,55,189,247,41,194,179,
+24,55,230,64,11,61,193,151,217,230,5,164,211,90,229,239,205,253,147,31,
+37,187,133,95,36,131,4,31,178,152,225,14,159,119,190,29,232,72,79,45,
+203,105,196,40,13,43,152,105,172,124,143,98,29,113,128,166,109,242,243,149,
+106,105,45,182,203,187,244,181,22,206,53,241,84,174,175,47,221,71,9,243,
+187,38,202,15,205,211,124,158,197,209,162,55,210,167,181,184,158,109,194,156,
+77,152,42,7,153,94,180,91,10,113,74,39,167,61,151,123,80,62,172,23,
+12,228,195,68,227,182,147,240,18,55,114,167,41,230,160,192,212,231,209,160,
+219,109,30,48,84,206,26,46,228,72,167,244,117,76,29,42,86,117,28,10,
+148,140,132,176,171,97,199,53,211,72,128,171,159,249,216,161,89,99,28,182,
+106,201,130,200,8,197,236,35,44,114,130,92,131,7,235,245,182,168,28,38,
+119,48,145,200,37,192,159,149,127,135,141,255,234,99,84,213,33,37,44,236,
+39,49,218,171,182,144,100,176,147,101,220,102,55,108,166,131,68,106,30,103,
+69,24,110,98,13,171,136,147,239,55,112,177,78,127,178,56,243,90,68,121,
+109,247,198,235,154,166,151,27,85,186,255,74,95,104,248,201,161,208,172,212,
+246,226,91,209,109,215,178,21,121,173,45,179,214,170,5,197,192,117,97,189,
+180,184,189,139,51,50,63,167,25,230,141,134,141,156,225,118,146,214,241,151,
+119,126,142,113,64,130,45,67,218,185,119,47,66,240,154,154,91,90,167,221,
+11,173,207,21,97,50,205,73,145,192,245,218,146,221,226,73,133,142,162,208,
+174,75,149,56,230,186,214,74,37,130,231,122,176,247,92,113,240,223,79,210,
+129,20,145,189,99,46,11,17,96,215,221,245,20,251,98,150,7,158,110,125,
+22,106,119,215,103,153,83,186,15,181,136,16,61,49,13,113,247,50,63,198,
+242,211,87,108,123,221,143,201,20,183,254,59,98,36,153,175,50,231,77,190,
+101,76,67,13,229,9,85,176,77,230,103,212,62,115,25,26,100,236,52,86,
+252,55,147,193,118,23,102,224,55,200,43,116,238,209,102,214,90,234,17,249,
+219,38,234,97,172,180,53,78,230,243,157,88,182,243,124,56,111,208,137,183,
+64,86,189,213,120,89,207,110,197,75,236,172,10,49,44,158,138,208,173,156,
+41,209,209,119,187,250,122,4,160,165,148,47,60,14,241,154,41,133,139,182,
+16,185,98,213,181,233,95,219,134,240,129,70,235,0,116,80,135,71,203,72,
+16,41,16,6,188,144,103,141,247,123,184,53,107,195,232,96,174,248,91,202,
+53,120,240,48,254,125,217,107,27,123,244,249,249,106,191,83,118,242,142,101,
+46,48,118,1,201,6,64,20,58,179,141,93,144,221,37,13,200,239,106,184,
+126,169,90,51,75,12,181,50,198,156,167,100,238,245,173,167,96,62,61,23,
+144,214,77,175,114,17,223,119,193,40,174,65,55,54,241,184,38,33,41,9,
+40,190,84,41,171,105,165,43,45,105,107,248,93,18,34,199,27,120,192,134,
+248,244,144,151,213,192,155,133,100,227,56,67,191,74,154,142,171,166,151,162,
+110,254,62,63,66,99,181,64,176,113,99,111,143,56,207,237,103,33,16,7,
+13,53,250,160,209,140,106,191,139,205,206,236,38,50,203,40,85,61,145,166,
+184,114,92,178,58,94,142,125,4,117,75,48,224,242,140,193,73,69,37,11,
+187,218,247,92,156,73,43,247,156,134,225,201,160,172,187,122,197,21,136,28,
+69,58,236,219,209,22,20,63,52,92,170,113,253,204,170,220,220,255,17,80,
+149,97,205,127,56,70,202,176,139,98,94,78,198,125,3,16,133,170,192,130,
+68,33,226,79,234,27,97,66,52,250,48,131,177,199,46,244,52,36,6,149,
+255,249,242,77,169,22,47,149,216,137,88,115,80,28,221,15,109,126,84,26,
+184,76,106,53,212,243,178,48,173,65,158,150,39,242,219,173,18,243,14,251,
+87,245,226,67,210,109,237,65,70,1,0,25,192,14,254,217,215,156,173,142,
+157,152,241,126,27,192,79,56,172,116,103,148,198,244,100,243,103,191,91,14,
+136,109,61,203,101,255,37,151,6,0,41,137,102,154,210,249,139,134,34,168,
+34,221,130,45,156,137,215,129,83,196,153,129,27,255,56,221,7,58,209,175,
+163,187,252,203,76,56,145,248,55,180,174,173,167,202,163,63,130,177,193,121,
+251,73,20,207,59,107,181,126,201,250,113,80,97,121,245,107,246,123,13,108,
+116,242,194,115,111,196,154,44,216,56,32,49,104,216,6,73,247,54,90,0,
+19,115,15,169,233,104,87,235,241,175,84,88,174,99,239,129,51,215,208,157,
+48,93,133,15,203,226,242,117,179,93,6,195,16,174,13,147,88,138,69,13,
+45,24,139,82,224,49,204,106,177,65,89,122,209,63,154,58,243,135,156,94,
+166,71,97,247,46,133,93,157,191,200,215,129,126,58,36,16,29,109,19,46,
+42,49,220,7,53,40,108,69,216,226,174,233,93,16,36,121,160,164,6,62,
+162,228,233,35,28,183,171,204,11,58,168,151,227,141,108,178,139,150,47,226,
+20,235,105,220,54,29,213,70,223,159,61,20,122,46,237,216,35,177,160,241,
+99,80,229,45,164,215,154,167,54,38,99,33,101,229,251,225,230,3,240,179,
+37,22,120,154,51,17,71,34,20,199,37,154,115,201,204,114,60,18,87,233,
+75,64,124,124,45,100,104,205,244,192,179,120,240,230,48,100,174,108,6,23,
+83,155,135,66,156,62,243,84,250,193,115,189,23,158,135,197,242,215,30,36,
+59,62,17,48,133,118,78,7,64,192,249,203,166,236,219,49,211,55,64,174,
+54,248,219,102,152,155,103,24,222,175,162,225,108,41,116,169,107,2,93,29,
+50,37,112,39,188,62,59,242,36,234,221,151,198,111,206,229,255,89,154,21,
+63,10,231,78,137,26,96,177,236,5,15,93,63,25,47,189,164,18,87,122,
+42,70,242,43,137,185,175,20,52,178,177,1,8,245,63,186,107,10,28,153,
+151,174,181,168,25,25,103,150,82,175,110,129,153,86,251,6,224,120,69,215,
+81,247,230,223,79,62,11,215,62,25,230,210,193,96,163,18,227,10,177,184,
+29,122,228,212,181,238,232,123,47,176,108,58,38,137,116,65,205,212,114,142,
+0,165,105,109,255,75,170,68,184,120,94,214,68,223,245,108,216,168,77,247,
+97,29,157,82,180,54,159,78,97,184,234,20,5,176,113,22,173,15,41,108,
+35,155,166,209,100,202,87,108,240,52,65,127,98,166,203,95,178,112,153,129,
+141,122,76,42,117,151,17,146,106,111,87,200,165,51,122,163,3,175,185,44,
+254,246,205,150,162,213,125,61,12,76,170,166,247,78,178,132,205,25,190,64,
+154,23,197,60,51,55,8,198,9,33,113,248,152,245,3,13,152,102,32,110,
+94,84,66,248,234,171,115,136,90,173,16,100,227,113,60,214,181,72,61,122,
+66,253,207,23,103,24,11,105,61,226,242,32,125,46,232,152,211,122,229,153,
+193,38,47,79,176,137,123,4,136,3,200,30,221,33,231,102,170,182,206,181,
+141,75,26,106,238,10,39,64,196,216,191,135,122,175,17,149,177,0,117,103,
+253,21,108,86,240,182,183,190,178,117,28,105,10,220,172,132,109,232,232,170,
+244,102,75,199,80,12,33,148,89,19,11,115,216,43,174,234,241,6,21,27,
+100,119,80,3,57,208,200,94,104,166,230,66,60,43,184,125,227,51,231,115,
+95,173,118,216,209,168,34,47,62,231,184,140,186,216,232,205,34,68,8,224,
+130,231,23,124,155,0,64,156,38,53,204,198,61,142,204,152,22,51,233,40,
+150,176,76,181,66,128,154,208,212,31,228,82,16,196,76,154,180,151,246,166,
+225,109,25,135,180,125,82,224,200,13,31,229,158,199,104,193,157,60,193,173,
+223,19,216,25,159,76,91,236,75,7,105,24,47,17,189,19,70,144,150,78,
+121,67,82,56,94,181,204,146,1,51,69,152,216,165,89,97,108,220,138,13,
+204,236,110,182,1,254,135,38,187,123,122,27,180,160,63,152,193,207,81,19,
+184,157,79,254,216,76,244,207,234,59,39,105,196,28,207,150,154,229,223,122,
+157,251,164,157,117,126,101,224,24,114,90,119,159,218,104,213,253,157,28,78,
+64,147,101,15,101,227,153,4,18,194,31,1,187,69,11,19,173,92,53,153,
+152,197,191,140,168,201,217,173,232,136,140,223,200,44,5,230,7,213,90,241,
+211,156,32,8,170,142,132,48,192,76,238,200,215,25,128,17,88,123,106,212,
+62,164,58,48,146,7,86,98,161,46,132,31,31,165,164,187,63,158,153,233,
+6,44,106,108,83,205,175,115,73,47,216,210,39,138,62,231,223,57,143,69,
+84,198,181,189,135,150,158,179,22,116,2,22,77,174,103,117,235,197,110,74,
+169,162,156,144,210,68,52,10,166,35,159,71,225,139,193,188,117,111,143,21,
+15,252,131,22,85,56,27,147,100,121,228,77,186,174,224,46,239,202,240,187,
+18,113,223,55,45,153,143,196,164,119,116,246,203,210,31,95,83,138,16,25,
+229,133,132,146,80,55,180,123,51,140,153,105,1,237,86,222,196,210,67,78,
+162,150,255,64,51,26,200,216,111,58,167,15,123,218,29,82,86,105,229,129,
+216,13,113,9,135,35,115,211,84,24,71,20,85,250,19,97,242,93,9,97,
+69,255,179,216,150,111,118,155,73,73,17,76,26,151,234,102,105,212,99,151,
+45,5,22,187,115,13,193,90,246,135,92,74,104,202,16,218,138,143,117,215,
+187,145,199,106,187,237,214,151,163,53,130,109,200,64,247,79,8,190,53,192,
+243,228,18,164,218,178,37,79,15,175,30,49,209,225,219,46,127,34,188,158,
+227,5,110,227,61,95,251,252,244,19,102,47,223,56,14,61,150,175,180,185,
+194,213,222,147,198,59,62,210,177,109,170,238,11,111,202,118,203,81,232,27,
+106,23,93,157,119,227,182,200,7,34,114,105,162,150,238,33,102,211,244,250,
+157,27,32,253,178,184,196,127,84,103,242,163,115,7,117,186,185,20,226,73,
+142,48,52,57,147,152,202,186,148,44,17,29,20,137,206,78,164,202,206,97,
+222,220,100,200,105,224,239,39,104,17,154,235,255,240,194,239,222,174,4,104,
+196,137,186,65,235,47,82,116,94,127,55,7,19,254,155,128,124,50,170,12,
+140,135,250,187,36,99,147,169,175,129,239,76,129,170,167,162,123,195,47,60,
+147,198,138,215,193,213,242,155,48,72,226,211,239,242,146,81,57,59,247,79,
+150,198,230,3,233,217,161,90,35,93,168,247,169,151,227,171,92,110,134,246,
+221,156,77,134,160,17,1,64,110,124,139,105,7,9,54,42,44,77,235,36,
+194,184,54,16,60,180,153,191,119,162,240,93,46,163,246,58,116,104,181,100,
+105,227,239,126,218,2,224,57,16,210,162,198,61,129,199,83,155,58,245,62,
+79,125,92,183,163,219,156,146,128,215,19,129,193,32,17,230,203,132,200,182,
+112,205,126,70,255,172,254,77,70,8,210,224,64,43,150,2,15,59,145,13,
+160,62,98,75,132,227,143,5,251,238,2,35,184,137,121,136,116,234,27,189,
+118,236,92,235,77,122,248,231,26,225,72,81,244,49,55,28,178,179,14,9,
+109,57,71,36,107,213,72,60,141,28,80,177,149,170,84,31,156,234,89,153,
+192,30,32,32,93,190,90,162,191,156,112,142,138,140,207,194,157,241,186,78,
+63,231,149,45,193,204,120,72,248,78,91,240,208,204,7,219,5,62,75,147,
+21,168,82,11,158,105,32,176,122,208,82,55,3,76,95,49,78,70,16,157,
+78,253,185,222,109,242,22,226,212,60,216,34,143,39,74,137,88,99,125,18,
+189,163,189,140,88,86,170,234,65,230,177,169,48,143,135,119,145,187,58,14,
+251,108,133,40,0,208,177,222,78,102,12,104,57,187,251,67,61,216,78,112,
+213,153,184,65,85,50,105,18,133,191,126,203,109,171,167,61,170,163,15,185,
+119,145,233,178,84,179,86,223,183,54,151,204,250,126,13,208,23,137,64,76,
+107,151,167,16,97,20,110,203,199,172,93,165,175,158,173,69,135,243,126,155,
+81,70,118,80,209,198,49,111,243,26,224,242,180,196,25,142,210,39,73,244,
+4,136,191,193,54,219,40,54,223,124,106,90,151,55,236,31,49,104,178,98,
+13,85,173,59,170,231,70,243,255,253,97,128,1,191,110,235,49,176,84,178,
+127,8,207,164,154,147,108,168,37,162,111,231,169,26,82,255,231,69,27,23,
+235,139,19,121,210,242,231,10,40,187,189,49,28,118,81,156,111,72,87,58,
+108,78,98,211,179,229,223,22,220,160,37,37,222,247,151,235,129,214,162,151,
+134,118,232,195,127,138,196,33,176,197,214,78,248,3,201,40,56,119,6,97,
+76,92,115,174,226,253,91,95,137,249,222,139,29,114,114,138,7,254,16,196,
+249,40,181,27,112,9,231,147,235,23,64,104,206,244,90,162,70,100,170,232,
+92,98,79,200,159,27,242,195,36,219,160,69,108,122,250,246,173,23,191,197,
+161,104,21,29,78,198,174,192,31,128,161,100,223,248,174,86,38,23,215,170,
+221,44,104,167,46,249,101,180,64,236,115,193,213,194,218,153,140,34,18,12,
+194,201,164,87,233,192,135,17,172,39,205,209,75,253,228,177,231,119,20,151,
+235,171,56,125,37,41,137,111,155,183,106,32,76,129,188,27,199,228,182,246,
+148,167,99,149,28,164,176,229,70,201,229,169,36,21,251,170,210,199,148,141,
+100,70,168,22,26,99,149,136,187,143,119,209,31,145,251,176,17,101,124,150,
+123,60,115,81,149,130,49,24,68,5,227,112,33,169,92,204,81,108,18,3,
+120,152,254,247,248,22,62,3,128,94,147,180,187,41,169,178,158,189,118,215,
+132,34,66,232,226,41,222,250,87,175,162,238,168,251,240,143,169,204,157,20,
+210,199,59,114,192,142,85,25,210,160,221,81,193,107,4,103,11,41,238,134,
+88,56,205,206,102,38,63,44,35,95,164,0,6,128,145,131,70,187,61,226,
+117,138,38,113,204,248,160,144,212,19,30,215,194,1,165,3,197,16,47,158,
+58,221,14,249,45,236,179,48,143,13,141,140,5,159,64,79,155,253,123,98,
+200,170,108,204,201,248,22,240,71,40,159,130,39,144,250,183,236,37,97,211,
+224,190,174,126,174,219,192,58,201,170,56,35,70,152,213,45,228,17,68,133,
+102,10,204,107,4,100,164,220,253,121,88,162,2,192,228,186,212,248,56,107,
+20,158,54,9,3,87,205,244,78,109,89,162,14,15,35,135,83,224,124,92,
+174,54,30,222,194,165,45,56,141,28,152,115,222,226,28,92,134,104,227,237,
+134,139,152,116,33,163,235,69,120,176,146,232,96,202,250,139,113,165,75,212,
+36,93,255,91,174,130,148,175,171,219,4,142,207,145,129,7,242,114,209,222,
+58,188,149,101,9,50,79,237,100,161,254,219,152,19,180,79,172,165,148,183,
+33,35,25,178,177,158,81,129,202,100,153,117,70,208,244,90,179,183,152,202,
+117,69,195,246,26,254,25,247,40,67,226,32,236,59,243,59,118,46,71,28,
+103,99,139,85,172,214,215,139,251,0,190,122,128,51,169,219,147,50,223,232,
+19,98,165,202,227,35,209,36,173,143,94,201,120,248,90,242,96,159,248,194,
+146,109,127,72,231,63,18,149,27,225,195,24,132,233,53,184,51,160,227,170,
+171,103,120,141,34,183,152,82,186,56,156,56,191,197,36,121,15,190,108,65,
+131,230,167,205,52,1,130,20,218,83,152,180,5,71,28,22,126,118,246,29,
+247,215,46,224,220,56,194,192,225,220,58,47,204,46,200,115,2,2,176,170,
+45,38,160,97,24,78,73,183,24,92,232,200,181,81,68,89,213,171,174,97,
+74,19,249,234,174,231,85,147,113,246,221,73,82,249,235,6,129,225,6,0,
+114,182,108,62,123,34,255,53,172,87,5,142,187,156,181,85,18,131,209,78,
+37,171,32,164,199,176,25,128,165,110,167,87,127,167,4,248,47,7,32,199,
+247,183,76,227,169,5,107,35,152,137,211,15,106,79,36,91,217,135,11,214,
+42,64,239,161,40,159,39,171,158,37,64,164,96,201,156,1,14,200,111,48,
+18,61,45,116,173,165,16,23,46,184,163,183,124,98,145,188,202,165,187,146,
+247,196,247,180,36,152,222,80,230,247,140,245,7,11,139,32,0,196,146,182,
+60,112,222,38,73,152,180,196,182,118,92,226,80,150,243,149,149,30,255,255,
+179,255,141,2,49,207,2,217,154,100,46,225,207,25,196,9,5,95,114,119,
+176,106,123,32,32,141,58,211,152,115,47,43,250,108,91,207,255,229,55,35,
+85,198,56,135,135,240,237,3,103,3,130,172,113,50,162,251,195,71,38,151,
+40,73,140,223,196,204,242,99,20,15,255,192,47,29,31,37,87,232,150,26,
+234,197,129,222,37,97,98,174,250,7,208,27,209,186,198,99,30,152,205,79,
+198,31,172,133,201,146,218,241,7,228,114,42,77,15,144,240,238,198,187,44,
+104,59,244,98,53,108,168,198,153,48,137,41,160,133,115,99,235,214,179,122,
+179,59,198,35,20,181,38,229,95,155,3,37,22,148,142,118,139,247,140,53,
+202,236,225,87,224,152,137,17,254,63,184,144,75,147,119,86,4,78,106,6,
+1,120,150,85,239,176,176,186,27,172,42,6,203,52,83,53,250,190,233,86,
+144,123,130,214,17,1,22,213,171,150,95,20,209,34,223,215,69,86,85,167,
+195,156,68,176,184,174,238,183,19,93,239,216,212,206,56,175,29,132,13,27,
+214,229,184,105,192,40,124,200,74,54,50,151,117,254,31,45,52,123,60,167,
+134,154,42,72,31,231,226,201,220,202,131,175,24,177,45,172,50,72,142,35,
+152,73,202,54,183,214,214,83,225,129,54,18,99,243,26,233,108,58,121,6,
+141,73,151,115,74,33,142,137,128,171,121,195,170,183,66,133,33,250,225,210,
+235,205,176,39,235,154,185,193,217,247,79,252,195,160,250,167,5,183,4,55,
+118,243,163,47,96,79,188,70,226,90,116,88,126,250,192,203,21,174,253,39,
+40,136,191,251,230,83,38,123,42,255,55,228,213,17,249,79,127,143,51,62,
+60,167,129,84,75,175,246,68,13,54,196,8,63,158,44,255,178,207,154,30,
+91,87,62,24,210,243,64,41,47,63,41,131,4,56,138,128,8,45,119,36,
+175,250,197,1,168,215,209,161,181,192,236,246,127,194,149,250,149,252,143,125,
+123,150,149,205,35,79,134,2,237,233,77,28,16,37,133,84,18,237,236,220,
+213,185,149,119,66,205,145,222,187,79,92,139,212,246,231,160,23,166,138,145,
+55,62,132,207,246,177,92,203,206,114,49,254,201,42,195,160,146,130,241,3,
+213,235,1,45,143,51,21,213,114,76,0,124,168,32,210,199,91,180,27,229,
+82,41,163,4,155,166,157,226,35,112,143,99,231,148,191,73,98,106,246,26,
+90,214,137,208,5,243,249,164,238,103,214,254,186,221,57,83,44,122,194,94,
+236,186,93,139,234,142,118,224,235,107,122,191,8,219,239,51,151,94,33,74,
+145,19,215,190,66,218,240,114,176,146,233,29,168,67,32,1,74,40,183,243,
+41,180,107,31,73,195,174,31,171,64,185,69,220,187,216,182,205,230,35,112,
+46,180,241,212,86,219,150,95,104,20,228,69,1,243,204,243,0,232,255,93,
+18,167,198,240,153,165,245,14,97,217,91,57,1,0,205,65,186,135,9,102,
+25,150,118,35,186,36,14,221,162,0,120,153,161,220,158,202,70,168,243,13,
+169,182,236,71,89,203,7,254,217,215,104,126,37,122,173,112,129,58,156,89,
+184,225,148,191,178,133,104,102,150,80,216,126,114,98,7,197,187,197,227,225,
+129,72,116,210,201,225,48,207,230,82,141,162,97,139,212,198,162,204,243,215,
+240,210,238,70,46,224,32,80,125,111,65,184,191,184,152,228,53,221,116,247,
+93,130,107,191,205,135,47,34,81,158,187,237,238,205,169,250,116,35,209,133,
+75,224,209,16,102,238,120,197,15,214,228,80,191,43,96,128,195,153,16,205,
+165,246,33,174,224,220,10,227,125,112,139,104,165,221,74,115,143,42,82,107,
+239,201,33,107,198,232,65,244,28,187,35,100,7,61,57,126,71,250,121,176,
+169,76,210,214,94,65,151,19,19,206,174,100,225,84,221,57,148,2,71,94,
+69,107,150,48,228,176,172,16,90,178,54,11,248,254,200,76,31,176,61,108,
+19,14,25,28,230,139,27,220,45,122,94,165,193,77,163,100,45,230,153,187,
+132,37,205,76,146,160,68,73,226,159,119,209,113,184,191,38,115,167,229,70,
+248,160,113,181,165,24,131,9,215,81,109,28,227,218,56,241,64,189,190,208,
+119,174,178,36,84,8,63,164,240,125,183,98,66,65,16,81,27,211,28,72,
+92,230,19,1,203,98,159,185,240,131,203,178,2,72,118,77,168,138,255,137,
+57,206,49,10,186,86,73,96,223,99,15,199,57,192,19,102,65,201,204,13,
+52,135,245,104,123,157,231,1,155,18,221,208,33,90,219,223,246,86,178,223,
+197,111,238,209,110,226,93,249,129,216,219,228,196,13,33,160,44,235,101,182,
+34,190,7,224,235,208,250,31,237,9,223,48,49,136,204,222,1,3,235,249,
+121,43,180,96,247,242,216,191,55,15,249,64,114,10,81,60,207,137,211,106,
+124,242,203,179,157,169,108,208,123,103,117,124,75,16,72,83,78,23,39,13,
+14,112,140,218,135,248,86,119,123,14,207,2,225,70,253,62,23,10,181,4,
+105,209,225,21,207,58,188,23,234,96,218,88,249,50,48,173,160,65,63,150,
+143,117,93,33,67,121,84,66,192,55,61,216,220,122,41,172,14,71,227,132,
+17,76,92,131,147,194,218,11,192,155,244,171,32,141,150,212,40,80,222,30,
+78,135,17,201,234,34,215,182,200,11,183,65,173,11,120,130,77,165,100,197,
+47,14,24,47,54,213,229,78,252,142,27,132,138,210,5,204,42,42,105,136,
+239,188,185,111,204,57,181,255,234,163,184,193,248,85,24,168,77,74,75,177,
+145,46,134,113,211,134,49,185,60,191,42,82,42,222,104,114,33,24,40,24,
+1,64,114,61,105,90,33,176,216,255,250,31,154,37,85,209,154,3,111,237,
+63,27,86,116,83,5,44,150,129,96,217,64,127,122,100,133,149,177,213,221,
+185,138,101,96,46,193,192,127,228,72,248,95,89,206,145,84,249,108,105,124,
+104,165,39,213,34,234,194,8,213,52,4,41,203,59,243,86,75,141,73,248,
+203,245,122,116,51,28,27,101,223,86,170,125,114,66,196,94,255,33,231,245,
+67,165,203,130,89,42,252,121,81,205,195,208,253,150,61,195,122,104,47,63,
+123,62,249,165,226,220,45,57,252,103,236,90,61,96,222,219,163,14,199,182,
+131,132,41,160,59,69,48,234,95,179,134,179,120,219,227,138,30,116,202,204,
+87,187,155,240,232,35,216,150,96,236,152,84,138,14,117,237,219,147,2,59,
+24,38,241,123,52,244,202,253,48,2,3,64,29,43,98,75,171,227,215,112,
+55,141,31,25,241,71,6,73,71,207,152,195,93,175,105,201,67,0,106,136,
+71,172,2,39,179,124,75,43,176,172,207,62,68,203,192,80,170,70,84,193,
+124,139,122,191,235,175,47,66,234,154,192,180,52,131,251,108,81,34,59,9,
+102,154,60,122,154,171,152,100,135,162,237,121,140,130,20,250,5,183,185,35,
+165,197,199,92,238,211,18,108,131,129,221,204,1,219,56,124,187,72,48,87,
+194,77,9,74,28,195,231,10,238,78,147,36,161,250,156,161,14,8,87,235,
+166,137,11,204,248,195,25,114,34,141,55,216,117,253,41,238,231,46,0,158,
+112,217,119,98,66,15,13,181,83,22,97,132,113,154,154,209,135,246,11,139,
+185,189,203,158,35,34,150,134,57,73,76,44,88,124,60,33,159,230,168,83,
+32,131,131,196,105,186,210,238,43,148,95,154,99,212,20,97,46,112,166,210,
+116,85,176,160,247,29,91,95,1,244,205,168,249,240,168,55,107,29,5,72,
+40,131,118,110,115,170,67,15,94,248,44,215,143,197,238,103,163,113,48,77,
+104,141,76,89,10,65,68,109,38,251,252,225,6,195,134,104,51,243,58,239,
+96,208,118,100,146,248,191,82,52,20,63,132,6,14,150,156,160,67,244,241,
+158,181,68,179,16,209,162,117,95,6,109,213,68,75,170,62,27,73,179,130,
+135,184,9,116,193,200,156,159,173,248,188,42,171,79,121,225,212,28,186,101,
+167,122,184,254,255,193,131,129,220,107,179,83,255,99,86,218,181,202,207,231,
+72,96,248,248,120,51,38,32,165,140,131,193,48,144,77,254,231,235,178,54,
+26,30,72,213,255,186,236,68,176,109,220,15,43,219,96,16,4,73,227,155,
+28,18,132,172,20,107,230,201,58,199,90,252,48,36,239,239,113,93,33,139,
+181,186,31,3,51,27,215,242,164,92,175,185,147,78,163,20,221,29,220,45,
+151,17,118,7,10,152,38,196,66,83,192,229,122,156,241,207,228,163,197,151,
+127,213,192,55,169,71,0,109,14,132,226,217,130,80,34,196,170,135,2,42,
+227,136,40,204,233,225,51,68,227,218,67,223,17,188,70,234,193,43,248,117,
+154,236,116,198,229,105,19,212,167,227,103,121,202,91,170,5,162,16,207,202,
+122,168,90,19,168,16,212,99,130,38,145,32,107,148,61,234,138,204,177,49,
+169,189,70,236,239,54,123,110,120,7,7,108,23,168,12,228,234,113,111,86,
+88,69,187,111,220,240,1,22,162,40,165,204,238,214,109,253,79,63,208,199,
+240,112,209,101,205,232,3,125,77,19,129,95,242,107,25,86,64,202,221,84,
+65,184,205,10,200,126,209,31,109,16,242,138,197,5,152,140,209,61,7,142,
+158,3,25,62,194,88,68,243,89,38,215,93,223,23,238,54,29,193,95,11,
+204,136,91,31,103,159,16,136,254,30,153,97,123,195,71,40,128,185,185,63,
+225,2,21,123,216,174,135,12,230,228,187,165,118,166,53,4,119,26,21,11,
+82,25,98,103,217,43,131,17,167,79,233,42,5,224,66,202,237,167,171,62,
+176,58,123,62,120,115,194,117,76,9,165,88,111,207,72,234,126,164,81,57,
+247,36,18,107,228,26,235,224,193,121,194,13,30,59,189,111,99,2,137,215,
+113,54,123,195,27,138,89,4,135,170,181,70,241,207,225,194,20,24,210,174,
+93,255,139,190,100,245,56,117,205,122,84,180,36,247,224,88,192,88,223,68,
+228,166,19,138,94,200,7,77,30,62,156,239,173,32,218,10,206,205,25,53,
+206,53,88,117,217,77,169,146,236,193,80,231,215,20,140,188,201,203,46,114,
+253,191,72,156,1,11,200,114,55,140,69,135,132,234,10,10,182,220,125,191,
+210,3,242,235,121,248,3,185,0,82,14,138,163,55,75,148,141,249,111,171,
+146,186,229,128,143,234,138,154,20,168,231,203,200,228,78,221,53,165,85,56,
+151,17,134,204,117,16,78,37,233,141,97,70,101,83,78,171,149,110,157,6,
+4,169,189,57,199,19,238,143,75,212,99,143,100,148,245,35,202,193,231,47,
+145,184,90,19,73,192,155,22,191,242,154,99,199,47,227,94,238,24,224,179,
+78,11,9,203,4,220,65,166,111,217,204,102,102,49,5,172,108,40,162,41,
+58,163,218,227,85,22,128,119,255,213,11,162,167,82,29,75,85,22,59,246,
+38,214,72,141,43,128,186,15,14,15,8,234,184,216,110,146,216,194,28,179,
+225,150,90,105,16,60,12,43,250,97,84,147,34,210,7,169,9,149,144,51,
+7,66,166,56,236,164,68,22,46,250,75,39,28,178,37,194,25,57,181,91,
+219,162,223,41,140,253,122,210,11,241,48,241,64,24,135,39,17,111,248,63,
+106,130,204,125,96,185,135,56,166,65,144,79,162,102,63,74,2,233,97,174,
+163,94,60,205,53,0,31,32,100,160,241,125,82,149,76,71,254,57,115,144,
+110,202,80,127,62,217,192,137,18,160,66,162,35,58,61,164,122,94,152,9,
+165,96,151,114,227,34,57,23,109,13,177,32,126,11,129,218,100,201,15,244,
+196,147,218,132,69,74,106,107,51,188,76,0,155,98,176,208,201,227,66,240,
+199,57,40,164,181,150,207,108,9,189,118,224,13,85,210,226,164,40,188,216,
+154,69,197,120,141,192,63,82,232,59,131,20,73,98,17,203,69,62,9,30,
+233,21,93,178,93,31,62,239,64,58,23,244,124,152,220,217,201,232,112,22,
+214,236,106,255,214,199,247,167,56,75,249,119,3,70,181,200,18,51,181,222,
+254,55,140,73,188,226,219,178,32,154,100,243,94,152,228,42,117,217,169,238,
+233,3,170,202,27,20,77,228,2,0,10,187,38,71,241,178,207,80,173,179,
+104,240,240,80,254,18,85,68,207,84,123,221,135,130,191,220,55,240,49,153,
+246,31,237,180,227,161,85,117,63,204,125,118,249,128,166,241,105,134,99,161,
+178,246,122,147,149,219,230,146,35,88,155,243,41,33,5,152,199,207,17,189,
+41,10,104,251,49,135,172,167,7,100,179,23,144,187,125,136,109,148,147,158,
+237,75,13,85,205,32,69,77,68,110,118,89,190,233,2,112,240,32,70,83,
+112,39,250,213,0,176,33,214,184,105,133,81,105,210,164,178,134,53,140,70,
+51,137,91,247,197,34,237,125,28,155,18,177,245,232,26,128,60,81,42,3,
+247,101,208,40,173,141,192,201,237,169,215,69,81,131,18,0,24,68,176,228,
+31,206,90,110,23,192,49,155,162,125,216,12,254,71,97,127,150,95,119,148,
+14,7,146,49,189,142,124,176,34,2,149,148,100,138,253,177,13,246,184,73,
+52,245,31,124,115,57,56,29,66,30,143,67,234,238,200,142,124,181,71,242,
+104,213,66,217,221,20,205,200,67,8,50,115,24,106,220,110,83,213,211,5,
+148,88,229,71,246,198,83,201,245,113,148,255,253,232,68,100,128,179,111,236,
+214,122,252,181,24,33,104,159,110,126,134,253,180,42,124,129,154,50,53,169,
+118,215,133,223,11,89,206,48,197,73,21,44,24,123,68,237,34,34,54,159,
+230,5,176,186,140,34,157,126,180,93,193,51,173,74,126,88,81,193,87,75,
+42,5,148,86,18,82,51,168,201,248,81,171,180,161,21,157,76,52,196,209,
+18,23,83,190,47,202,10,11,160,27,240,128,175,25,190,101,253,166,203,170,
+194,4,231,187,35,51,210,18,230,190,68,233,251,186,153,39,244,198,190,6,
+93,238,90,219,90,142,2,83,200,145,27,116,104,90,49,178,83,221,152,135,
+146,160,218,127,167,237,201,3,217,140,130,80,3,203,85,152,181,25,171,109,
+150,125,174,208,57,51,122,210,183,235,67,216,76,91,153,177,194,37,59,7,
+29,59,22,128,158,28,159,251,75,14,73,67,61,219,104,179,87,238,1,10,
+152,8,234,201,81,73,235,252,237,6,60,34,228,76,203,208,80,70,152,244,
+46,87,35,235,199,163,55,234,8,161,249,218,143,195,243,240,166,225,146,251,
+61,140,39,144,183,177,182,191,207,17,157,17,116,132,183,13,244,170,61,199,
+55,55,238,42,15,104,235,201,145,12,224,107,186,2,5,147,2,118,221,151,
+229,109,106,164,181,231,159,81,183,193,104,130,240,35,71,154,217,35,1,131,
+30,136,139,54,225,211,60,180,149,251,29,27,239,124,46,76,143,192,112,239,
+181,168,152,234,231,161,186,125,133,125,125,26,155,92,234,121,35,32,44,42,
+36,194,215,204,95,243,27,83,34,121,94,14,201,210,193,248,242,131,217,12,
+0,27,156,3,28,97,107,71,100,41,161,113,199,191,147,65,59,198,154,110,
+149,242,190,51,131,247,181,250,139,149,20,219,8,216,136,120,248,207,144,172,
+125,141,138,65,122,73,47,61,155,41,112,122,143,225,183,127,164,117,124,86,
+79,139,221,58,178,44,121,253,74,235,215,212,186,80,99,190,158,139,75,87,
+46,223,119,184,190,58,172,169,163,133,100,90,129,131,197,188,229,227,27,94,
+112,177,64,142,92,184,37,58,118,146,3,84,182,212,200,247,84,175,58,175,
+43,60,59,41,135,22,187,150,220,33,217,155,144,161,103,62,23,245,206,220,
+171,226,42,93,135,18,95,199,133,57,223,240,155,92,79,149,108,247,201,97,
+168,186,156,216,14,144,99,98,150,157,189,72,201,161,150,206,12,250,164,129,
+10,185,137,7,132,251,44,168,93,36,254,70,172,216,36,71,23,139,217,7,
+243,248,116,173,192,78,51,86,66,212,117,143,161,159,187,246,47,241,58,168,
+44,167,76,102,31,243,14,143,26,207,248,73,63,25,143,194,99,10,233,190,
+59,176,115,186,138,173,254,126,21,28,164,168,208,229,163,252,142,20,29,224,
+74,203,80,193,0,198,231,211,236,111,144,219,157,145,202,199,11,190,198,60,
+100,115,54,19,227,130,133,144,87,113,109,37,29,54,179,1,173,54,74,95,
+118,253,121,173,33,239,186,221,238,163,16,240,129,233,33,39,35,206,203,54,
+46,78,158,102,208,49,148,181,52,157,43,195,130,12,74,16,20,129,133,103,
+254,21,80,193,128,30,193,26,241,196,124,211,156,54,24,106,224,128,154,162,
+117,128,60,37,115,90,43,235,217,95,146,68,141,171,126,152,174,226,213,105,
+213,162,204,20,123,131,248,24,146,202,81,231,197,230,218,6,18,33,154,37,
+100,114,237,16,241,232,93,163,8,137,227,220,248,1,178,200,123,49,86,250,
+108,239,48,14,108,80,175,244,10,191,73,100,192,44,114,62,2,92,62,248,
+193,9,145,1,168,139,173,23,161,7,219,88,186,109,102,38,39,130,254,93,
+109,240,230,224,139,158,64,135,158,200,221,218,192,244,60,31,216,56,167,33,
+195,13,235,74,10,214,28,239,166,31,122,231,45,151,184,133,117,209,85,251,
+38,101,46,137,31,52,45,46,157,34,3,116,13,83,212,226,176,79,41,70,
+15,22,154,124,164,19,97,212,197,119,97,140,254,38,25,158,173,158,4,251,
+178,226,79,210,126,4,193,65,67,45,86,119,121,214,200,249,13,45,66,5,
+211,234,50,132,16,31,141,8,109,90,104,107,245,131,48,213,107,211,101,37,
+30,145,187,189,114,96,236,136,8,138,69,59,143,121,87,195,74,230,109,77,
+21,15,255,203,231,2,100,245,252,171,166,205,224,33,216,68,122,28,22,177,
+26,185,11,158,249,6,196,254,27,26,51,75,144,232,51,43,48,94,12,179,
+170,187,171,196,236,206,91,59,134,134,99,169,42,201,195,170,166,248,163,116,
+250,232,74,254,125,105,103,7,25,55,151,171,200,199,32,41,234,164,214,21,
+126,168,89,182,6,114,45,18,66,117,167,185,223,174,145,213,173,13,120,2,
+17,2,234,154,247,211,152,151,68,178,189,243,191,100,69,56,220,204,135,8,
+8,243,106,81,200,161,112,254,168,188,204,223,41,199,179,118,175,24,80,181,
+170,251,119,53,153,144,106,152,193,156,110,2,76,22,210,189,54,142,117,176,
+41,12,75,205,151,114,6,13,189,73,198,171,142,128,101,182,209,222,11,114,
+45,39,232,119,242,223,111,146,173,163,117,232,158,82,227,218,55,181,153,209,
+191,44,235,153,169,79,18,202,33,154,149,43,125,156,37,109,244,43,86,226,
+56,81,141,51,169,137,156,143,5,117,89,207,170,127,215,28,116,171,7,16,
+180,47,117,179,112,16,184,46,223,127,230,15,226,49,143,34,161,0,74,190,
+164,205,33,6,69,47,76,151,176,62,237,109,170,137,103,59,171,2,60,229,
+86,41,126,159,61,9,180,190,18,223,84,23,218,160,202,214,171,221,113,204,
+223,96,236,3,154,57,72,156,53,119,189,98,142,172,23,47,223,207,157,224,
+57,119,67,120,18,183,228,129,205,184,146,183,200,106,135,187,11,115,171,131,
+251,158,86,210,209,130,59,30,225,207,255,154,136,104,232,164,216,111,202,39,
+238,160,200,71,247,26,87,37,249,213,204,171,77,16,203,192,251,40,218,194,
+95,209,125,2,40,146,99,233,97,2,230,42,37,159,217,203,58,10,96,182,
+87,9,157,37,95,143,33,75,0,126,233,64,250,86,148,124,114,213,94,31,
+0,78,79,79,222,171,188,103,37,48,242,25,135,37,248,43,201,102,126,30,
+100,113,223,59,89,234,155,103,117,168,48,180,178,58,35,117,89,197,128,226,
+143,119,68,57,227,238,78,14,8,80,192,90,193,1,100,162,163,120,57,94,
+116,120,206,152,147,169,54,85,42,86,144,120,19,99,62,206,56,224,76,128,
+251,235,161,87,69,139,212,197,6,191,205,144,101,220,184,119,103,245,6,58,
+183,235,138,35,144,193,33,211,70,5,69,3,102,203,97,194,49,102,114,200,
+52,199,181,175,114,151,75,40,18,3,10,73,189,60,47,246,169,114,6,229,
+65,165,111,45,114,7,51,136,91,213,86,46,225,115,110,169,124,107,27,26,
+32,226,191,146,204,127,15,192,70,59,90,187,83,189,66,74,43,106,248,135,
+173,68,134,6,70,29,237,15,147,187,75,94,62,253,39,202,198,17,10,42,
+17,170,214,42,24,75,111,46,1,39,12,20,163,215,188,178,161,61,191,80,
+127,254,5,8,191,218,123,54,100,108,192,225,250,11,159,198,32,103,77,33,
+72,35,22,227,209,71,206,205,179,45,195,11,59,1,125,216,24,109,81,44,
+177,108,134,55,180,112,120,22,112,47,193,108,184,206,50,45,91,19,179,17,
+64,241,139,188,116,249,37,142,99,243,134,29,141,233,131,199,53,191,77,133,
+1,130,226,123,237,139,236,88,124,251,156,103,146,19,194,154,14,104,17,186,
+224,105,154,25,43,154,2,100,58,218,113,159,64,135,241,111,232,168,201,201,
+255,210,172,125,247,147,19,14,159,203,141,177,112,138,78,106,147,76,34,124,
+193,212,81,244,140,12,98,186,140,231,112,24,201,156,35,87,63,226,132,82,
+242,34,126,13,25,93,191,210,33,108,109,167,141,54,137,243,188,174,7,154,
+34,150,166,58,77,250,40,77,63,235,52,242,134,63,14,44,133,88,72,217,
+4,216,7,212,165,80,138,105,150,212,172,80,213,55,164,231,22,234,128,223,
+80,53,230,137,13,232,174,84,130,18,222,110,192,9,114,210,213,161,76,212,
+89,121,158,133,55,189,140,211,65,214,202,133,208,12,223,229,237,202,131,113,
+131,128,189,56,172,139,26,159,43,75,179,164,52,199,246,144,186,76,255,50,
+0,243,35,107,159,192,242,228,194,64,173,204,29,200,146,187,117,191,55,215,
+123,148,157,165,204,129,176,90,56,155,10,86,24,237,70,159,164,238,197,183,
+218,222,23,227,139,159,121,169,153,244,163,59,220,128,211,45,186,64,205,15,
+108,212,2,16,99,232,130,209,70,224,159,2,183,217,187,41,43,62,121,158,
+237,206,181,1,41,115,64,118,11,122,24,47,119,19,252,15,79,206,35,27,
+244,246,90,32,145,55,118,149,255,220,93,169,24,253,78,247,129,226,45,106,
+190,92,81,86,92,107,235,67,94,236,37,184,157,40,70,69,210,210,229,208,
+45,199,17,25,21,63,155,44,200,30,112,139,74,123,197,199,131,127,176,222,
+164,226,78,206,56,140,9,137,152,44,246,78,127,250,215,40,181,188,57,235,
+36,37,183,16,21,70,166,215,58,172,137,159,76,226,5,140,62,246,223,238,
+54,15,2,212,228,87,10,80,27,172,202,63,128,132,117,56,254,85,163,79,
+47,191,180,135,19,60,133,249,43,165,183,91,179,0,184,162,237,99,61,44,
+62,72,27,64,12,82,15,231,212,134,147,94,247,63,62,65,203,62,63,208,
+1,254,11,10,73,133,42,53,81,151,235,54,106,23,106,133,187,45,67,11,
+220,225,54,217,139,72,65,24,218,101,144,141,196,88,138,115,231,49,182,77,
+105,63,10,186,191,129,125,7,195,86,255,239,79,88,91,104,198,237,25,142,
+184,215,24,191,254,220,211,216,176,142,167,54,180,238,41,193,177,183,134,162,
+33,208,20,151,217,77,238,191,232,55,76,60,117,52,87,56,61,110,238,135,
+122,201,202,58,85,125,4,23,134,232,10,23,18,149,122,116,28,243,196,33,
+22,171,33,243,48,180,94,122,145,168,54,194,80,56,79,188,144,153,154,185,
+219,200,76,64,155,221,18,102,43,112,37,62,215,95,63,190,161,149,194,23,
+15,237,228,170,241,171,168,210,81,232,66,145,32,105,240,146,3,115,36,144,
+15,223,90,251,152,171,182,191,144,220,16,72,80,126,119,247,78,146,88,3,
+164,181,76,161,23,92,211,178,84,20,250,218,69,22,222,204,41,89,82,160,
+91,96,118,47,242,172,147,51,15,233,200,124,62,53,24,91,89,45,83,20,
+221,234,89,42,62,67,221,9,63,113,34,170,117,42,102,152,22,20,48,74,
+15,0,163,53,243,82,198,54,209,39,124,42,178,14,199,11,222,84,44,34,
+240,146,136,128,225,36,94,254,224,229,25,174,220,11,76,118,61,19,226,54,
+62,108,218,152,42,27,191,132,254,53,232,167,57,220,70,207,187,24,35,210,
+251,71,39,236,56,136,113,157,188,131,165,134,182,87,231,60,96,211,101,222,
+235,215,86,227,123,98,83,56,175,252,13,209,18,192,168,169,155,106,207,99,
+176,59,188,244,30,58,77,168,180,214,80,200,228,12,206,13,250,58,114,62,
+128,147,68,34,88,176,235,17,174,36,154,126,129,108,165,166,178,92,123,210,
+71,101,101,240,159,50,171,95,93,19,216,232,131,90,247,173,1,228,223,125,
+209,118,54,67,56,130,23,3,132,88,68,93,130,52,250,57,196,172,165,212,
+49,90,87,98,33,200,254,203,199,43,50,177,150,18,41,218,3,84,7,124,
+244,188,128,197,143,83,175,18,113,47,6,233,102,247,217,228,88,218,113,248,
+102,10,110,96,67,13,104,162,181,228,52,147,61,6,240,61,222,31,174,255,
+180,217,140,153,208,148,41,216,59,245,15,182,31,113,248,211,85,34,120,161,
+67,179,95,46,232,227,209,191,162,73,67,98,20,32,16,159,167,118,72,41,
+110,17,152,39,57,140,245,99,248,197,255,63,72,155,132,18,87,207,241,17,
+82,232,4,159,9,135,30,54,7,228,220,133,170,58,103,50,123,159,107,57,
+143,7,186,29,130,69,199,39,60,242,30,36,40,159,246,70,127,45,45,194,
+113,31,113,85,63,94,248,45,180,242,52,57,50,68,243,122,83,40,233,249,
+30,93,74,34,165,209,93,158,88,39,29,67,73,239,137,0,141,105,31,61,
+52,206,146,61,43,169,249,144,64,24,112,195,219,141,197,15,11,232,55,225,
+231,167,33,227,86,182,213,83,138,184,25,111,80,135,115,203,120,134,43,54,
+118,104,222,51,33,70,91,144,226,163,132,241,117,252,80,218,68,28,246,147,
+63,212,205,175,225,156,189,64,77,113,141,85,64,155,158,11,243,177,60,194,
+116,132,214,175,191,86,106,132,37,70,166,36,75,129,212,95,197,227,49,17,
+143,54,18,21,146,162,132,154,42,222,252,176,80,59,219,26,118,103,154,204,
+236,158,63,124,94,30,37,38,93,195,182,7,89,157,254,0,129,176,6,40,
+241,126,20,247,202,48,206,118,190,200,72,2,241,47,223,122,191,82,127,46,
+14,130,30,170,123,249,18,54,201,151,143,78,73,6,168,152,122,24,7,42,
+175,56,56,193,112,137,173,34,199,12,236,144,192,161,186,205,246,141,100,79,
+70,188,140,93,131,10,138,255,185,149,22,176,215,163,131,214,9,59,11,29,
+100,208,239,15,98,76,78,35,129,191,16,208,123,147,25,232,200,252,179,69,
+39,240,34,1,183,255,65,243,65,220,163,110,151,85,72,180,100,208,49,1,
+7,23,52,220,14,181,106,192,148,47,175,168,40,216,118,15,65,245,218,140,
+0,165,169,58,246,118,79,14,59,45,185,250,39,20,207,87,73,123,90,194,
+55,45,168,21,97,59,116,137,97,91,86,130,18,150,187,123,43,112,93,14,
+224,175,47,228,179,207,130,237,155,51,185,249,83,94,38,59,64,41,93,226,
+113,129,48,253,166,49,56,39,114,108,196,118,81,159,250,186,49,110,79,18,
+21,223,66,227,97,0,162,114,114,199,164,192,194,73,125,139,210,195,254,69,
+31,101,63,227,15,249,180,137,93,98,190,214,238,130,92,193,44,108,114,33,
+107,86,186,84,26,20,100,160,12,59,225,43,22,80,238,66,159,61,86,200,
+165,127,34,57,0,9,251,63,83,172,206,229,225,69,128,199,49,120,62,127,
+178,144,72,29,81,158,57,107,9,77,182,232,171,220,215,183,72,176,85,237,
+203,99,218,1,81,89,225,227,51,208,227,20,132,193,205,96,94,10,55,27,
+92,83,0,184,199,133,173,21,105,13,70,95,92,4,249,251,121,220,53,91,
+65,147,235,105,36,86,201,240,222,182,105,96,40,197,230,253,131,255,226,157,
+42,155,119,55,31,85,225,88,220,231,106,136,239,127,240,39,157,123,27,191,
+8,157,247,206,2,108,170,74,29,153,99,184,243,218,200,154,6,73,43,169,
+66,181,29,98,199,150,30,35,217,171,174,195,35,75,174,104,4,131,70,53,
+8,143,41,74,84,5,128,26,84,37,82,225,53,4,29,96,197,228,149,248,
+42,230,118,220,15,243,112,95,74,227,83,99,107,253,242,48,127,74,53,128,
+33,248,100,24,132,142,2,117,218,29,108,153,194,119,239,219,189,200,233,24,
+117,64,117,148,160,162,178,36,201,162,148,103,215,94,223,86,195,140,214,224,
+106,7,87,9,58,84,89,1,178,176,152,250,44,117,94,147,182,113,102,108,
+45,47,89,19,184,156,100,20,202,74,239,141,205,178,73,175,15,254,59,124,
+137,62,153,142,151,223,200,94,240,244,61,31,201,41,36,117,85,1,87,96,
+50,204,126,221,209,135,142,85,32,255,234,138,110,3,186,130,226,205,230,20,
+68,128,58,30,94,102,17,242,24,159,133,118,219,128,128,69,54,31,40,94,
+50,241,17,58,163,44,191,220,192,250,39,44,221,51,179,141,212,98,43,8,
+187,8

<TRUNCATED>

[11/23] orc git commit: ORC-23. Simplify directory structure.

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/orc/blob/7f55b453/c++/test/TestRle.cc
----------------------------------------------------------------------
diff --git a/c++/test/TestRle.cc b/c++/test/TestRle.cc
new file mode 100644
index 0000000..5cd03a7
--- /dev/null
+++ b/c++/test/TestRle.cc
@@ -0,0 +1,2639 @@
+/**
+ * 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 "Compression.hh"
+#include "OrcTest.hh"
+#include "RLE.hh"
+#include "wrap/gtest-wrapper.h"
+
+#include <iostream>
+#include <vector>
+
+namespace orc {
+
+std::vector<int64_t> decodeRLEv2(const unsigned char *bytes,
+                                 unsigned long l,
+                                 size_t n,
+                                 size_t count,
+                                 const char* notNull = nullptr) {
+  std::unique_ptr<RleDecoder> rle =
+    createRleDecoder(std::unique_ptr<SeekableInputStream>
+                     (new SeekableArrayInputStream(bytes,l)), true,
+                     RleVersion_2, *getDefaultPool());
+  std::vector<int64_t> results;
+  for (size_t i = 0; i < count; i+=n) {
+    size_t remaining = count - i;
+    size_t nread = std::min(n, remaining);
+    std::vector<int64_t> data(nread);
+    rle->next(data.data(), nread, notNull);
+    if (notNull) {
+      notNull += nread;
+    }
+    results.insert(results.end(), data.begin(), data.end());
+  }
+
+  return results;
+}
+
+void checkResults(const std::vector<int64_t> &e, const std::vector<int64_t> &a,
+                  size_t n, const char* notNull = nullptr) {
+  EXPECT_EQ(e.size(), a.size()) << "vectors differ in size";
+  for (size_t i = 0; i < e.size(); ++i) {
+    if (!notNull || notNull[i]) {
+      EXPECT_EQ(e[i], a[i]) << "Output wrong at " << i << ", n=" << n;
+    }
+  }
+}
+
+TEST(RLEv2, basicDelta0) {
+  const size_t count = 20;
+  std::vector<int64_t> values;
+  for (size_t i = 0; i < count; ++i) {
+    values.push_back(static_cast<int64_t>(i));
+  }
+
+  const unsigned char bytes[] = {0xc0,0x13,0x00,0x02};
+  unsigned long l = sizeof(bytes) / sizeof(char);
+  // Read 1 at a time, then 3 at a time, etc.
+  checkResults(values, decodeRLEv2(bytes, l, 1, count), 1);
+  checkResults(values, decodeRLEv2(bytes, l, 3, count), 3);
+  checkResults(values, decodeRLEv2(bytes, l, 7, count), 7);
+  checkResults(values, decodeRLEv2(bytes, l, count, count), count);
+};
+
+TEST(RLEv2, basicDelta1) {
+  std::vector<int64_t> values(5);
+  values[0] = -500;
+  values[1] = -400;
+  values[2] = -350;
+  values[3] = -325;
+  values[4] = -310;
+
+  const unsigned char bytes[] = {0xce,0x04,0xe7,0x07,0xc8,0x01,0x32,0x19,0x0f};
+  unsigned long l = sizeof(bytes) / sizeof(char);
+  // Read 1 at a time, then 3 at a time, etc.
+  checkResults(values, decodeRLEv2(bytes, l, 1, values.size()), 1);
+  checkResults(values, decodeRLEv2(bytes, l, 3, values.size()), 3);
+  checkResults(values, decodeRLEv2(bytes, l, 7, values.size()), 7);
+  checkResults(values, decodeRLEv2(bytes, l, values.size(), values.size()),
+               values.size());
+};
+
+TEST(RLEv2, basicDelta2) {
+  std::vector<int64_t> values(5);
+  values[0] = -500;
+  values[1] = -600;
+  values[2] = -650;
+  values[3] = -675;
+  values[4] = -710;
+
+  const unsigned char bytes[] = {0xce,0x04,0xe7,0x07,0xc7,0x01,0x32,0x19,0x23};
+  unsigned long l = sizeof(bytes) / sizeof(char);
+  // Read 1 at a time, then 3 at a time, etc.
+  checkResults(values, decodeRLEv2(bytes, l, 1, values.size()), 1);
+  checkResults(values, decodeRLEv2(bytes, l, 3, values.size()), 3);
+  checkResults(values, decodeRLEv2(bytes, l, 7, values.size()), 7);
+  checkResults(values, decodeRLEv2(bytes, l, values.size(), values.size()),
+               values.size());
+};
+
+TEST(RLEv2, basicDelta3) {
+  std::vector<int64_t> values(5);
+  values[0] = 500;
+  values[1] = 400;
+  values[2] = 350;
+  values[3] = 325;
+  values[4] = 310;
+
+  const unsigned char bytes[] = {0xce,0x04,0xe8,0x07,0xc7,0x01,0x32,0x19,0x0f};
+  unsigned long l = sizeof(bytes) / sizeof(char);
+  // Read 1 at a time, then 3 at a time, etc.
+  checkResults(values, decodeRLEv2(bytes, l, 1, values.size()), 1);
+  checkResults(values, decodeRLEv2(bytes, l, 3, values.size()), 3);
+  checkResults(values, decodeRLEv2(bytes, l, 7, values.size()), 7);
+  checkResults(values, decodeRLEv2(bytes, l, values.size(), values.size()),
+               values.size());
+};
+
+TEST(RLEv2, basicDelta4) {
+  std::vector<int64_t> values(5);
+  values[0] = 500;
+  values[1] = 600;
+  values[2] = 650;
+  values[3] = 675;
+  values[4] = 710;
+
+  const unsigned char bytes[] = {0xce,0x04,0xe8,0x07,0xc8,0x01,0x32,0x19,0x23};
+  unsigned long l = sizeof(bytes) / sizeof(char);
+  // Read 1 at a time, then 3 at a time, etc.
+  checkResults(values, decodeRLEv2(bytes, l, 1, values.size()), 1);
+  checkResults(values, decodeRLEv2(bytes, l, 3, values.size()), 3);
+  checkResults(values, decodeRLEv2(bytes, l, 7, values.size()), 7);
+  checkResults(values, decodeRLEv2(bytes, l, values.size(), values.size()),
+               values.size());
+};
+
+TEST(RLEv2, delta0Width) {
+  const unsigned char buffer[] = {0x4e, 0x2, 0x0, 0x1, 0x2, 0xc0, 0x2, 0x42,
+				  0x0};
+  std::unique_ptr<RleDecoder> decoder =
+    createRleDecoder(std::unique_ptr<SeekableInputStream>
+                     (new SeekableArrayInputStream
+                      (buffer, ARRAY_SIZE(buffer))),
+                     false, RleVersion_2, *getDefaultPool());
+  int64_t values[6];
+  decoder->next(values, 6, 0);
+  EXPECT_EQ(0, values[0]);
+  EXPECT_EQ(1, values[1]);
+  EXPECT_EQ(2, values[2]);
+  EXPECT_EQ(0x42, values[3]);
+  EXPECT_EQ(0x42, values[4]);
+  EXPECT_EQ(0x42, values[5]);
+}
+
+TEST(RLEv2, basicDelta0WithNulls) {
+  std::vector<int64_t> values;
+  std::vector<char> notNull;
+  for (size_t i = 0; i < 20; ++i) {
+    values.push_back(static_cast<int64_t>(i));
+    notNull.push_back(true);
+    // throw in a null every third value
+    bool addNull = (i % 3 == 0);
+    if (addNull) {
+      values.push_back(-1);
+      notNull.push_back(false);
+    }
+  }
+
+  const unsigned char bytes[] = {0xc0,0x13,0x00,0x02};
+  unsigned long l = sizeof(bytes) / sizeof(char);
+  const size_t count = values.size();
+  // Read 1 at a time, then 3 at a time, etc.
+  checkResults(values, decodeRLEv2(bytes, l, 1, count, notNull.data()),
+               1, notNull.data());
+  checkResults(values, decodeRLEv2(bytes, l, 3, count, notNull.data()),
+               3, notNull.data());
+  checkResults(values, decodeRLEv2(bytes, l, 7, count, notNull.data()),
+               7, notNull.data());
+  checkResults(values, decodeRLEv2(bytes, l, count, count, notNull.data()),
+               count, notNull.data());
+};
+
+TEST(RLEv2, shortRepeats) {
+  const size_t runLength = 7;
+  const size_t nVals = 10;
+  const size_t count = nVals * runLength;
+  std::vector<int64_t> values;
+  for (size_t i = 0; i < nVals; ++i) {
+    for (size_t j = 0; j < runLength; ++j) {
+      values.push_back(static_cast<int64_t>(i));
+    }
+  }
+
+  const unsigned char bytes[] = {0x04,0x00,0x04,0x02,0x04,0x04,0x04,
+                                 0x06,0x04,0x08,0x04,0x0a,0x04,0x0c,
+                                 0x04,0x0e,0x04,0x10,0x04,0x12};
+  unsigned long l = sizeof(bytes) / sizeof(char);
+  // Read 1 at a time, then 3 at a time, etc.
+  checkResults(values, decodeRLEv2(bytes, l, 1, count), 1);
+  checkResults(values, decodeRLEv2(bytes, l, 3, count), 3);
+  checkResults(values, decodeRLEv2(bytes, l, 7, count), 7);
+  checkResults(values, decodeRLEv2(bytes, l, count, count), count);
+};
+
+TEST(RLEv2, multiByteShortRepeats) {
+  const size_t runLength = 7;
+  const size_t nVals = 3;
+  const size_t count = nVals * runLength;
+  std::vector<int64_t> values;
+  for (size_t i = 0; i < nVals; ++i) {
+    for (size_t j = 0; j < runLength; ++j) {
+      values.push_back(static_cast<int64_t>(i)+(1L<<62));
+    }
+  }
+
+  const unsigned char bytes[] = {0x3c,0x80,0x00,0x00,0x00,0x00,0x00,0x00,0x00,
+                                 0x3c,0x80,0x00,0x00,0x00,0x00,0x00,0x00,0x02,
+                                 0x3c,0x80,0x00,0x00,0x00,0x00,0x00,0x00,0x04};
+  unsigned long l = sizeof(bytes) / sizeof(char);
+  // Read 1 at a time, then 3 at a time, etc.
+  checkResults(values, decodeRLEv2(bytes, l, 1, count), 1);
+  checkResults(values, decodeRLEv2(bytes, l, 3, count), 3);
+  checkResults(values, decodeRLEv2(bytes, l, 7, count), 7);
+  checkResults(values, decodeRLEv2(bytes, l, count, count), count);
+};
+
+TEST(RLEv2, 0to2Repeat1Direct) {
+  const unsigned char buffer[] = {0x46, 0x02, 0x02, 0x40};
+  std::unique_ptr<RleDecoder> rle =
+      createRleDecoder(std::unique_ptr<SeekableInputStream>
+		       (new SeekableArrayInputStream(buffer,
+						     ARRAY_SIZE(buffer))),
+		       true, RleVersion_2, *getDefaultPool());
+  std::vector<int64_t> data(3);
+  rle->next(data.data(), 3, nullptr);
+
+  for (size_t i = 0; i < data.size(); ++i) {
+    EXPECT_EQ(i, data[i]) << "Output wrong at " << i;
+  }
+};
+
+TEST(RLEv2, bitSize2Direct) {
+ // 0,1 repeated 10 times (signed ints)
+ const size_t count = 20;
+ std::vector<int64_t> values;
+ for (size_t i = 0; i < count; ++i) {
+     values.push_back(i%2);
+ }
+
+ const unsigned char bytes[] = {0x42, 0x13, 0x22, 0x22, 0x22, 0x22, 0x22};
+ unsigned long l = sizeof(bytes) / sizeof(char);
+ // Read 1 at a time, then 3 at a time, etc.
+ checkResults(values, decodeRLEv2(bytes, l, 1, count), 1);
+ checkResults(values, decodeRLEv2(bytes, l, 3, count), 3);
+ checkResults(values, decodeRLEv2(bytes, l, 7, count), 7);
+ checkResults(values, decodeRLEv2(bytes, l, count, count), count);
+};
+
+TEST(RLEv2, bitSize4Direct) {
+ // 0,2 repeated 10 times (signed ints)
+ const size_t count = 20;
+ std::vector<int64_t> values;
+ for (size_t i = 0; i < count; ++i) {
+     values.push_back((i%2)*2);
+ }
+
+ const unsigned char bytes[] = {0x46,0x13,0x04,0x04,0x04,0x04,
+                                0x04,0x04,0x04,0x04,0x04,0x04};
+ unsigned long l = sizeof(bytes) / sizeof(char);
+
+ // Read 1 at a time, then 3 at a time, etc.
+ checkResults(values, decodeRLEv2(bytes, l, 1, count), 1);
+ checkResults(values, decodeRLEv2(bytes, l, 3, count), 3);
+ checkResults(values, decodeRLEv2(bytes, l, 7, count), 7);
+ checkResults(values, decodeRLEv2(bytes, l, count, count), count);
+};
+
+TEST(RLEv2, multipleRunsDirect) {
+ std::vector<int64_t> values;
+ // 0,1 repeated 10 times (signed ints)
+ for (size_t i = 0; i < 20; ++i) {
+     values.push_back(i%2);
+ }
+ // 0,2 repeated 10 times (signed ints)
+ for (size_t i = 0; i < 20; ++i) {
+     values.push_back((i%2)*2);
+ }
+
+ const unsigned char bytes[] = {0x42,0x13,0x22,0x22,0x22,0x22,0x22,
+                                0x46,0x13,0x04,0x04,0x04,0x04,0x04,
+                                0x04,0x04,0x04,0x04,0x04};
+ unsigned long l = sizeof(bytes) / sizeof(char);
+
+ // Read 1 at a time, then 3 at a time, etc.
+ checkResults(values, decodeRLEv2(bytes, l, 1, values.size()), 1);
+ checkResults(values, decodeRLEv2(bytes, l, 3, values.size()), 3);
+ checkResults(values, decodeRLEv2(bytes, l, 7, values.size()), 7);
+ checkResults(values, decodeRLEv2(bytes, l, values.size(), values.size()),
+              values.size());
+};
+
+TEST(RLEv2, largeNegativesDirect) {
+  const unsigned char buffer[] =
+    {0x7e,0x04,0xcf,0xca,0xcc,0x91,0xba,0x38,0x93,0xab,0x00,0x00,
+     0x00,0x00,0x00,0x00,0x00,0x00,0x00,0x00,0x00,0x00,0x00,0x00,
+     0x00,0x02,0x00,0x00,0x00,0x00,0x00,0x00,0x00,0x02,0x99,0xa5,
+     0xcc,0x28,0x03,0xf7,0xe0,0xff};
+  std::unique_ptr<RleDecoder> rle =
+      createRleDecoder(
+          std::unique_ptr<SeekableInputStream>(
+             new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer))),
+          true, RleVersion_2, *getDefaultPool());
+  std::vector<int64_t> data(5);
+  rle->next(data.data(), 5, nullptr);
+
+  EXPECT_EQ(-7486502418706614742, data[0]) << "Output wrong at " << 0;
+  EXPECT_EQ(0, data[1]) << "Output wrong at " << 1;
+  EXPECT_EQ(1, data[2]) << "Output wrong at " << 2;
+  EXPECT_EQ(1, data[3]) << "Output wrong at " << 3;
+  EXPECT_EQ(-5535739865598783616, data[4]) << "Output wrong at " << 4;
+};
+
+TEST(RLEv2, overflowDirect) {
+  std::vector<int64_t> values(4);
+  values[0] = 4513343538618202719l;
+  values[1] = 4513343538618202711l;
+  values[2] = 2911390882471569739l;
+  values[3] = -9181829309989854913l;
+
+  const unsigned char bytes[] = {0x7e,0x03,0x7d,0x45,0x3c,0x12,0x41,0x48,0xf4,
+                                 0xbe,0x7d,0x45,0x3c,0x12,0x41,0x48,0xf4,0xae,
+                                 0x50,0xce,0xad,0x2a,0x30,0x0e,0xd2,0x96,0xfe,
+                                 0xd8,0xd2,0x38,0x54,0x6e,0x3d,0x81};
+  unsigned long l = sizeof(bytes) / sizeof(char);
+  // Read 1 at a time, then 3 at a time, etc.
+  checkResults(values, decodeRLEv2(bytes, l, 1, values.size()), 1);
+  checkResults(values, decodeRLEv2(bytes, l, 3, values.size()), 3);
+  checkResults(values, decodeRLEv2(bytes, l, 7, values.size()), 7);
+  checkResults(values, decodeRLEv2(bytes, l, values.size(), values.size()),
+               values.size());
+};
+
+TEST(RLEv2, basicPatched0) {
+  long v[] = {2030,2000,2020,1000000,2040,2050,2060,2070,2080,2090};
+  std::vector<int64_t> values;
+  for (size_t i = 0; i < sizeof(v) / sizeof(long); ++i) {
+      values.push_back(v[i]);
+  }
+
+  const unsigned char bytes[] = {0x8e,0x09,0x2b,0x21,0x07,0xd0,0x1e,0x00,0x14,
+                                 0x70,0x28,0x32,0x3c,0x46,0x50,0x5a,0xfc,0xe8};
+  unsigned long l = sizeof(bytes) / sizeof(char);
+  // Read 1 at a time, then 3 at a time, etc.
+  checkResults(values, decodeRLEv2(bytes, l, 1, values.size()), 1);
+  checkResults(values, decodeRLEv2(bytes, l, 3, values.size()), 3);
+  checkResults(values, decodeRLEv2(bytes, l, 7, values.size()), 7);
+  checkResults(values, decodeRLEv2(bytes, l, values.size(), values.size()),
+               values.size());
+};
+
+TEST(RLEv2, basicPatched1) {
+  long v[] = {20, 2, 3, 2, 1, 3, 17, 71, 35, 2, 1, 139, 2, 2, 3, 1783, 475, 2,
+              1, 1, 3, 1, 3, 2, 32, 1, 2, 3, 1, 8, 30, 1, 3, 414, 1, 1, 135, 3,
+              3, 1, 414, 2, 1, 2, 2, 594, 2, 5, 6, 4, 11, 1, 2, 2, 1, 1, 52, 4,
+              1, 2, 7, 1, 17, 334, 1, 2, 1, 2, 2, 6, 1, 266, 1, 2, 217, 2, 6, 2,
+              13, 2, 2, 1, 2, 3, 5, 1, 2, 1, 7244, 11813, 1, 33, 2, -13, 1, 2, 3,
+              13, 1, 92, 3, 13, 5, 14, 9, 141, 12, 6, 15, 25};
+  std::vector<int64_t> values;
+  for (size_t i = 0; i < sizeof(v) / sizeof(long); ++i) {
+    values.push_back(v[i]);
+  }
+
+  const unsigned char bytes[] = {0x90,0x6d,0x04,0xa4,0x8d,0x10,0x83,0xc2,0x00,
+                                 0xf0,0x70,0x40,0x3c,0x54,0x18,0x03,0xc1,0xc9,
+                                 0x80,0x78,0x3c,0x21,0x04,0xf4,0x03,0xc1,0xc0,
+                                 0xe0,0x80,0x38,0x20,0x0f,0x16,0x83,0x81,0xe1,
+                                 0x00,0x70,0x54,0x56,0x0e,0x08,0x6a,0xc1,0xc0,
+                                 0xe4,0xa0,0x40,0x20,0x0e,0xd5,0x83,0xc1,0xc0,
+                                 0xf0,0x79,0x7c,0x1e,0x12,0x09,0x84,0x43,0x00,
+                                 0xe0,0x78,0x3c,0x1c,0x0e,0x20,0x84,0x41,0xc0,
+                                 0xf0,0xa0,0x38,0x3d,0x5b,0x07,0x03,0xc1,0xc0,
+                                 0xf0,0x78,0x4c,0x1d,0x17,0x07,0x03,0xdc,0xc0,
+                                 0xf0,0x98,0x3c,0x34,0x0f,0x07,0x83,0x81,0xe1,
+                                 0x00,0x90,0x38,0x1e,0x0e,0x2c,0x8c,0x81,0xc2,
+                                 0xe0,0x78,0x00,0x1c,0x0f,0x08,0x06,0x81,0xc6,
+                                 0x90,0x80,0x68,0x24,0x1b,0x0b,0x26,0x83,0x21,
+                                 0x30,0xe0,0x98,0x3c,0x6f,0x06,0xb7,0x03,0x70};
+  unsigned long l = sizeof(bytes) / sizeof(char);
+  // Read 1 at a time, then 3 at a time, etc.
+  checkResults(values, decodeRLEv2(bytes, l, 1, values.size()), 1);
+  checkResults(values, decodeRLEv2(bytes, l, 3, values.size()), 3);
+  checkResults(values, decodeRLEv2(bytes, l, 7, values.size()), 7);
+  checkResults(values, decodeRLEv2(bytes, l, values.size(), values.size()),
+               values.size());
+};
+
+TEST(RLEv2, mixedPatchedAndShortRepeats) {
+  long v[] = {20, 2, 3, 2, 1, 3, 17, 71, 35, 2, 1, 139, 2, 2, 3, 1783, 475, 2, 1, 1, 3, 1, 3, 2, 32, 1, 2, 3, 1, 8, 30, 1, 3, 414, 1, 1, 135, 3, 3, 1, 414, 2, 1, 2, 2, 594, 2, 5, 6, 4, 11, 1, 2, 2, 1, 1, 52, 4, 1, 2, 7, 1, 17, 334, 1, 2, 1, 2, 2, 6, 1, 266, 1, 2, 217, 2, 6, 2, 13, 2, 2, 1, 2, 3, 5, 1, 2, 1, 7244, 11813, 1, 33, 2, -13, 1, 2, 3, 13, 1, 92, 3, 13, 5, 14, 9, 141, 12, 6, 15, 25, 1, 1, 1, 46, 2, 1, 1, 141, 3, 1, 1, 1, 1, 2, 1, 4, 34, 5, 78, 8, 1, 2, 2, 1, 9, 10, 2, 1, 4, 13, 1, 5, 4, 4, 19, 5, 1, 1, 1, 68, 33, 399, 1, 1885, 25, 5, 2, 4, 1, 1, 2, 16, 1, 2966, 3, 1, 1, 25501, 1, 1, 1, 66, 1, 3, 8, 131, 14, 5, 1, 2, 2, 1, 1, 8, 1, 1, 2, 1, 5, 9, 2, 3, 112, 13, 2, 2, 1, 5, 10, 3, 1, 1, 13, 2, 3, 4, 1, 3, 1, 1, 2, 1, 1, 2, 4, 2, 207, 1, 1, 2, 4, 3, 3, 2, 2, 16};
+  std::vector<int64_t> values;
+  for (size_t i = 0; i < sizeof(v) / sizeof(long); ++i) {
+    values.push_back(v[i]);
+  }
+
+  const unsigned char bytes[] = {0x90,0x6d,0x04,0xa4,0x8d,0x10,0x83,0xc2,0x00,0xf0,0x70,0x40,0x3c,0x54,0x18,0x03,0xc1,0xc9,0x80,0x78,0x3c,0x21,0x04,0xf4,0x03,0xc1,0xc0,0xe0,0x80,0x38,0x20,0x0f,0x16,0x83,0x81,0xe1,0x00,0x70,0x54,0x56,0x0e,0x08,0x6a,0xc1,0xc0,0xe4,0xa0,0x40,0x20,0x0e,0xd5,0x83,0xc1,0xc0,0xf0,0x79,0x7c,0x1e,0x12,0x09,0x84,0x43,0x00,0xe0,0x78,0x3c,0x1c,0x0e,0x20,0x84,0x41,0xc0,0xf0,0xa0,0x38,0x3d,0x5b,0x07,0x03,0xc1,0xc0,0xf0,0x78,0x4c,0x1d,0x17,0x07,0x03,0xdc,0xc0,0xf0,0x98,0x3c,0x34,0x0f,0x07,0x83,0x81,0xe1,0x00,0x90,0x38,0x1e,0x0e,0x2c,0x8c,0x81,0xc2,0xe0,0x78,0x00,0x1c,0x0f,0x08,0x06,0x81,0xc6,0x90,0x80,0x68,0x24,0x1b,0x0b,0x26,0x83,0x21,0x30,0xe0,0x98,0x3c,0x6f,0x06,0xb7,0x03,0x70,0x00,0x02,0x5e,0x05,0x00,0x5c,0x00,0x04,0x00,0x02,0x00,0x02,0x01,0x1a,0x00,0x06,0x01,0x02,0x8a,0x16,0x00,0x41,0x01,0x04,0x00,0xe1,0x10,0xd1,0xc0,0x04,0x10,0x08,0x24,0x10,0x03,0x30,0x01,0x03,0x0d,0x21,0x00,0xb0,0x00,0x02,0x5e,0x12,0x00,0x88,0x00,0x42,0x03,0x1e,0x00,0x02,0x0e,0xba,0x00,0x32
 ,0x00,0x0a,0x00,0x04,0x00,0x08,0x00,0x02,0x00,0x02,0x00,0x04,0x00,0x20,0x00,0x02,0x17,0x2c,0x00,0x06,0x00,0x02,0x00,0x02,0xc7,0x3a,0x00,0x02,0x8c,0x36,0x00,0xa2,0x01,0x82,0x00,0x10,0x70,0x43,0x42,0x00,0x02,0x04,0x00,0x00,0xe0,0x00,0x01,0x00,0x10,0x40,0x10,0x5b,0xc6,0x01,0x02,0x00,0x20,0x90,0x40,0x00,0x0c,0x02,0x08,0x18,0x00,0x40,0x00,0x01,0x00,0x00,0x08,0x30,0x33,0x80,0x00,0x02,0x0c,0x10,0x20,0x20,0x47,0x80,0x13,0x4c};
+  unsigned long l = sizeof(bytes) / sizeof(char);
+  // Read 1 at a time, then 3 at a time, etc.
+  checkResults(values, decodeRLEv2(bytes, l, 1, values.size()), 1);
+  checkResults(values, decodeRLEv2(bytes, l, 3, values.size()), 3);
+  checkResults(values, decodeRLEv2(bytes, l, 7, values.size()), 7);
+  checkResults(values, decodeRLEv2(bytes, l, values.size(), values.size()),
+               values.size());
+};
+
+TEST(RLEv2, basicDirectSeek) {
+  // 0,1 repeated 10 times (signed ints) followed by
+  // 0,2 repeated 10 times (signed ints)
+  const unsigned char bytes[] = {0x42,0x13,0x22,0x22,0x22,0x22,0x22,
+                                 0x46,0x13,0x04,0x04,0x04,0x04,0x04,
+                                 0x04,0x04,0x04,0x04,0x04};
+  unsigned long l = sizeof(bytes) / sizeof(char);
+
+  std::unique_ptr<RleDecoder> rle =
+      createRleDecoder(std::unique_ptr<SeekableInputStream>
+                       (new SeekableArrayInputStream(bytes,l)), true,
+                       RleVersion_2, *getDefaultPool());
+  std::list<uint64_t> position;
+  position.push_back(7); // byte position; skip first 20 [0 to 19]
+  position.push_back(13); // value position; skip 13 more [20 to 32]
+
+  PositionProvider location(position);
+  rle->seek(location);
+  std::vector<int64_t> data(3);
+  rle->next(data.data(), 3, nullptr);
+  EXPECT_EQ(2, data[0]);
+  EXPECT_EQ(0, data[1]);
+  EXPECT_EQ(2, data[2]);
+  rle->next(data.data(), 3, nullptr);
+  EXPECT_EQ(0, data[0]);
+  EXPECT_EQ(2, data[1]);
+  EXPECT_EQ(0, data[2]);
+  rle->next(data.data(), 1, nullptr);
+  EXPECT_EQ(2, data[0]);
+};
+
+TEST(RLEv2, bitsLeftByPreviousStream) {
+  // test for #109
+  // 118 DIRECT values, followed by PATHCED values
+  const unsigned char bytes[] = {
+    0x5a, 0x75, 0x92, 0x42, 0x49, 0x09, 0x2b, 0xa4,
+    0xae, 0x92, 0xc2, 0x4b, 0x89, 0x2f, 0x24, 0xbc,
+    0x93, 0x2a, 0x4c, 0xa9, 0x34, 0x24, 0xe0, 0x93,
+    0x92, 0x4e, 0xe9, 0x40, 0xa5, 0x04, 0x94, 0x12,
+    0x62, 0xa9, 0xc9, 0xa7, 0x26, 0x9c, 0xaa, 0x73,
+    0x09, 0xcd, 0x27, 0x34, 0x9c, 0xf2, 0x74, 0x49,
+    0xd3, 0x27, 0x50, 0x9d, 0x42, 0x75, 0x29, 0xd4,
+    0xa7, 0x5a, 0x9d, 0xaa, 0x79, 0x89, 0xe9, 0x27,
+    0xa4, 0x9e, 0xea, 0x7c, 0x29, 0xf6, 0x27, 0xdc,
+    0x9f, 0xb2, 0x7f, 0x4a, 0x00, 0xa8, 0x14, 0xa0,
+    0x72, 0x82, 0x8a, 0x19, 0x28, 0x6e, 0xa2, 0x52,
+    0x89, 0x4a, 0x28, 0x28, 0xa6, 0xa2, 0x9a, 0x8b,
+    0x6a, 0x2d, 0xa8, 0xb8, 0xa2, 0xe2, 0x8b, 0xaa,
+    0x53, 0xa9, 0x54, 0xa5, 0x92, 0x98, 0x6a, 0x62,
+    0xa9, 0x9c, 0xa6, 0x8a, 0x9b, 0xea, 0x70, 0x29,
+    0xd2, 0xa7, 0x52, 0x9d, 0x4a, 0x77, 0x29, 0xe0,
+    0xa7, 0xa2, 0x9e, 0xaa, 0x7b, 0x29, 0xf0, 0xa7,
+    0xd2, 0xa0, 0x0a, 0x84, 0x2a, 0x18, 0xa8, 0x72,
+    0xa1, 0xca, 0x89, 0x2a, 0x30, 0xa9, 0x4a, 0xa5,
+    0x4a, 0x96, 0x2a, 0xae, 0xab, 0x02, 0xac, 0x2b,
+    0x8d, 0x2e, 0x60, 0xb9, 0x82, 0xe7, 0x2b, 0x9f,
+    0xae, 0x84, 0xba, 0x52, 0xe9, 0xeb, 0xad, 0x2e,
+    0xb6, 0xbc, 0x32, 0xf1, 0xcb, 0xcc, 0x2f, 0x42,
+    0xbd, 0x8a, 0xf7, 0xcb, 0xe1, 0xaf, 0xa4, 0xbe,
+    0x9a, 0xfa, 0x6b, 0xeb, 0xaf, 0xba, 0xbe, 0xea,
+    0xfd, 0x2b, 0xf4, 0xaf, 0xd8, 0xbf, 0xfb, 0x00,
+    0x80, // <= end of DIRECT, start of PATCHED =>
+    0x90,0x6d,0x04,0xa4,0x8d,0x10,0x83,0xc2,0x00,
+    0xf0,0x70,0x40,0x3c,0x54,0x18,0x03,0xc1,0xc9,
+    0x80,0x78,0x3c,0x21,0x04,0xf4,0x03,0xc1,0xc0,
+    0xe0,0x80,0x38,0x20,0x0f,0x16,0x83,0x81,0xe1,
+    0x00,0x70,0x54,0x56,0x0e,0x08,0x6a,0xc1,0xc0,
+    0xe4,0xa0,0x40,0x20,0x0e,0xd5,0x83,0xc1,0xc0,
+    0xf0,0x79,0x7c,0x1e,0x12,0x09,0x84,0x43,0x00,
+    0xe0,0x78,0x3c,0x1c,0x0e,0x20,0x84,0x41,0xc0,
+    0xf0,0xa0,0x38,0x3d,0x5b,0x07,0x03,0xc1,0xc0,
+    0xf0,0x78,0x4c,0x1d,0x17,0x07,0x03,0xdc,0xc0,
+    0xf0,0x98,0x3c,0x34,0x0f,0x07,0x83,0x81,0xe1,
+    0x00,0x90,0x38,0x1e,0x0e,0x2c,0x8c,0x81,0xc2,
+    0xe0,0x78,0x00,0x1c,0x0f,0x08,0x06,0x81,0xc6,
+    0x90,0x80,0x68,0x24,0x1b,0x0b,0x26,0x83,0x21,
+    0x30,0xe0,0x98,0x3c,0x6f,0x06,0xb7,0x03,0x70
+  };
+  unsigned long l = sizeof(bytes) / sizeof(unsigned char);
+
+  // PATCHED values.
+  long v[] = {20, 2, 3, 2, 1, 3, 17, 71, 35, 2, 1, 139, 2, 2, 3, 1783, 475, 2,
+              1, 1, 3, 1, 3, 2, 32, 1, 2, 3, 1, 8, 30, 1, 3, 414, 1, 1, 135, 3,
+              3, 1, 414, 2, 1, 2, 2, 594, 2, 5, 6, 4, 11, 1, 2, 2, 1, 1, 52, 4,
+              1, 2, 7, 1, 17, 334, 1, 2, 1, 2, 2, 6, 1, 266, 1, 2, 217, 2, 6, 2,
+              13, 2, 2, 1, 2, 3, 5, 1, 2, 1, 7244, 11813, 1, 33, 2, -13, 1, 2, 3,
+              13, 1, 92, 3, 13, 5, 14, 9, 141, 12, 6, 15, 25};
+  unsigned long D = 118, P = sizeof(v) / sizeof(long), N = D + P;
+
+  std::unique_ptr<RleDecoder> rle =
+      createRleDecoder(std::unique_ptr<SeekableInputStream>
+                       (new SeekableArrayInputStream(bytes,l)), true,
+                       RleVersion_2, *getDefaultPool());
+
+  std::vector<int64_t> data(N);
+  rle->next(data.data(), N, nullptr);
+  // check patched values
+  for (size_t i=0; i < P; ++i) {
+      EXPECT_EQ(v[i], data[i+D]);
+  }
+};
+
+TEST(RLEv1, simpleTest) {
+  const unsigned char buffer[] = {0x61, 0xff, 0x64, 0xfb, 0x02, 0x03, 0x5, 0x7,
+				  0xb};
+  std::unique_ptr<RleDecoder> rle =
+      createRleDecoder(std::unique_ptr<SeekableInputStream>
+		       (new SeekableArrayInputStream(buffer,
+						     ARRAY_SIZE(buffer))),
+		       false, RleVersion_1, *getDefaultPool());
+  std::vector<int64_t> data(105);
+  rle->next(data.data(), 105, nullptr);
+
+  for (size_t i = 0; i < 100; ++i) {
+    EXPECT_EQ(100 - i, data[i]) << "Output wrong at " << i;
+  }
+  EXPECT_EQ(2, data[100]);
+  EXPECT_EQ(3, data[101]);
+  EXPECT_EQ(5, data[102]);
+  EXPECT_EQ(7, data[103]);
+  EXPECT_EQ(11, data[104]);
+};
+
+TEST(RLEv1, signedNullLiteralTest) {
+  const unsigned char buffer[]= {0xf8, 0x0, 0x1, 0x2, 0x3, 0x4, 0x5, 0x6, 0x7};
+  std::unique_ptr<RleDecoder> rle =
+      createRleDecoder(std::unique_ptr<SeekableInputStream>
+		       (new SeekableArrayInputStream(buffer,
+						     ARRAY_SIZE(buffer))),
+		       true, RleVersion_1, *getDefaultPool());
+  std::vector<int64_t> data(8);
+  std::vector<char> notNull(8, 1);
+  rle->next(data.data(), 8, notNull.data());
+
+  for (size_t i = 0; i < 8; ++i) {
+    EXPECT_EQ(i % 2 == 0 ? i/2 : -((i+1)/2),
+              data[i]);
+  }
+}
+
+TEST(RLEv1, splitHeader) {
+  const unsigned char buffer[] = {0x0, 0x00, 0xdc, 0xba, 0x98, 0x76};
+  std::unique_ptr<RleDecoder> rle =
+      createRleDecoder(
+          std::unique_ptr<SeekableInputStream>
+          (new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer), 4)),
+          false, RleVersion_1, *getDefaultPool());
+  std::vector<int64_t> data(200);
+  rle->next(data.data(), 3, nullptr);
+
+  for (size_t i = 0; i < 3; ++i) {
+    EXPECT_EQ(247864668, data[i]) << "Output wrong at " << i;
+  }
+}
+
+TEST(RLEv1, splitRuns) {
+  const unsigned char buffer[] = {0x7d, 0x01, 0xff, 0x01, 0xfb, 0x01,
+				  0x02, 0x03, 0x04, 0x05};
+  SeekableInputStream* const stream =
+    new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer));
+  std::unique_ptr<RleDecoder> rle =
+      createRleDecoder(std::unique_ptr<SeekableInputStream>(stream),
+                       false, RleVersion_1, *getDefaultPool());
+  std::vector<int64_t> data(200);
+  for (size_t i = 0; i < 42; ++i) {
+    rle->next(data.data(), 3, nullptr);
+    for (size_t j = 0; j < 3; ++j) {
+      EXPECT_EQ(255 + i * 3 + j, data[j])
+      << "Wrong output at " << i << ", " << j;
+    }
+  }
+  rle->next(data.data(), 3, nullptr);
+  EXPECT_EQ(381, data[0]);
+  EXPECT_EQ(382, data[1]);
+  EXPECT_EQ(1, data[2]);
+  rle->next(data.data(), 3, nullptr);
+  EXPECT_EQ(2, data[0]);
+  EXPECT_EQ(3, data[1]);
+  EXPECT_EQ(4, data[2]);
+  rle->next(data.data(), 1, nullptr);
+  EXPECT_EQ(5, data[0]);
+}
+
+TEST(RLEv1, testSigned) {
+  const unsigned char buffer[] = {0x7f, 0xff, 0x20};
+  SeekableInputStream* const stream =
+    new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer));
+  std::unique_ptr<RleDecoder> rle =
+      createRleDecoder(std::unique_ptr<SeekableInputStream>(stream),
+                       true, RleVersion_1, *getDefaultPool());
+  std::vector<int64_t> data(100);
+  rle->next(data.data(), data.size(), nullptr);
+  for (size_t i = 0; i < data.size(); ++i) {
+    EXPECT_EQ(16 - i, data[i]) << "Wrong output at " << i;
+  }
+  rle->next(data.data(), 30, nullptr);
+  for(size_t i = 0; i < 30; ++i) {
+    EXPECT_EQ(16 - 100 - static_cast<long>(i), data[i])
+      << "Wrong output at " << (i + 100);
+  }
+}
+
+TEST(RLEv1, testNull) {
+  const unsigned char buffer[] = {0x75, 0x02, 0x00};
+  SeekableInputStream* const stream =
+    new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer));
+  std::unique_ptr<RleDecoder> rle =
+      createRleDecoder(std::unique_ptr<SeekableInputStream>(stream),
+                       true, RleVersion_1, *getDefaultPool());
+  std::vector<int64_t> data(24);
+  std::vector<char> notNull(24);
+  for (size_t i = 0; i < notNull.size(); ++i) {
+    notNull[i] = (i + 1) % 2;
+  }
+  for (size_t i = 0; i < 10; ++i) {
+    for(size_t j = 0; j < data.size(); ++j) {
+      data[j] = -1;
+    }
+    rle->next(data.data(), 24, notNull.data());
+    for (size_t j = 0; j < 24; ++j) {
+      if (notNull[j]) {
+        EXPECT_EQ(i * 24 + j, data[j]);
+      } else {
+        EXPECT_EQ(-1, data[j]);
+      }
+    }
+  }
+}
+
+TEST(RLEv1, testAllNulls) {
+  const unsigned char buffer[] = 
+    {0xf0, 0x00, 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07,
+     0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f,
+     0x3d, 0x00, 0x12};
+  SeekableInputStream* const stream =
+    new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer));
+  std::unique_ptr<RleDecoder> rle =
+      createRleDecoder(std::unique_ptr<SeekableInputStream>(stream),
+                       false, RleVersion_1, *getDefaultPool());
+  std::vector<int64_t> data(16, -1);
+  std::vector<char> allNull(16, 0);
+  std::vector<char> noNull(16, 1);
+  rle->next(data.data(), 16, allNull.data());
+  for (size_t i = 0; i < data.size(); ++i) {
+    EXPECT_EQ(-1, data[i]) << "Output wrong at " << i;
+  }
+  rle->next(data.data(), data.size(), noNull.data());
+  for (size_t i = 0; i < data.size(); ++i) {
+    EXPECT_EQ(i, data[i]) << "Output wrong at " << i;
+    data[i] = -1;
+  }
+  rle->next(data.data(), data.size(), allNull.data());
+  for (size_t i = 0; i < data.size(); ++i) {
+    EXPECT_EQ(-1, data[i]) << "Output wrong at " << i;
+  }
+  for (size_t i = 0; i < 4; ++i) {
+    rle->next(data.data(), data.size(), noNull.data());
+    for(size_t j = 0; j < data.size(); ++j) {
+      EXPECT_EQ(18, data[j]) << "Output wrong at " << i;
+    }
+  }
+  rle->next(data.data(), data.size(), allNull.data());
+}
+
+TEST(RLEv1, skipTest) {
+  // Create the RLE stream from Java's TestRunLengthIntegerEncoding.testSkips
+  // for (size_t i = 0; i < 1024; ++i)
+  //   out.write(i);
+  // for (size_t i = 1024; i < 2048; ++i)
+  //   out.write(i * 256);
+  // This causes the first half to be delta encoded and the second half to
+  // be literal encoded.
+  const unsigned char buffer[] = 
+{
+127,   1,   0, 127,   1, 132,   2, 127,   1, 136,   4, 127,   1, 140,   6, 127,
+  1, 144,   8, 127,   1, 148,  10, 127,   1, 152,  12, 111,   1, 156,  14, 128,
+128, 128,  32, 128, 132,  32, 128, 136,  32, 128, 140,  32, 128, 144,  32, 128,
+148,  32, 128, 152,  32, 128, 156,  32, 128, 160,  32, 128, 164,  32, 128, 168,
+ 32, 128, 172,  32, 128, 176,  32, 128, 180,  32, 128, 184,  32, 128, 188,  32,
+128, 192,  32, 128, 196,  32, 128, 200,  32, 128, 204,  32, 128, 208,  32, 128,
+212,  32, 128, 216,  32, 128, 220,  32, 128, 224,  32, 128, 228,  32, 128, 232,
+ 32, 128, 236,  32, 128, 240,  32, 128, 244,  32, 128, 248,  32, 128, 252,  32,
+128, 128,  33, 128, 132,  33, 128, 136,  33, 128, 140,  33, 128, 144,  33, 128,
+148,  33, 128, 152,  33, 128, 156,  33, 128, 160,  33, 128, 164,  33, 128, 168,
+ 33, 128, 172,  33, 128, 176,  33, 128, 180,  33, 128, 184,  33, 128, 188,  33,
+128, 192,  33, 128, 196,  33, 128, 200,  33, 128, 204,  33, 128, 208,  33, 128,
+212,  33, 128, 216,  33, 128, 220,  33, 128, 224,  33, 128, 228,  33, 128, 232,
+ 33, 128, 236,  33, 128, 240,  33, 128, 244,  33, 128, 248,  33, 128, 252,  33,
+128, 128,  34, 128, 132,  34, 128, 136,  34, 128, 140,  34, 128, 144,  34, 128,
+148,  34, 128, 152,  34, 128, 156,  34, 128, 160,  34, 128, 164,  34, 128, 168,
+ 34, 128, 172,  34, 128, 176,  34, 128, 180,  34, 128, 184,  34, 128, 188,  34,
+128, 192,  34, 128, 196,  34, 128, 200,  34, 128, 204,  34, 128, 208,  34, 128,
+212,  34, 128, 216,  34, 128, 220,  34, 128, 224,  34, 128, 228,  34, 128, 232,
+ 34, 128, 236,  34, 128, 240,  34, 128, 244,  34, 128, 248,  34, 128, 252,  34,
+128, 128,  35, 128, 132,  35, 128, 136,  35, 128, 140,  35, 128, 144,  35, 128,
+148,  35, 128, 152,  35, 128, 156,  35, 128, 160,  35, 128, 164,  35, 128, 168,
+ 35, 128, 172,  35, 128, 176,  35, 128, 180,  35, 128, 184,  35, 128, 188,  35,
+128, 192,  35, 128, 196,  35, 128, 200,  35, 128, 204,  35, 128, 208,  35, 128,
+212,  35, 128, 216,  35, 128, 220,  35, 128, 224,  35, 128, 228,  35, 128, 232,
+ 35, 128, 236,  35, 128, 240,  35, 128, 244,  35, 128, 248,  35, 128, 252,  35,
+128, 128, 128,  36, 128, 132,  36, 128, 136,  36, 128, 140,  36, 128, 144,  36,
+128, 148,  36, 128, 152,  36, 128, 156,  36, 128, 160,  36, 128, 164,  36, 128,
+168,  36, 128, 172,  36, 128, 176,  36, 128, 180,  36, 128, 184,  36, 128, 188,
+ 36, 128, 192,  36, 128, 196,  36, 128, 200,  36, 128, 204,  36, 128, 208,  36,
+128, 212,  36, 128, 216,  36, 128, 220,  36, 128, 224,  36, 128, 228,  36, 128,
+232,  36, 128, 236,  36, 128, 240,  36, 128, 244,  36, 128, 248,  36, 128, 252,
+ 36, 128, 128,  37, 128, 132,  37, 128, 136,  37, 128, 140,  37, 128, 144,  37,
+128, 148,  37, 128, 152,  37, 128, 156,  37, 128, 160,  37, 128, 164,  37, 128,
+168,  37, 128, 172,  37, 128, 176,  37, 128, 180,  37, 128, 184,  37, 128, 188,
+ 37, 128, 192,  37, 128, 196,  37, 128, 200,  37, 128, 204,  37, 128, 208,  37,
+128, 212,  37, 128, 216,  37, 128, 220,  37, 128, 224,  37, 128, 228,  37, 128,
+232,  37, 128, 236,  37, 128, 240,  37, 128, 244,  37, 128, 248,  37, 128, 252,
+ 37, 128, 128,  38, 128, 132,  38, 128, 136,  38, 128, 140,  38, 128, 144,  38,
+128, 148,  38, 128, 152,  38, 128, 156,  38, 128, 160,  38, 128, 164,  38, 128,
+168,  38, 128, 172,  38, 128, 176,  38, 128, 180,  38, 128, 184,  38, 128, 188,
+ 38, 128, 192,  38, 128, 196,  38, 128, 200,  38, 128, 204,  38, 128, 208,  38,
+128, 212,  38, 128, 216,  38, 128, 220,  38, 128, 224,  38, 128, 228,  38, 128,
+232,  38, 128, 236,  38, 128, 240,  38, 128, 244,  38, 128, 248,  38, 128, 252,
+ 38, 128, 128,  39, 128, 132,  39, 128, 136,  39, 128, 140,  39, 128, 144,  39,
+128, 148,  39, 128, 152,  39, 128, 156,  39, 128, 160,  39, 128, 164,  39, 128,
+168,  39, 128, 172,  39, 128, 176,  39, 128, 180,  39, 128, 184,  39, 128, 188,
+ 39, 128, 192,  39, 128, 196,  39, 128, 200,  39, 128, 204,  39, 128, 208,  39,
+128, 212,  39, 128, 216,  39, 128, 220,  39, 128, 224,  39, 128, 228,  39, 128,
+232,  39, 128, 236,  39, 128, 240,  39, 128, 244,  39, 128, 248,  39, 128, 252,
+ 39, 128, 128, 128,  40, 128, 132,  40, 128, 136,  40, 128, 140,  40, 128, 144,
+ 40, 128, 148,  40, 128, 152,  40, 128, 156,  40, 128, 160,  40, 128, 164,  40,
+128, 168,  40, 128, 172,  40, 128, 176,  40, 128, 180,  40, 128, 184,  40, 128,
+188,  40, 128, 192,  40, 128, 196,  40, 128, 200,  40, 128, 204,  40, 128, 208,
+ 40, 128, 212,  40, 128, 216,  40, 128, 220,  40, 128, 224,  40, 128, 228,  40,
+128, 232,  40, 128, 236,  40, 128, 240,  40, 128, 244,  40, 128, 248,  40, 128,
+252,  40, 128, 128,  41, 128, 132,  41, 128, 136,  41, 128, 140,  41, 128, 144,
+ 41, 128, 148,  41, 128, 152,  41, 128, 156,  41, 128, 160,  41, 128, 164,  41,
+128, 168,  41, 128, 172,  41, 128, 176,  41, 128, 180,  41, 128, 184,  41, 128,
+188,  41, 128, 192,  41, 128, 196,  41, 128, 200,  41, 128, 204,  41, 128, 208,
+ 41, 128, 212,  41, 128, 216,  41, 128, 220,  41, 128, 224,  41, 128, 228,  41,
+128, 232,  41, 128, 236,  41, 128, 240,  41, 128, 244,  41, 128, 248,  41, 128,
+252,  41, 128, 128,  42, 128, 132,  42, 128, 136,  42, 128, 140,  42, 128, 144,
+ 42, 128, 148,  42, 128, 152,  42, 128, 156,  42, 128, 160,  42, 128, 164,  42,
+128, 168,  42, 128, 172,  42, 128, 176,  42, 128, 180,  42, 128, 184,  42, 128,
+188,  42, 128, 192,  42, 128, 196,  42, 128, 200,  42, 128, 204,  42, 128, 208,
+ 42, 128, 212,  42, 128, 216,  42, 128, 220,  42, 128, 224,  42, 128, 228,  42,
+128, 232,  42, 128, 236,  42, 128, 240,  42, 128, 244,  42, 128, 248,  42, 128,
+252,  42, 128, 128,  43, 128, 132,  43, 128, 136,  43, 128, 140,  43, 128, 144,
+ 43, 128, 148,  43, 128, 152,  43, 128, 156,  43, 128, 160,  43, 128, 164,  43,
+128, 168,  43, 128, 172,  43, 128, 176,  43, 128, 180,  43, 128, 184,  43, 128,
+188,  43, 128, 192,  43, 128, 196,  43, 128, 200,  43, 128, 204,  43, 128, 208,
+ 43, 128, 212,  43, 128, 216,  43, 128, 220,  43, 128, 224,  43, 128, 228,  43,
+128, 232,  43, 128, 236,  43, 128, 240,  43, 128, 244,  43, 128, 248,  43, 128,
+252,  43, 128, 128, 128,  44, 128, 132,  44, 128, 136,  44, 128, 140,  44, 128,
+144,  44, 128, 148,  44, 128, 152,  44, 128, 156,  44, 128, 160,  44, 128, 164,
+ 44, 128, 168,  44, 128, 172,  44, 128, 176,  44, 128, 180,  44, 128, 184,  44,
+128, 188,  44, 128, 192,  44, 128, 196,  44, 128, 200,  44, 128, 204,  44, 128,
+208,  44, 128, 212,  44, 128, 216,  44, 128, 220,  44, 128, 224,  44, 128, 228,
+ 44, 128, 232,  44, 128, 236,  44, 128, 240,  44, 128, 244,  44, 128, 248,  44,
+128, 252,  44, 128, 128,  45, 128, 132,  45, 128, 136,  45, 128, 140,  45, 128,
+144,  45, 128, 148,  45, 128, 152,  45, 128, 156,  45, 128, 160,  45, 128, 164,
+ 45, 128, 168,  45, 128, 172,  45, 128, 176,  45, 128, 180,  45, 128, 184,  45,
+128, 188,  45, 128, 192,  45, 128, 196,  45, 128, 200,  45, 128, 204,  45, 128,
+208,  45, 128, 212,  45, 128, 216,  45, 128, 220,  45, 128, 224,  45, 128, 228,
+ 45, 128, 232,  45, 128, 236,  45, 128, 240,  45, 128, 244,  45, 128, 248,  45,
+128, 252,  45, 128, 128,  46, 128, 132,  46, 128, 136,  46, 128, 140,  46, 128,
+144,  46, 128, 148,  46, 128, 152,  46, 128, 156,  46, 128, 160,  46, 128, 164,
+ 46, 128, 168,  46, 128, 172,  46, 128, 176,  46, 128, 180,  46, 128, 184,  46,
+128, 188,  46, 128, 192,  46, 128, 196,  46, 128, 200,  46, 128, 204,  46, 128,
+208,  46, 128, 212,  46, 128, 216,  46, 128, 220,  46, 128, 224,  46, 128, 228,
+ 46, 128, 232,  46, 128, 236,  46, 128, 240,  46, 128, 244,  46, 128, 248,  46,
+128, 252,  46, 128, 128,  47, 128, 132,  47, 128, 136,  47, 128, 140,  47, 128,
+144,  47, 128, 148,  47, 128, 152,  47, 128, 156,  47, 128, 160,  47, 128, 164,
+ 47, 128, 168,  47, 128, 172,  47, 128, 176,  47, 128, 180,  47, 128, 184,  47,
+128, 188,  47, 128, 192,  47, 128, 196,  47, 128, 200,  47, 128, 204,  47, 128,
+208,  47, 128, 212,  47, 128, 216,  47, 128, 220,  47, 128, 224,  47, 128, 228,
+ 47, 128, 232,  47, 128, 236,  47, 128, 240,  47, 128, 244,  47, 128, 248,  47,
+128, 252,  47, 128, 128, 128,  48, 128, 132,  48, 128, 136,  48, 128, 140,  48,
+128, 144,  48, 128, 148,  48, 128, 152,  48, 128, 156,  48, 128, 160,  48, 128,
+164,  48, 128, 168,  48, 128, 172,  48, 128, 176,  48, 128, 180,  48, 128, 184,
+ 48, 128, 188,  48, 128, 192,  48, 128, 196,  48, 128, 200,  48, 128, 204,  48,
+128, 208,  48, 128, 212,  48, 128, 216,  48, 128, 220,  48, 128, 224,  48, 128,
+228,  48, 128, 232,  48, 128, 236,  48, 128, 240,  48, 128, 244,  48, 128, 248,
+ 48, 128, 252,  48, 128, 128,  49, 128, 132,  49, 128, 136,  49, 128, 140,  49,
+128, 144,  49, 128, 148,  49, 128, 152,  49, 128, 156,  49, 128, 160,  49, 128,
+164,  49, 128, 168,  49, 128, 172,  49, 128, 176,  49, 128, 180,  49, 128, 184,
+ 49, 128, 188,  49, 128, 192,  49, 128, 196,  49, 128, 200,  49, 128, 204,  49,
+128, 208,  49, 128, 212,  49, 128, 216,  49, 128, 220,  49, 128, 224,  49, 128,
+228,  49, 128, 232,  49, 128, 236,  49, 128, 240,  49, 128, 244,  49, 128, 248,
+ 49, 128, 252,  49, 128, 128,  50, 128, 132,  50, 128, 136,  50, 128, 140,  50,
+128, 144,  50, 128, 148,  50, 128, 152,  50, 128, 156,  50, 128, 160,  50, 128,
+164,  50, 128, 168,  50, 128, 172,  50, 128, 176,  50, 128, 180,  50, 128, 184,
+ 50, 128, 188,  50, 128, 192,  50, 128, 196,  50, 128, 200,  50, 128, 204,  50,
+128, 208,  50, 128, 212,  50, 128, 216,  50, 128, 220,  50, 128, 224,  50, 128,
+228,  50, 128, 232,  50, 128, 236,  50, 128, 240,  50, 128, 244,  50, 128, 248,
+ 50, 128, 252,  50, 128, 128,  51, 128, 132,  51, 128, 136,  51, 128, 140,  51,
+128, 144,  51, 128, 148,  51, 128, 152,  51, 128, 156,  51, 128, 160,  51, 128,
+164,  51, 128, 168,  51, 128, 172,  51, 128, 176,  51, 128, 180,  51, 128, 184,
+ 51, 128, 188,  51, 128, 192,  51, 128, 196,  51, 128, 200,  51, 128, 204,  51,
+128, 208,  51, 128, 212,  51, 128, 216,  51, 128, 220,  51, 128, 224,  51, 128,
+228,  51, 128, 232,  51, 128, 236,  51, 128, 240,  51, 128, 244,  51, 128, 248,
+ 51, 128, 252,  51, 128, 128, 128,  52, 128, 132,  52, 128, 136,  52, 128, 140,
+ 52, 128, 144,  52, 128, 148,  52, 128, 152,  52, 128, 156,  52, 128, 160,  52,
+128, 164,  52, 128, 168,  52, 128, 172,  52, 128, 176,  52, 128, 180,  52, 128,
+184,  52, 128, 188,  52, 128, 192,  52, 128, 196,  52, 128, 200,  52, 128, 204,
+ 52, 128, 208,  52, 128, 212,  52, 128, 216,  52, 128, 220,  52, 128, 224,  52,
+128, 228,  52, 128, 232,  52, 128, 236,  52, 128, 240,  52, 128, 244,  52, 128,
+248,  52, 128, 252,  52, 128, 128,  53, 128, 132,  53, 128, 136,  53, 128, 140,
+ 53, 128, 144,  53, 128, 148,  53, 128, 152,  53, 128, 156,  53, 128, 160,  53,
+128, 164,  53, 128, 168,  53, 128, 172,  53, 128, 176,  53, 128, 180,  53, 128,
+184,  53, 128, 188,  53, 128, 192,  53, 128, 196,  53, 128, 200,  53, 128, 204,
+ 53, 128, 208,  53, 128, 212,  53, 128, 216,  53, 128, 220,  53, 128, 224,  53,
+128, 228,  53, 128, 232,  53, 128, 236,  53, 128, 240,  53, 128, 244,  53, 128,
+248,  53, 128, 252,  53, 128, 128,  54, 128, 132,  54, 128, 136,  54, 128, 140,
+ 54, 128, 144,  54, 128, 148,  54, 128, 152,  54, 128, 156,  54, 128, 160,  54,
+128, 164,  54, 128, 168,  54, 128, 172,  54, 128, 176,  54, 128, 180,  54, 128,
+184,  54, 128, 188,  54, 128, 192,  54, 128, 196,  54, 128, 200,  54, 128, 204,
+ 54, 128, 208,  54, 128, 212,  54, 128, 216,  54, 128, 220,  54, 128, 224,  54,
+128, 228,  54, 128, 232,  54, 128, 236,  54, 128, 240,  54, 128, 244,  54, 128,
+248,  54, 128, 252,  54, 128, 128,  55, 128, 132,  55, 128, 136,  55, 128, 140,
+ 55, 128, 144,  55, 128, 148,  55, 128, 152,  55, 128, 156,  55, 128, 160,  55,
+128, 164,  55, 128, 168,  55, 128, 172,  55, 128, 176,  55, 128, 180,  55, 128,
+184,  55, 128, 188,  55, 128, 192,  55, 128, 196,  55, 128, 200,  55, 128, 204,
+ 55, 128, 208,  55, 128, 212,  55, 128, 216,  55, 128, 220,  55, 128, 224,  55,
+128, 228,  55, 128, 232,  55, 128, 236,  55, 128, 240,  55, 128, 244,  55, 128,
+248,  55, 128, 252,  55, 128, 128, 128,  56, 128, 132,  56, 128, 136,  56, 128,
+140,  56, 128, 144,  56, 128, 148,  56, 128, 152,  56, 128, 156,  56, 128, 160,
+ 56, 128, 164,  56, 128, 168,  56, 128, 172,  56, 128, 176,  56, 128, 180,  56,
+128, 184,  56, 128, 188,  56, 128, 192,  56, 128, 196,  56, 128, 200,  56, 128,
+204,  56, 128, 208,  56, 128, 212,  56, 128, 216,  56, 128, 220,  56, 128, 224,
+ 56, 128, 228,  56, 128, 232,  56, 128, 236,  56, 128, 240,  56, 128, 244,  56,
+128, 248,  56, 128, 252,  56, 128, 128,  57, 128, 132,  57, 128, 136,  57, 128,
+140,  57, 128, 144,  57, 128, 148,  57, 128, 152,  57, 128, 156,  57, 128, 160,
+ 57, 128, 164,  57, 128, 168,  57, 128, 172,  57, 128, 176,  57, 128, 180,  57,
+128, 184,  57, 128, 188,  57, 128, 192,  57, 128, 196,  57, 128, 200,  57, 128,
+204,  57, 128, 208,  57, 128, 212,  57, 128, 216,  57, 128, 220,  57, 128, 224,
+ 57, 128, 228,  57, 128, 232,  57, 128, 236,  57, 128, 240,  57, 128, 244,  57,
+128, 248,  57, 128, 252,  57, 128, 128,  58, 128, 132,  58, 128, 136,  58, 128,
+140,  58, 128, 144,  58, 128, 148,  58, 128, 152,  58, 128, 156,  58, 128, 160,
+ 58, 128, 164,  58, 128, 168,  58, 128, 172,  58, 128, 176,  58, 128, 180,  58,
+128, 184,  58, 128, 188,  58, 128, 192,  58, 128, 196,  58, 128, 200,  58, 128,
+204,  58, 128, 208,  58, 128, 212,  58, 128, 216,  58, 128, 220,  58, 128, 224,
+ 58, 128, 228,  58, 128, 232,  58, 128, 236,  58, 128, 240,  58, 128, 244,  58,
+128, 248,  58, 128, 252,  58, 128, 128,  59, 128, 132,  59, 128, 136,  59, 128,
+140,  59, 128, 144,  59, 128, 148,  59, 128, 152,  59, 128, 156,  59, 128, 160,
+ 59, 128, 164,  59, 128, 168,  59, 128, 172,  59, 128, 176,  59, 128, 180,  59,
+128, 184,  59, 128, 188,  59, 128, 192,  59, 128, 196,  59, 128, 200,  59, 128,
+204,  59, 128, 208,  59, 128, 212,  59, 128, 216,  59, 128, 220,  59, 128, 224,
+ 59, 128, 228,  59, 128, 232,  59, 128, 236,  59, 128, 240,  59, 128, 244,  59,
+128, 248,  59, 128, 252,  59, 128, 128, 128,  60, 128, 132,  60, 128, 136,  60,
+128, 140,  60, 128, 144,  60, 128, 148,  60, 128, 152,  60, 128, 156,  60, 128,
+160,  60, 128, 164,  60, 128, 168,  60, 128, 172,  60, 128, 176,  60, 128, 180,
+ 60, 128, 184,  60, 128, 188,  60, 128, 192,  60, 128, 196,  60, 128, 200,  60,
+128, 204,  60, 128, 208,  60, 128, 212,  60, 128, 216,  60, 128, 220,  60, 128,
+224,  60, 128, 228,  60, 128, 232,  60, 128, 236,  60, 128, 240,  60, 128, 244,
+ 60, 128, 248,  60, 128, 252,  60, 128, 128,  61, 128, 132,  61, 128, 136,  61,
+128, 140,  61, 128, 144,  61, 128, 148,  61, 128, 152,  61, 128, 156,  61, 128,
+160,  61, 128, 164,  61, 128, 168,  61, 128, 172,  61, 128, 176,  61, 128, 180,
+ 61, 128, 184,  61, 128, 188,  61, 128, 192,  61, 128, 196,  61, 128, 200,  61,
+128, 204,  61, 128, 208,  61, 128, 212,  61, 128, 216,  61, 128, 220,  61, 128,
+224,  61, 128, 228,  61, 128, 232,  61, 128, 236,  61, 128, 240,  61, 128, 244,
+ 61, 128, 248,  61, 128, 252,  61, 128, 128,  62, 128, 132,  62, 128, 136,  62,
+128, 140,  62, 128, 144,  62, 128, 148,  62, 128, 152,  62, 128, 156,  62, 128,
+160,  62, 128, 164,  62, 128, 168,  62, 128, 172,  62, 128, 176,  62, 128, 180,
+ 62, 128, 184,  62, 128, 188,  62, 128, 192,  62, 128, 196,  62, 128, 200,  62,
+128, 204,  62, 128, 208,  62, 128, 212,  62, 128, 216,  62, 128, 220,  62, 128,
+224,  62, 128, 228,  62, 128, 232,  62, 128, 236,  62, 128, 240,  62, 128, 244,
+ 62, 128, 248,  62, 128, 252,  62, 128, 128,  63, 128, 132,  63, 128, 136,  63,
+128, 140,  63, 128, 144,  63, 128, 148,  63, 128, 152,  63, 128, 156,  63, 128,
+160,  63, 128, 164,  63, 128, 168,  63, 128, 172,  63, 128, 176,  63, 128, 180,
+ 63, 128, 184,  63, 128, 188,  63, 128, 192,  63, 128, 196,  63, 128, 200,  63,
+128, 204,  63, 128, 208,  63, 128, 212,  63, 128, 216,  63, 128, 220,  63, 128,
+224,  63, 128, 228,  63, 128, 232,  63, 128, 236,  63, 128, 240,  63, 128, 244,
+63, 128, 248,  63, 128, 252,  63};
+  SeekableInputStream* const stream =
+    new SeekableArrayInputStream(buffer, ARRAY_SIZE(buffer));
+  std::unique_ptr<RleDecoder> rle =
+      createRleDecoder(std::unique_ptr<SeekableInputStream>(stream),
+                       true, RleVersion_1, *getDefaultPool());
+  std::vector<int64_t> data(1);
+  for (size_t i = 0; i < 2048; i += 10) {
+    rle->next(data.data(), 1, nullptr);
+    if (i < 1024) {
+      EXPECT_EQ(i, data[0]) << "Wrong output at " << i;
+    } else {
+      EXPECT_EQ(256 * i, data[0]) << "Wrong output at " << i;
+    }
+    if (i < 2038) {
+      rle->skip(9);
+    }
+    rle->skip(0);
+  }
+}
+
+TEST(RLEv1, seekTest) {
+  // Create the RLE stream from Java's
+  // TestRunLengthIntegerEncoding.testUncompressedSeek
+  // for (size_t i = 0; i < 1024; ++i)
+  //   out.write(i / 4);
+  // for (size_t i = 1024; i < 2048; ++i)
+  //   out.write(2 * i);
+  // for (size_t i = 0; i < 2048; ++i)
+  //   out.write(junk[i]);
+  // This causes the first half to be delta encoded and the second half to
+  // be literal encoded.
+  const unsigned char buffer[] = {
+  1,   0,   0,   1,   0,   2,   1,   0,   4,   1,   0,   6,   1,   0,   8,   1,
+  0,  10,   1,   0,  12,   1,   0,  14,   1,   0,  16,   1,   0,  18,   1,   0,
+ 20,   1,   0,  22,   1,   0,  24,   1,   0,  26,   1,   0,  28,   1,   0,  30,
+  1,   0,  32,   1,   0,  34,   1,   0,  36,   1,   0,  38,   1,   0,  40,   1,
+  0,  42,   1,   0,  44,   1,   0,  46,   1,   0,  48,   1,   0,  50,   1,   0,
+ 52,   1,   0,  54,   1,   0,  56,   1,   0,  58,   1,   0,  60,   1,   0,  62,
+  1,   0,  64,   1,   0,  66,   1,   0,  68,   1,   0,  70,   1,   0,  72,   1,
+  0,  74,   1,   0,  76,   1,   0,  78,   1,   0,  80,   1,   0,  82,   1,   0,
+ 84,   1,   0,  86,   1,   0,  88,   1,   0,  90,   1,   0,  92,   1,   0,  94,
+  1,   0,  96,   1,   0,  98,   1,   0, 100,   1,   0, 102,   1,   0, 104,   1,
+  0, 106,   1,   0, 108,   1,   0, 110,   1,   0, 112,   1,   0, 114,   1,   0,
+116,   1,   0, 118,   1,   0, 120,   1,   0, 122,   1,   0, 124,   1,   0, 126,
+  1,   0, 128,   1,   1,   0, 130,   1,   1,   0, 132,   1,   1,   0, 134,   1,
+  1,   0, 136,   1,   1,   0, 138,   1,   1,   0, 140,   1,   1,   0, 142,   1,
+  1,   0, 144,   1,   1,   0, 146,   1,   1,   0, 148,   1,   1,   0, 150,   1,
+  1,   0, 152,   1,   1,   0, 154,   1,   1,   0, 156,   1,   1,   0, 158,   1,
+  1,   0, 160,   1,   1,   0, 162,   1,   1,   0, 164,   1,   1,   0, 166,   1,
+  1,   0, 168,   1,   1,   0, 170,   1,   1,   0, 172,   1,   1,   0, 174,   1,
+  1,   0, 176,   1,   1,   0, 178,   1,   1,   0, 180,   1,   1,   0, 182,   1,
+  1,   0, 184,   1,   1,   0, 186,   1,   1,   0, 188,   1,   1,   0, 190,   1,
+  1,   0, 192,   1,   1,   0, 194,   1,   1,   0, 196,   1,   1,   0, 198,   1,
+  1,   0, 200,   1,   1,   0, 202,   1,   1,   0, 204,   1,   1,   0, 206,   1,
+  1,   0, 208,   1,   1,   0, 210,   1,   1,   0, 212,   1,   1,   0, 214,   1,
+  1,   0, 216,   1,   1,   0, 218,   1,   1,   0, 220,   1,   1,   0, 222,   1,
+  1,   0, 224,   1,   1,   0, 226,   1,   1,   0, 228,   1,   1,   0, 230,   1,
+  1,   0, 232,   1,   1,   0, 234,   1,   1,   0, 236,   1,   1,   0, 238,   1,
+  1,   0, 240,   1,   1,   0, 242,   1,   1,   0, 244,   1,   1,   0, 246,   1,
+  1,   0, 248,   1,   1,   0, 250,   1,   1,   0, 252,   1,   1,   0, 254,   1,
+  1,   0, 128,   2,   1,   0, 130,   2,   1,   0, 132,   2,   1,   0, 134,   2,
+  1,   0, 136,   2,   1,   0, 138,   2,   1,   0, 140,   2,   1,   0, 142,   2,
+  1,   0, 144,   2,   1,   0, 146,   2,   1,   0, 148,   2,   1,   0, 150,   2,
+  1,   0, 152,   2,   1,   0, 154,   2,   1,   0, 156,   2,   1,   0, 158,   2,
+  1,   0, 160,   2,   1,   0, 162,   2,   1,   0, 164,   2,   1,   0, 166,   2,
+  1,   0, 168,   2,   1,   0, 170,   2,   1,   0, 172,   2,   1,   0, 174,   2,
+  1,   0, 176,   2,   1,   0, 178,   2,   1,   0, 180,   2,   1,   0, 182,   2,
+  1,   0, 184,   2,   1,   0, 186,   2,   1,   0, 188,   2,   1,   0, 190,   2,
+  1,   0, 192,   2,   1,   0, 194,   2,   1,   0, 196,   2,   1,   0, 198,   2,
+  1,   0, 200,   2,   1,   0, 202,   2,   1,   0, 204,   2,   1,   0, 206,   2,
+  1,   0, 208,   2,   1,   0, 210,   2,   1,   0, 212,   2,   1,   0, 214,   2,
+  1,   0, 216,   2,   1,   0, 218,   2,   1,   0, 220,   2,   1,   0, 222,   2,
+  1,   0, 224,   2,   1,   0, 226,   2,   1,   0, 228,   2,   1,   0, 230,   2,
+  1,   0, 232,   2,   1,   0, 234,   2,   1,   0, 236,   2,   1,   0, 238,   2,
+  1,   0, 240,   2,   1,   0, 242,   2,   1,   0, 244,   2,   1,   0, 246,   2,
+  1,   0, 248,   2,   1,   0, 250,   2,   1,   0, 252,   2,   1,   0, 254,   2,
+  1,   0, 128,   3,   1,   0, 130,   3,   1,   0, 132,   3,   1,   0, 134,   3,
+  1,   0, 136,   3,   1,   0, 138,   3,   1,   0, 140,   3,   1,   0, 142,   3,
+  1,   0, 144,   3,   1,   0, 146,   3,   1,   0, 148,   3,   1,   0, 150,   3,
+  1,   0, 152,   3,   1,   0, 154,   3,   1,   0, 156,   3,   1,   0, 158,   3,
+  1,   0, 160,   3,   1,   0, 162,   3,   1,   0, 164,   3,   1,   0, 166,   3,
+  1,   0, 168,   3,   1,   0, 170,   3,   1,   0, 172,   3,   1,   0, 174,   3,
+  1,   0, 176,   3,   1,   0, 178,   3,   1,   0, 180,   3,   1,   0, 182,   3,
+  1,   0, 184,   3,   1,   0, 186,   3,   1,   0, 188,   3,   1,   0, 190,   3,
+  1,   0, 192,   3,   1,   0, 194,   3,   1,   0, 196,   3,   1,   0, 198,   3,
+  1,   0, 200,   3,   1,   0, 202,   3,   1,   0, 204,   3,   1,   0, 206,   3,
+  1,   0, 208,   3,   1,   0, 210,   3,   1,   0, 212,   3,   1,   0, 214,   3,
+  1,   0, 216,   3,   1,   0, 218,   3,   1,   0, 220,   3,   1,   0, 222,   3,
+  1,   0, 224,   3,   1,   0, 226,   3,   1,   0, 228,   3,   1,   0, 230,   3,
+  1,   0, 232,   3,   1,   0, 234,   3,   1,   0, 236,   3,   1,   0, 238,   3,
+  1,   0, 240,   3,   1,   0, 242,   3,   1,   0, 244,   3,   1,   0, 246,   3,
+  1,   0, 248,   3,   1,   0, 250,   3,   1,   0, 252,   3,   1,   0, 254,   3,
+127,   2, 128,  32, 127,   2, 136,  36, 127,   2, 144,  40, 127,   2, 152,  44,
+127,   2, 160,  48, 127,   2, 168,  52, 127,   2, 176,  56, 111,   2, 184,  60,
+128, 147, 150, 232, 240,   8, 168, 134, 179, 187,  12, 246, 145, 173, 142,  11,
+241, 162, 190, 162,   9, 239, 218, 128, 243,   5, 202, 175, 131, 196,  12, 151,
+253, 204, 160,   4, 229, 167, 247, 255,  12, 255, 177, 140, 184,   7, 188, 145,
+181, 229,   1, 178, 190, 158, 163,   8, 147, 179, 151, 132,   8, 150, 133, 222,
+129,  11, 193, 218, 187, 242,  14, 181, 177, 154, 155,   9, 150, 145, 194, 135,
+  8, 186, 222, 142, 242,  10, 140, 195, 254, 237,  11, 141, 189, 143, 198,  14,
+229, 146, 237, 203,   8, 251, 162, 179, 211,   3, 222, 237, 175, 145,  13, 221,
+178, 163, 162,   3, 211, 192, 165, 189,  14, 230, 228, 168, 250,   4, 141, 140,
+247, 178,   7, 143, 164, 170, 152,   2, 131, 166, 136,  26, 171, 143, 232, 134,
+ 12, 158, 239, 246, 204,  11, 133, 128, 213, 223,  14, 255, 213, 190, 250,  15,
+143, 162, 252, 157,   4, 204, 181, 135, 245,   7, 206, 241, 254, 136,   4, 184,
+182, 211, 190,  15, 172, 156, 202, 135,  10, 249, 180, 139, 131,   4, 202, 128,
+204, 221,   9, 131, 247, 166, 249,   8, 141, 236, 241, 185,   3, 128, 229, 150,
+186,   2, 237, 189, 141, 218,   9, 193, 240, 241, 156,   3, 210, 142, 198, 202,
+ 10, 227, 241, 194, 234,   7, 145, 180, 228, 254,   6, 171, 249, 185, 188,  11,
+215, 135, 224, 219,   4, 133, 132, 178, 165,   7, 205, 180, 133, 209,  11, 198,
+253, 246, 145,  12, 190, 194, 153, 146,   8, 139, 220, 235, 249,   1, 170, 203,
+205, 159,   6, 136, 130, 154, 166,  14, 250, 189, 153, 191,   7, 178, 163, 191,
+158,  12, 251, 138, 135, 245,  10, 175, 249, 219, 164,  14, 136, 185, 220, 188,
+  7, 170, 135, 221, 146,   7, 209, 224, 204, 171,  11, 216, 144, 236, 172,   1,
+133, 205, 202, 170,   6, 215, 250, 133, 181,   3, 181, 133, 142, 158,   5, 166,
+192, 134, 238,  13, 246, 243, 233, 218,  12, 163, 202, 238, 241,  14, 241, 214,
+224, 215,   2, 212, 192, 237, 243,  10, 163, 165, 163, 206,   6, 159, 161, 227,
+152,  14, 209, 234, 225, 249,  13, 167, 206, 188, 161,   3, 143, 209, 188, 214,
+ 11, 184, 224, 210, 200,  10, 185, 171, 199, 183,   3, 177, 229, 245,  86, 255,
+183, 178, 142,   9, 232, 209, 135, 151,   8, 191, 153, 174, 175,   7, 190, 245,
+224, 174,   9, 243, 165, 145, 169,   1, 145, 161, 221, 249,  13, 195, 221, 244,
+240,   5, 157, 156, 217, 237,  15, 143, 201, 155, 207,   5, 169, 136, 192, 238,
+ 12, 135, 223, 244, 200,   2, 137, 228, 167, 187,   1, 134, 212, 158, 155,  15,
+186, 224, 212, 214,   7, 193, 141, 216, 241,   2, 246, 159, 138, 117, 216, 230,
+215,  29, 204, 178, 147, 255,   8, 195, 140, 136, 164,  11, 234, 204, 155, 222,
+ 10, 193, 156, 138, 187,   8, 161, 161, 184, 212,   1, 128, 141, 162, 133,  13,
+180, 211, 132, 210,   9, 239, 203, 201, 177,   5, 236, 191, 140, 207,  13, 173,
+205, 192, 186,   7, 179, 214, 222, 136,   8, 189, 142, 204, 152,   5, 221, 176,
+135, 241,   1, 223, 146, 195, 166,  11, 146, 133, 226, 137,   6, 150, 243, 247,
+  1, 153, 246, 184,  42, 234, 194, 229,  98, 237, 144, 253, 133,  11, 196, 131,
+158, 244,   6, 218, 149, 253, 221,   7, 219, 180, 234, 156,  10, 179, 255, 197,
+218,  13, 150, 137, 240, 204,   9, 240, 185, 181, 203,   2, 160, 194, 146, 246,
+  5, 131, 168, 191, 138,   4, 158, 245, 240, 150,  15, 157, 202, 136,  14, 135,
+154, 226, 240,   5, 153, 168, 212, 222,   8, 128, 218, 198, 244, 133,  13, 183,
+245, 153, 118, 139, 141, 238, 141,   1, 235, 193, 197,   5, 169, 141, 210,  62,
+231, 186, 238, 219,   6, 141, 243, 204, 242,  12, 172, 165, 150, 187,  13, 163,
+254, 250, 230,  12, 203, 166, 166, 223,   3, 177, 155, 168, 182,   4, 213, 130,
+148, 221,   3, 150, 178, 146, 235,   6, 149, 226, 237, 225,   2, 177, 149, 218,
+ 10, 205, 241, 161,  21, 186, 239, 197, 189,  15, 132, 249, 249, 171,   5, 130,
+223, 220, 167,   5, 171, 235, 129,  84, 207, 145, 246, 231,   2, 183, 176, 230,
+148,  11, 180, 142, 254, 128,   1, 171, 251, 177, 177,   1, 188, 190, 157, 222,
+ 11, 140, 195, 192, 141,  10, 200, 139, 160, 247,   9, 139, 247, 194, 144,   1,
+160, 160, 234, 208,  11, 174, 210, 150, 196,  15, 209, 201, 176, 208,  14, 199,
+183, 218, 132,   8, 175, 143, 188, 168,   7, 172, 234, 158, 248,  11, 192, 223,
+160, 152,   7, 178, 134, 130, 235,   3, 243, 134, 181, 181,   4, 225, 135, 251,
+236,   7, 203, 166, 149, 169,  10, 181, 213, 156, 193,  12, 239, 138, 235, 252,
+  2, 183, 243, 201, 133,  10, 137, 186, 227, 237,  13, 255, 188, 221, 148,  14,
+188, 156, 198, 143,  15, 223, 224, 252, 208,   9, 160, 241, 190, 221,  13, 195,
+241, 163, 241,   9, 199, 253, 138, 163,  12, 173, 251, 143, 133,  12, 167, 246,
+153, 247,  14, 237, 223, 140, 174,  14, 219, 229, 138, 242,   2, 200, 163, 210,
+ 86, 197, 251, 199, 241,   9, 243, 211, 209, 132,   3, 178, 176, 152, 224,  13,
+195, 131, 248, 159,   5, 194, 255, 160, 171,  14, 145, 243, 143, 173,   3, 222,
+168, 246, 134,   2, 178, 145, 204, 240,   1, 176, 240, 236, 165,  14, 254, 145,
+162, 165,   8, 243, 173, 131, 238,   3, 247, 192, 235, 163,   4, 244, 239, 180,
+203,  15, 214, 167, 152, 233,  13, 176, 158, 206, 235,   9, 252, 150, 228, 160,
+ 13, 148, 243, 234, 239,   2, 225, 152, 250, 167,   5, 252, 143, 229, 254,   4,
+184, 202, 161, 157,  14, 233, 190, 185, 195,   9, 159, 223, 240, 216,  11, 132,
+172, 243, 200,   6, 212, 182, 191, 194,  13, 230, 245, 240, 130,  12, 189, 146,
+233, 239,   2, 155, 190, 214, 183,  15, 159, 222, 148, 155,  13, 195, 158, 248,
+112, 224, 219, 145, 234,  12, 145, 169, 172, 135,  10, 234, 184, 245, 220,   4,
+138, 150, 232, 212,   5, 132, 195, 135, 214,   5, 181, 247, 216, 205,  12, 239,
+160, 183, 178,   9, 161, 143, 210, 206,  11, 248, 209, 207,  94, 166, 178, 165,
+ 97, 133, 162, 246, 212,   9, 206, 240, 235, 156,   1, 200, 228, 176, 252,  12,
+163, 215, 219, 141,   1, 236, 133, 216, 202,   9, 220, 170, 222, 242,  10, 239,
+203, 197, 220,  11, 148, 218, 209, 161,   7, 185, 175, 210, 171,  15, 153, 213,
+208, 214,  15, 188, 239, 128, 244,  13, 141, 220, 136, 166,  12, 150, 148, 250,
+175,  13, 130, 145, 226, 216,   1, 216, 204, 215, 193,   9, 191, 211, 181, 229,
+ 14, 233, 168, 165,   9, 240, 188, 146, 132,  12, 173, 220, 201, 244,   4, 140,
+147, 190, 199,  15, 190, 213, 175, 213,   1, 254, 212, 239, 171,  10, 200, 161,
+168, 144,  10, 161, 188, 230, 163,   6, 192, 198, 213, 167,   3, 240, 251, 180,
+243,   5, 202, 165, 247, 147,   7, 173, 191, 133, 228,   3, 229, 139, 154, 210,
+  7, 147, 254, 164, 236,  13, 162, 214, 180, 128,   8, 202, 176, 252, 143,  13,
+154, 179, 169, 149,   3, 169, 156, 168, 229,   1, 164, 128, 214, 138,  15, 128,
+239, 253, 160, 181,   2, 232, 203, 196, 235,  11, 181, 153, 131, 240,  12, 145,
+178, 179, 206,  12, 134, 244, 215, 141,  10, 138, 228, 171, 244,   7, 246, 160,
+221, 177,  14, 176, 231, 208, 135,   9, 194, 210, 159, 234,   2, 238, 250, 139,
+146,  10, 249, 191, 224, 241,  10, 250, 140, 140, 147,   5, 190, 185, 216, 220,
+ 15, 248, 131, 153, 236,   9, 140, 219, 183, 252,  14, 254, 184, 223, 216,  14,
+253, 211, 235, 254,  14, 252, 180, 147, 152,   9, 147, 221, 188, 174,   1, 222,
+219, 180, 185,  12, 185, 175, 244, 136,   9, 214, 147, 217, 182,   4, 191, 193,
+233, 157,   2, 238, 191, 156, 211,  14, 229, 221, 129, 224,   2, 230, 212, 248,
+128,   3, 186, 165, 136,  84, 129, 216, 148, 139,  15, 150, 231, 196, 184,   8,
+160, 156, 253, 171,   2, 156, 198, 161, 183,  11, 164, 181, 155, 137,   8, 133,
+196, 192, 213,   6, 140, 174, 143, 152,  12, 142, 202, 143, 192,   9, 128, 167,
+234, 152,  13, 214, 131, 156, 246,  14, 167, 223, 250, 135,   4, 233, 185, 236,
+128,   1, 138, 131, 251, 181,   9, 184, 141, 213, 136,  15, 171, 224, 222, 192,
+ 12, 244, 168, 162, 144,   1, 212, 183, 184, 200,   9, 177, 193, 168, 174,  14,
+249, 175, 129, 197,   1, 142, 181, 130, 162,  10, 214, 197, 196, 214,   4, 148,
+146, 228, 202,  13, 213, 154, 241, 127, 165, 166, 144, 164,   4, 205, 251, 139,
+128,  13, 244, 188, 143, 236,  12, 190, 247, 138, 217,   8, 185, 201, 217, 187,
+  4, 130, 142, 167, 137,   4, 139, 185, 215,  95, 136, 170, 224, 218,   9, 154,
+158, 177, 200,  15, 227, 154, 189, 136,  15, 224, 233, 220, 179,   3, 227, 203,
+160, 188,   7, 236, 228, 239, 162,  15, 214, 227, 159, 242,   4, 151, 252, 232,
+ 42, 151, 166, 168, 245,   3, 135, 180, 250, 243,  15, 167, 254, 137, 160,  13,
+214, 240, 225, 152,   8, 190, 229, 204, 136,  13, 150, 219, 186,  10, 163, 249,
+225, 249,   6, 215, 233, 254, 162,   9, 171, 204, 237, 189,   5, 229, 137, 174,
+157,   6, 135, 205, 140, 164,  10, 189, 136, 130, 244,   1, 210, 222, 223, 247,
+  1, 189, 128, 142, 203,  12, 232, 241, 180, 195,  12, 237, 228, 243, 183,   7,
+218, 155, 204, 158,  14, 235, 167, 134, 183,   6, 171, 218, 141, 128,   3, 184,
+152, 251, 187,  10, 138, 217, 169, 182,   2, 210, 140, 240, 138,   7, 150, 156,
+232, 128,   9, 209, 231, 181, 174,  14, 243, 210, 173,  34, 220, 254, 188, 199,
+ 14, 245, 195, 226, 124, 141, 228, 248, 228,  15, 158, 166, 194, 150,   6, 152,
+220, 238, 252,  13, 179, 132, 217, 220,  15, 213, 168, 186, 245,   4, 241, 243,
+200, 226,  10, 216, 178, 141, 137,  13, 134, 176, 169, 179,   6, 212, 242, 197,
+ 75, 175, 222, 238, 237,  10, 185, 143, 171, 166,   6, 180, 198, 129, 170,   5,
+159, 129, 176, 134,  11, 130, 248, 213, 183,  12, 204, 162, 169, 238,   8, 139,
+139, 145, 227,  15, 232, 239, 206, 163,   3, 145, 157, 143, 183,  10, 250, 190,
+179, 189,   3, 185, 138, 211, 215,   3, 179, 147, 158, 165,  13, 231, 226, 199,
+245,  11, 147, 179, 178, 190,   1, 208, 217, 154, 195,  14, 226, 194, 229, 142,
+  8, 198, 175, 184, 231,   4, 199, 198, 191,  24, 184, 134, 226, 231,  10, 152,
+208, 222, 254,   1, 134, 167, 234,  69, 175, 214, 177, 218,   3, 218, 234, 128,
+162,   3, 160, 177, 187, 166,   3, 201, 210, 191, 159,  13, 240, 152, 160, 250,
+  6, 235, 130, 214, 240,  11, 128, 237, 251, 245, 225,   3, 245, 237, 174, 230,
+  9, 252, 148, 229, 201,   7, 152, 148, 165, 153,   7, 223, 238, 242,  16, 156,
+212, 237, 228,   7, 139, 153, 178,  37, 219, 217, 217, 172,  15, 178, 168, 128,
+199,   9, 236, 189, 144, 226,  12, 214, 248, 134, 230,  13, 163, 252, 247,  55,
+239, 252, 149, 196,   3, 230, 159, 214, 139,   6, 132, 200, 241, 154,   2, 129,
+231, 153, 173,  12, 235, 131, 255, 157,   2, 246, 190, 145,  55, 205, 201, 240,
+141,   9, 188, 202, 199, 189,   6, 196, 235, 245, 205,  11, 249, 253, 241, 223,
+  6, 187, 250, 137, 241,   9, 133, 135, 168, 146,   8, 132, 248, 219, 156,   8,
+132, 241, 185,   4, 198, 209, 147, 129,  11, 229, 192, 218, 178,   4, 199, 210,
+138, 166,  13, 244, 148, 172, 141,   2, 194, 215, 171, 220,   1, 192, 248, 230,
+128,   2, 238, 167, 209, 222,  11, 240, 200, 227, 150,  11, 182, 217, 170, 158,
+ 14, 223, 223, 254, 201,  10, 140, 164, 245, 175,   2, 178, 140, 153, 102, 139,
+145, 181, 242,   8, 188, 154, 214, 154,  15, 149, 187, 204, 192,   2, 223, 153,
+219,  51, 245, 236, 130, 133,   5, 197, 138, 169,  80, 243, 162, 164, 167,   1,
+206, 232, 180, 137,  12, 180, 191, 164, 226,   8, 162, 180, 231, 222,  13, 184,
+143, 156,  74, 134, 230, 248, 219,  10, 203, 156, 149, 205,   1, 219, 205, 173,
+167,  10, 174, 146, 180, 141,   7, 214, 231, 229, 231,  10, 181, 246, 174, 180,
+ 15, 236, 175, 222, 241,   7, 191, 150, 253, 209,   8, 233, 139, 167, 149,  13,
+142, 249, 150, 223,  10, 220, 151, 135, 222,   5, 138, 228, 133, 131,   4, 232,
+183, 160, 245,   3, 157, 219, 209, 200,   5, 159, 242, 142, 148,  13, 241, 207,
+248, 177,  11, 179, 226, 169, 150,  13, 169, 201, 212, 218,   8, 172, 214, 220,
+ 31, 155, 173, 251, 231,  12, 221, 150, 137, 174,  15, 146, 137, 251, 255,  14,
+245, 216, 203, 138,   1, 163, 170, 194, 133,  12, 205, 157, 188, 131,  12, 184,
+220, 161,  97, 162, 240, 190, 243,   2, 213, 134, 147, 251,   3, 178, 160, 193,
+188,  14, 214, 153, 226, 140,  12, 191, 208, 235, 174,  13, 138, 188, 204, 236,
+ 11, 214, 135, 129, 235,  10, 198, 242, 226, 128,  11, 154, 219, 163, 144,   7,
+236, 134, 217, 197,   2, 181, 248, 144, 157,   8, 150, 174, 195, 224,  12, 156,
+247, 234, 192,   7, 156, 206, 174, 246,   2, 181, 214, 138, 155,   1, 246, 242,
+141, 152,   9, 207, 157, 139, 243,   1, 153, 135, 158, 249,   6, 162, 129, 144,
+170,  13, 227, 162, 245, 246,   1, 130, 237, 192, 208,  13, 187, 165, 153, 215,
+  8, 178, 141, 203, 163,  15, 172, 179, 180, 172,  10, 206, 200, 237, 194,  12,
+129, 235, 165, 143,   7, 129, 230, 217, 244,   8, 223, 249, 152, 233,   2, 160,
+224, 204, 187,  10, 167, 211, 138, 247,   7, 207, 204, 131, 200,   1, 207, 240,
+161, 219,   9, 219, 213, 129, 183,  11, 186, 163, 243, 198,  13, 217, 197, 175,
+218,   8, 195, 228, 209, 137,   1, 149, 253, 193, 190,   8, 216, 231, 225, 190,
+ 15, 244, 168, 191, 152,   6, 180, 210, 162, 198,   9, 172, 159, 195, 158,   9,
+173, 151, 226,  34, 143, 231, 162, 212,   6, 250, 171, 192, 187,  11, 229, 212,
+155, 156,   9, 234, 159, 165, 254,   8, 180, 154, 227, 197,   3, 175, 158, 214,
+235,   8, 164, 157, 160, 130,   4, 158, 223, 243, 254,  10, 178, 236, 213, 212,
+ 12, 194, 173, 185, 159,   6, 184, 214, 195, 172,   5, 128, 161, 203, 183, 194,
+ 10, 207, 218, 209, 222,  12, 136, 166, 226, 224,   3, 148, 153, 145, 214,   4,
+164, 178, 253, 243,   4, 173, 162, 237, 129,   4, 236, 134, 193, 169,  14, 140,
+234, 164, 190,   7, 211, 148, 252, 223,   8, 213, 149, 180, 170,  12, 194, 182,
+191, 205,  15, 206, 233, 190, 211,   2, 241, 136, 223, 152,  12, 184, 185, 231,
+176,  10, 201, 166, 182, 211,   4, 209, 201, 205, 235,   1, 141, 184, 205, 173,
+ 15, 244, 222, 218, 113, 175, 190, 179, 140,   4, 234, 232, 231, 183,   8, 174,
+167, 140, 130,   9, 169, 157, 136, 196,  14, 187, 244, 242, 135,   7, 248, 183,
+178, 253,  10, 135, 216, 152, 153,  15, 226, 223, 172, 161,  11, 236, 183, 231,
+216,   3, 183, 169, 209, 137,  13, 130, 219, 233, 167,   4, 168, 132, 197, 161,
+  7, 164, 146, 152, 207,   4, 239, 229, 147, 130,   2, 172, 156, 244, 148,   6,
+171, 253, 185, 213,   4, 184, 181, 241, 207,   1, 144, 250, 219, 222,   1, 213,
+189, 209, 177,  10, 207, 252, 251, 239,   9, 181, 132, 203, 147,   6, 159, 135,
+181,  18, 215, 252, 202, 234,   7, 207, 215, 210, 222,  12, 195, 211, 185, 171,
+ 14, 178, 132, 165, 140,   9, 139, 160, 171, 250,   1, 248, 176, 203, 170,  14,
+148, 184, 131, 141,   4, 158, 226, 204, 197,   3, 215, 157, 148, 219,  15, 228,
+206, 156, 132,   3, 234, 206, 202, 231,   8, 232, 177, 135, 215,  10, 173, 253,
+176, 172,   5, 144, 188, 170, 229,  14, 200, 165, 144,  50, 198, 153, 206, 184,
+  3, 150, 128, 128, 141,  14, 155, 221, 221, 199,  12, 229, 199, 160, 156,   3,
+176, 172, 200,  97, 222, 255, 134, 158,   9, 233, 155, 199, 193,  14, 146, 216,
+186, 250,  13, 156, 152, 194, 212,   8, 254, 190, 240, 232,   2, 178, 210, 194,
+160,   3, 142, 216, 141, 184,  10, 173, 210, 214, 187,   2, 161, 211, 201, 143,
+  5, 213, 149, 210, 222,  15, 134, 165, 184, 171,   9, 211, 175, 153, 241,   9,
+227, 201, 184, 213,   1, 173, 225, 213, 176,  13, 143, 228, 200, 151,  12, 224,
+224, 224, 186,   8, 188, 153, 234, 254,   7, 137, 188, 238, 186,   8, 166, 236,
+135, 180,  13, 202, 174, 133, 194,  13, 179, 243, 158, 193,  13, 210, 173, 128,
+149,   2, 208, 216, 158, 168,  13, 205, 251, 152, 230,   3, 245, 245, 254, 163,
+  9, 211, 243, 234, 164,   9, 173, 221, 221, 215,   4, 146, 220, 209, 198,   1,
+235, 237, 170, 130,   7, 181, 227, 149, 141,   2, 170, 245, 149, 217,   5, 153,
+179, 215, 195,  14, 249, 206, 140, 148,   1, 247, 200, 219, 152,  15, 165, 228,
+197, 152,  11, 234, 192, 242, 244,   6, 217, 229, 173, 147,   3, 216, 209, 206,
+189,   7, 165, 171, 221, 214,   2, 151, 250, 211, 138,   2, 144, 169, 182, 176,
+ 13, 179, 254, 191, 225,   3, 244, 147, 218, 212,   3, 129, 187, 183, 253,  10,
+218, 149, 188, 168,  10, 223, 241, 149, 129,   8, 209, 128, 150, 126, 153, 139,
+195, 131,   6, 201, 208, 246, 221,   1, 194, 165, 175, 173,   5, 197, 133, 207,
+196,   2, 192, 211, 129, 210,   7, 211, 147, 163, 220,   9, 173, 191, 188, 152,
+  1, 169, 242, 205,  20, 167, 133, 213, 211,   2, 213, 226, 129, 166,  12, 186,
+202, 155, 203,   5, 180, 251, 220, 174,  12, 145, 228, 247, 146,  12, 196, 151,
+247, 184,  10, 217, 233, 238, 147,   6, 149, 174, 181, 128,  13, 128, 246, 173,
+207,  15, 200, 162, 139, 103, 237, 199, 220, 252,   7, 208, 201, 133, 231,   3,
+140, 148, 223, 137,   5, 128, 242, 251, 140, 228,  11, 214, 205, 158, 228,   2,
+147, 190, 212, 138,   4, 228, 228, 253, 154,   9, 146, 191, 248, 187,   8, 168,
+200, 246, 160,   4, 224, 168, 147, 211,  11, 153, 197, 133, 229,   5, 176, 131,
+167, 203,   6, 213, 183, 189, 178,  10, 185, 222, 229, 183,   5, 171, 185, 208,
+162,  15, 203, 130, 137, 201,   6, 236, 152, 138, 176,   1, 221, 200, 169, 183,
+ 11, 237, 230, 219, 108, 152, 247, 239, 145,  14, 242, 220, 245, 148,   6, 183,
+147, 218, 144,  11, 236, 190, 230, 197,   1, 253, 147, 205, 165,  10, 181, 130,
+138, 249,  10, 193, 135, 148, 142,  10, 232, 132, 254, 163,   4, 244, 153, 241,
+197,  13, 251, 150, 230, 242,  10, 211, 255, 182, 243,   3, 247, 137, 150, 236,
+  5, 137, 168, 208, 161,  10, 192, 178, 137, 210,  13, 192, 158, 177, 203,   7,
+237, 221, 208, 153,   4, 180, 129, 195, 139,   4, 195, 220, 254, 129,   8, 235,
+249, 252, 142,   2, 171, 195, 208, 162,  12, 205, 185, 192, 166,   9, 208, 205,
+169, 160,  10, 156, 148, 150, 185,   2, 246, 165, 207, 129,  12, 145, 207, 129,
+130,  15, 253, 209, 184, 133,  11, 247, 226, 200, 185,   9, 193, 147, 150, 128,
+  8, 251, 208, 155,  45, 251, 142, 248, 144,  15, 174, 199, 157, 236,  12, 206,
+215, 156, 131,  14, 224, 242, 193, 145,   9, 194, 231, 136, 243,   7, 135, 188,
+221, 220,  10, 252, 138, 172, 180,  15, 222, 245, 235, 161,   2, 147, 195, 191,
+195,   7, 191, 205, 163, 247,   3, 237, 172, 239, 187,   6, 137, 141, 231, 233,
+ 10, 246, 253, 140, 184,   5, 191, 252, 199, 190,  13, 235, 212, 206, 220,   8,
+163, 219, 233, 232,  13, 166, 129, 242, 168,  12, 131, 217, 184, 209,   7, 138,
+139, 223, 216,   8, 186, 152, 149, 207,   6, 229, 191, 144, 149,   8, 223, 167,
+204, 251,   1, 181, 240, 166, 200,   9, 194, 230, 150, 122, 210, 176, 221, 179,
+  5, 137, 169, 225, 196,   2, 190, 138, 243, 173,  10, 155, 224, 148, 154,  15,
+180, 176, 218, 153,   2, 194, 220, 179, 239,   3, 209, 243, 151, 171,   1, 135,
+192, 192, 129,   3, 154, 145, 158, 166,   8, 174, 159, 201, 207,   1, 134, 247,
+247, 152,   5, 169, 139, 159, 171,   3, 173, 170, 159, 244,  15, 201, 205, 215,
+223,   9, 227, 214, 226, 134,  14, 237, 245, 216, 153,   1, 207, 208, 244,  63,
+136, 146, 237, 215,   2, 131, 173, 129, 187,   4, 150, 204, 222, 185,   6, 243,
+177, 246, 252,   5, 246, 173, 234, 215,  14, 207, 252, 211, 199,   3, 177, 211,
+230, 228,   5, 208, 143, 209, 191,  13, 173, 192, 232, 246,  12, 132, 255, 207,
+139,  14, 171, 129, 141, 173,   7, 255, 222, 227, 255,  12, 155, 193, 184, 244,
+ 14, 171, 144, 214, 163,   1, 241, 232, 221, 228,  15, 188, 160, 210, 226,  13,
+189, 190, 189,   5, 204, 252, 250, 234,  10, 228, 161, 153, 190,   9, 210, 208,
+187, 214,   7, 198, 154, 214, 242,   9, 197, 163, 254,  27, 220, 251, 130, 172,
+  2, 193, 147, 157, 255,  14, 242, 131, 138, 180,  14, 200, 239, 175, 239,   5,
+181, 157, 238, 152,   1, 203, 211, 156, 220,  10, 210, 166, 223, 241,   2, 214,
+243, 250, 244,  10, 238, 200, 226, 216,   9, 168, 140, 235, 228,  14, 149, 176,
+161, 188,   9, 180, 224, 247, 138,  11, 168, 159, 157, 226,   7, 216, 226, 212,
+131,   5, 158, 162, 174, 190,   2, 147, 131, 155, 194,   4, 227, 156, 248, 169,
+ 14, 210, 216, 130, 142,  14, 233, 234, 248, 230,  13, 146, 190, 216, 248,   9,
+128, 173, 190, 149, 182,  11, 254, 210, 132, 152,   8, 211, 239, 231, 248,   9,
+132, 255, 247, 168,   7, 149, 224, 145, 136,  14, 162, 220, 148, 134,   6, 204,
+244, 192, 159,   8, 178, 160, 245, 237,  15, 193, 167, 249, 251,   5, 238, 159,
+153, 199,   9, 228, 225, 136, 225,   9, 147, 221, 134, 220,   7, 249, 129, 250,
+131,   5, 255, 249, 227, 129,  15, 183, 246, 177, 190,  10, 217, 182, 196, 128,
+  6, 136, 242, 159, 173,   1, 244, 128, 137, 210,  10, 154, 223, 230, 173,   7,
+193, 171, 203, 220,   9, 193, 222, 146, 129,   2, 159, 229, 247, 153,   1, 205,
+139, 189, 204,  13, 181, 152, 211, 186,   3, 252, 181, 234, 182,   4, 230, 212,
+233, 169,  13, 134, 211, 157, 165,   1, 218, 165, 218, 239,   4, 148, 140, 245,
+130,  11, 197, 152, 165, 199,   2, 235, 219, 158, 232,   9, 187, 231, 171, 149,
+ 12, 134, 191, 248, 157,   3, 219, 140, 128, 208,   1, 181, 140, 225, 226,  15,
+234, 239, 208, 170,  10, 166, 152, 192, 138,  15, 237, 204, 242, 197,  12, 230,
+224, 210,  68, 128, 170, 249, 251,  10, 193, 202, 171, 142,   7, 235, 192, 224,
+175,  14, 147, 243, 214,  94, 165, 202, 243, 157,   6, 192, 178, 204, 211,   8,
+242, 240, 207, 231,   4, 251, 234, 238, 218,   1, 207, 227, 224, 149,   4, 155,
+215, 210, 203,   2, 164, 248, 235, 166,   6, 226, 234, 165, 222,  13, 228, 197,
+249, 231,  14, 169, 172, 201, 163,  14, 149, 206, 208, 159,  15, 178, 216, 205,
+227,  15, 210, 228, 223, 220,   5, 161, 214, 153, 136,  11, 181, 178, 246, 212,
+  7, 128, 131, 238, 218,  13, 138, 156, 141, 139,  15, 134, 187, 137, 234,   4,
+152, 215, 181, 142,   6, 160, 185, 166, 193,  13, 213, 145, 204, 240,  13, 190,
+164, 216, 231,  13, 251, 208, 176, 231,   4, 243, 160, 187, 150,   5, 235, 251,
+246, 205,   3, 142, 232, 229, 222,   5, 227, 251, 238, 161,  12, 224, 198, 250,
+176,   3, 187, 162, 200, 223,   5, 199, 133, 234, 181,   3, 167, 160, 247, 232,
+  4, 174, 198, 216, 180,  15, 144, 251, 131, 187,  10, 161, 171, 169, 190,   9,
+223, 175, 171, 171,   4, 141, 165, 211, 128,   5, 139, 239, 131, 173,   3, 211,
+163, 253,  45, 212, 199, 216, 226,  11, 137, 216, 228, 198,   3, 216, 209, 199,
+233,   3, 249, 144, 225, 146,   1, 216, 184, 225, 218,   9, 197, 219, 219, 247,
+ 12, 214, 227, 243, 240,  14, 221, 155, 244, 141,   4, 239, 249, 179, 130,   4,
+161, 187, 191, 135,   3, 245, 241, 237, 241,  12, 194, 211, 209, 238,   5, 252,
+210, 135, 149,   1, 134, 241, 220, 170,  12, 175, 208, 242, 229,   9, 181, 144,
+172, 202,   7, 170, 195, 174, 180,   5, 198, 153, 178, 158,   6, 146, 142, 204,
+119, 137, 185, 250, 204,  10, 208, 190, 240, 166,   1, 138, 183, 212, 226,   3,
+241, 240, 245, 140,  15, 250, 184, 161, 117, 198, 194, 173, 133,  15, 135, 247,
+179, 180,  11, 158, 233, 195, 162,   2, 209, 143, 142, 203,  13, 156, 215, 224,
+192,   5, 228, 223, 167, 163,   6, 253, 160, 223, 182,   5, 178, 178, 223, 147,
+  5, 150, 180, 221, 189,  10, 168, 197, 173, 169,   6, 166, 146, 252, 254,  15,
+154, 211, 198, 238,   6, 182, 166, 227, 223,   3, 152, 209, 173, 192,   3, 147,
+255, 130, 153,   9, 152, 159, 128, 195,   7, 204, 199, 174, 227,   8, 149, 133,
+142,  33, 236, 185, 160, 136,  14, 154, 137, 143, 236,   7, 246, 149, 237, 166,
+  3, 150, 184, 224, 232,   3, 204, 220, 171, 245,  15, 128, 131, 146, 236, 219,
+ 10, 168, 253, 226, 198,   3, 196, 185, 159, 245,  14, 246, 239, 172, 207,   7,
+172, 188, 238, 233,  13, 193, 158, 247, 192,  10, 178, 146, 230, 233,   8, 143,
+221, 252, 145,   5, 169, 173, 160, 149,   7, 141, 199, 235,  35, 225, 224, 227,
+213,   7, 233, 249, 164, 132,  11, 255, 158, 248, 254,   2, 248, 200, 154, 176,
+  3, 168, 248, 134, 165,   8, 145, 177, 231, 188,  10, 189, 223, 182, 129,   7,
+246, 146, 219,  62, 185, 190, 133, 217,   3, 228, 177, 227, 170,   1, 230, 175,
+223, 120, 150, 130, 206, 166,   5, 223, 216, 157, 168,   1, 225, 151, 175, 248,
+  5, 140, 228, 227, 235,   7, 243, 148, 219, 250,   3, 250, 215, 234, 130,   1,
+191, 146, 221, 133,   8, 220, 223, 135, 100, 233, 148, 197, 224,  11, 164, 203,
+178, 134,   9, 170, 133, 159, 133,   8, 162, 189, 239,  68, 144, 186, 204, 211,
+  6, 167, 218, 219, 144,   2, 208, 155, 181, 237,   2, 253, 223, 151, 180,  15,
+137, 132, 173, 135,   7, 172, 137, 239, 146,  13, 250, 140, 255, 211,  11, 231,
+134, 228, 145,   3, 149, 220, 253, 168,  10, 236, 163, 149, 221,  10, 247, 151,
+236, 190,   6, 166, 210, 238,  52, 192, 248, 168, 229,   9, 237, 182, 227, 199,
+ 12, 189, 199, 195, 216,  12, 178, 236, 220, 158,   2, 247, 182, 235, 221,  14,
+219, 148, 216, 159,  15, 158, 234, 200, 167,   2, 184, 132, 251, 232,   2, 138,
+227, 158, 204,  14, 225, 192, 227, 165,   8, 130, 214, 149, 173,  13, 210, 140,
+161, 181,   9, 222, 217, 168, 158,  10, 220, 222, 238, 137,  10, 237, 248, 184,
+ 57, 167, 213, 169, 132,   5, 236, 173, 141,  25, 131, 201, 181, 180,   4, 133,
+182, 179, 134,  14, 243, 180, 195, 169,  11, 145, 153, 139, 242,  14, 210, 148,
+136, 230,   2, 174, 147, 246, 185,   7, 185, 230, 252, 230,  10, 247, 210, 139,
+242,  13, 187, 227, 199, 158,  14, 186, 209, 178, 166,   8, 148, 174, 212, 154,
+  6, 193, 139, 246, 160,   4, 180, 129, 135, 190,   7, 253, 202, 252, 194,   1,
+145, 192, 198, 192,   2, 136, 201, 194, 165,   5, 238, 198, 216, 222,   8, 148,
+132, 194, 231,   2, 179, 212, 226, 152,  13, 216, 203, 190,  81, 241, 158, 205,
+205,   3, 153, 250, 248, 251,  11, 157, 223, 163, 229,  11, 160, 240, 198, 156,
+ 13, 155, 254, 151, 138,  14, 219, 233, 172, 254,   4, 186, 194, 189, 227,   4,
+169, 243, 181, 201,  14, 161, 158, 146, 201,   3, 135, 139, 242, 206,   4, 222,
+141, 186, 201,  11, 247, 182, 166, 198,  12, 141, 168, 155, 172,   4, 206, 218,
+254, 175,   4, 140, 213, 159, 204,   7, 214, 128, 160, 215,   9, 253, 242, 237,
+147,   8, 162, 233, 151, 181,   5, 183, 223, 151,  21, 132, 164, 206, 242,   1,
+179, 227, 155, 165,  11, 189, 251, 195, 212,   3, 154, 195, 137, 190,   6, 129,
+212, 227, 177,   4, 185, 141, 235, 183,   7, 233, 220, 229, 174,   4, 215, 138,
+248,  25, 161, 210, 193, 241,  14, 239, 201, 231, 152,  12, 240, 169, 204, 169,
+ 14, 228, 195, 196, 225,   6, 250, 159, 144, 234,   1, 167, 238, 191, 142,  11,
+202, 222, 151, 207,   9, 205, 219, 185, 142,   3, 230, 224, 187, 235,   5, 194,
+167, 210, 173,   7, 235, 250, 253, 178,  12, 239, 128, 215, 198,  13, 130, 141,
+191, 238,   3, 173, 252, 172, 217,  14, 129, 203, 164,  16, 191, 131, 153, 141,
+  8, 133, 200, 131, 240,  15, 173, 165, 172,  11, 182, 247, 244, 165,   9, 128,
+238, 232, 219,  37, 214, 148, 220, 206,  10, 199, 154, 167, 130,   1, 188, 191,
+233, 235,   9, 167, 131, 215, 154,   5, 133, 224, 241, 202,   1, 237, 213, 192,
+223,   4, 160, 202, 178, 132,  10, 248, 217, 142, 133,  12, 199, 164, 231, 189,
+  5, 240, 129, 134, 189,   6, 173, 135, 204, 176,  15, 164, 142, 214, 137,   8,
+208, 169, 163, 251,  15, 196, 171, 247, 187,  14, 230, 177, 251, 130,  13, 200,
+234, 146, 173,   4, 252, 218, 210, 212,  10, 206, 187, 236, 129,   5, 165, 161,
+220, 171,  11, 135, 129, 179, 205,   2, 240, 251, 134, 254,   3, 136, 185, 186,
+220,  10, 230, 142, 156, 211,   1, 215, 243, 241, 179,  12, 141, 140, 140, 166,
+  5, 136, 183, 213, 220,  14, 182, 213, 134, 202,  10, 177, 197, 170, 230,   6,
+210, 133, 203, 128,  14, 145, 196, 176, 139,   5, 191, 143, 140, 133,  11, 247,
+155, 221, 233,  10, 131, 192, 238, 143,   3, 194, 196, 146, 129,   9, 245, 183,
+142, 133,   6, 200, 197, 143, 185,   2, 133, 144, 194, 144,   4, 149, 202, 240,
+ 36, 230, 214, 182, 211,   5, 254, 227, 217, 246,   2, 128, 164, 220, 255,   5,
+132, 138, 149, 153,   6, 200, 139, 167,  97, 203, 137, 179, 195,   2, 141, 176,
+199, 134,   9, 165, 244, 225, 254,   3, 136, 180, 252, 193,   3, 200, 165, 159,
+207,  12, 147, 222, 142, 148,   5, 191, 146, 228, 191,   9, 213, 255, 236, 152,
+ 13, 132, 240, 164, 174,   2, 204, 152, 214,   3, 251, 240, 222, 248,  10, 219,
+208, 211, 189,  15, 175, 252, 221,  88, 182, 234, 154, 107, 208, 190, 199, 159,
+  2, 209, 139, 150, 182,  13, 212, 219, 146, 154,  15, 221, 178, 221, 188,  11,
+148, 200, 197,  17, 129, 218, 170, 253,  11, 164, 244, 228, 252,   2, 220, 175,
+146, 195,  15, 141, 223, 154, 232,   9, 227, 186, 130, 220,   8, 153, 157, 145,
+139,  12, 233, 140, 173, 183,  12, 223, 255, 155, 139,  13, 162, 238, 129, 242,
+ 11, 252, 162, 211, 191,   2, 228, 182, 210, 101, 171, 202, 191, 167,  11, 247,
+189, 170, 255,   3, 217, 150, 238, 215,  10, 173, 188, 234, 177,   5, 166, 139,
+147, 132,  12, 230, 216, 153, 200,   3, 182, 202, 167, 210,  12, 222, 169, 137,
+180,   7, 253, 249, 181, 197,   2, 198, 205, 156, 192,  12, 168, 135, 243, 185,
+  2, 138, 158, 139, 159,  11, 138, 210, 248, 255,  14, 157, 141, 161, 207,   9,
+218, 206, 244, 191,   4, 222, 169, 188, 238,   5, 133, 211, 152, 218,  14, 248,
+191, 242, 250,  13, 217, 188, 239, 231,  14, 137, 198, 135, 144,   1, 231, 227,
+214, 168,   7, 128, 136, 152, 103, 150, 151, 161, 171,  12, 251, 222, 212, 229,
+  4, 154, 193, 182,  62, 251, 246, 205, 142,   3, 132, 140, 242, 166,  14, 165,
+231, 192, 250,   6, 136, 154, 230, 163,   1, 230, 228, 246, 182,   3, 187, 215,
+217, 177,   8, 137, 171, 251,  15, 211, 128, 230, 244,  15, 160, 146, 188, 255,
+  4, 204, 242, 150, 194,   1, 128, 184, 177, 139,  14, 139, 209, 245, 134,  11,
+241, 167, 181, 139,   5, 159, 129, 160,  74, 159, 200, 133, 222,   5, 157, 204,
+165, 199,  10, 193, 159, 169, 151,  11, 205, 219, 226, 134,   9, 197, 252, 179,
+128,  14, 230, 250, 244, 215,   5, 207, 138, 239, 212,  14, 237, 216, 191, 199,
+ 15, 250, 250, 198, 148,   9, 212, 228, 174, 146,  15, 221, 137, 207, 196,   3,
+146, 165, 245, 220,  13, 157, 249, 149, 228,   5, 185, 219, 188, 185,   8, 212,
+150, 240, 218,  15, 128, 211, 229, 202, 129,  14, 132, 225, 178, 226,   1, 251,
+195, 132,  66, 210, 245, 154, 234,   5, 145, 183, 146, 177,   9, 218, 223, 128,
+170,  13, 238, 227, 168, 197,  11, 189, 225, 206, 179,   6, 221, 169, 239, 193,
+  4, 194, 207, 170, 203,   7, 163, 206, 232, 197,   1, 160, 130, 131, 160,   4,
+139, 146, 149, 173,  10, 140, 240, 243, 180,   4, 231, 180, 202, 245,   9, 146,
+250, 195, 157,   1, 233, 199, 188, 210,  15, 253, 222, 137, 142,  10, 174, 245,
+231,  20, 219, 156, 185, 201,   5, 139, 137, 230, 135,   1, 236, 207, 146, 138,
+  4, 149, 174, 164, 221,   4, 158, 227, 224, 210,   7, 206, 150, 186, 244,   9,
+156, 183, 159, 142,  13, 176, 152, 163, 193,   8, 190, 229, 232, 155,   7, 234,
+132, 236, 132,   9, 242, 254, 204, 134,  14, 143, 226, 253, 180,   2, 138, 226,
+214, 218,   2, 199, 228, 210, 186,  12, 147, 179, 230, 254,   5, 249, 135, 247,
+147,  10, 148, 253, 186, 214,  12, 250, 240, 173, 159,  14, 162, 215, 177,  42,
+162, 142, 248, 135,   3, 196, 143, 150, 150,  10, 236, 221, 178, 147,   7, 165,
+248, 197, 136,   7, 199, 152, 158, 228,  13, 229, 215, 242, 194,   7, 145, 249,
+246, 181,  13, 134, 191, 196, 245,   3, 161, 251, 235, 200,  14, 255, 232, 248,
+228,  10, 170, 188, 227, 177,  14, 212, 202, 144, 143,  13, 199, 230, 234, 155,
+ 10, 247, 239, 142, 167,   6, 197, 129, 192, 235,   2, 207, 229, 194, 237,  12,
+228, 239, 211, 136,   3, 199, 135, 194, 244,   4, 167, 137, 158, 132,  15, 208,
+199, 176, 183,   2, 161, 181, 218, 155,  11, 218, 235, 160, 207,   5, 250, 181,
+244, 252,   9, 197, 130, 193, 168,   1, 153, 235, 181, 253,   2, 203, 245, 229,
+255,  11, 134, 136, 148, 249,   8, 179, 174, 133, 187,   8, 145, 212, 156, 196,
+  7, 163, 222, 227, 236,  11, 242, 171, 200, 143,  12, 185, 225, 231, 211,  15,
+135, 230, 213, 153,   6, 254, 187, 227, 167,   2, 147, 191, 160, 185,  12, 177,
+145, 137, 133,   1, 241, 244, 217, 231,   3, 225, 213, 246, 253,  11, 138, 185,
+169, 229,   8, 129, 248, 228, 155,   4, 150, 208, 194, 129,  13, 149, 233, 140,
+159,   7, 149, 223, 199,  33, 153, 214, 176, 117, 175, 193, 163, 144,   9, 135,
+207, 150,  12, 216, 138, 151,  55, 233, 245, 225, 219,   8, 215, 194, 201, 214,
+  6, 235, 254, 134,  70, 251, 142, 174, 209,  12, 215, 218, 132, 174,   4, 209,
+177, 189, 144,   3, 247, 136, 205, 212,   8, 152, 220, 178, 208,   5, 183, 146,
+202, 149,   6, 248, 229, 196, 211,  12, 226, 191, 237, 227,   7, 234, 157, 195,
+196,   4, 203, 147, 213, 156,   1, 245, 161, 241,  97, 186, 245, 223, 246,   8,
+170, 241, 234, 188,   8, 171, 155, 201, 168,   8, 193, 168, 145, 142,  10, 254,
+183, 192, 202,  14, 137, 175, 147, 223,   9, 176, 133, 131, 166,  12, 211, 168,
+155, 225,   4, 197, 193, 255, 204,   8, 154, 208, 144, 165,   1, 134, 190, 143,
+217,  11, 148, 242, 203, 237,  11, 161, 142, 172, 215,   3, 166, 203, 240, 162,
+  6, 200, 195, 186, 162,   7, 198, 211, 223, 252,  15, 132, 160, 226, 204,  15,
+158, 187, 167, 222,   6, 174, 214, 139, 220,   9, 130, 243, 221, 206,   6, 190,
+217, 211, 145,   4, 160, 255, 142, 201,   5, 201, 166, 217, 174,   7, 240, 197,
+130, 214,   7, 216, 133, 220, 184,   3, 241, 148, 192, 185,   6, 213, 181, 240,
+210,   2, 137, 194, 206, 172,   5, 221, 189, 134, 241,  10, 128, 180, 234, 178,
+219,  13, 203, 213, 182, 247,  10, 172, 229, 222, 178,  15, 188, 154, 206, 196,
+ 12, 240, 136, 172, 156,  11, 165, 151, 164, 200,   7, 189, 152, 225, 146,   7,
+214, 167, 205, 147,   4, 216, 175, 130, 230,  10, 243, 162, 145, 154,   7, 155,
+169, 190, 182,   6, 255, 212, 152, 251,   6, 147, 152, 160, 237,   2, 170, 228,
+233, 210,  13, 166, 255, 247, 207,  14, 238, 175, 242, 171,   9, 174, 241, 193,
+193,   4, 245, 210, 147, 167,  14, 151, 233, 199, 154,   1, 193, 184, 194, 249,
+  9, 216, 255, 201, 246,  10, 138, 198, 240, 208,   5, 187, 230, 137, 145,   1,
+200, 237, 144, 115, 131, 149, 167, 201,  15, 249, 130, 240, 202,   5, 141, 220,
+198, 233,   3, 216, 165, 204, 210,  12, 176, 166, 249, 207,   4, 244, 158, 162,
+140,   8, 174, 153, 181, 253,  14, 249, 157, 148, 130,  10, 178, 203, 201, 162,
+  4, 161, 215, 176, 137,   3, 164, 232, 198, 200,  15, 141, 189, 153, 206,   6,
+148, 138, 219, 252,  12, 147, 134, 206, 210,   9, 214, 186, 141, 183,  10, 235,
+192, 204, 245,  10, 155, 177, 148, 174,   7, 246, 150, 200, 167,  15, 134, 228,
+212, 210,   7, 128, 198, 173, 133,  10, 173, 148, 155, 170,   4, 131, 242, 205,
+148,  14, 154, 220, 156, 236,  11, 213, 150, 219, 145,   4, 171, 231, 199, 224,
+ 12, 190, 139, 161, 155,   3, 136, 151, 199, 129,   9, 182, 161, 156, 237,   1,
+218, 151, 248, 132,  13, 201, 207, 164, 115, 190, 137, 205, 255,  11, 191, 198,
+251, 165,  10, 234, 205, 249, 181,   3, 172, 185, 218, 244,  14, 134, 171, 214,
+151,   9, 152, 245, 182, 215,  10, 204, 161, 209, 196,  14, 180, 134, 204, 240,
+  4, 242, 196, 170, 185,  13, 156, 255, 134, 178,  14, 203, 145, 211, 216,   3,
+190, 148, 160, 180,  14, 189, 162, 214, 209,  10, 238, 176, 239, 248,  15, 151,
+163, 176, 168,   5, 152, 247, 207, 238,  14, 181, 238, 168, 251,   8, 181, 189,
+202,  33, 232, 239, 229, 226,   5, 133, 156, 212, 180,   4, 224, 169, 249, 216,
+  4, 198, 245, 205, 147,   8, 231, 232, 149, 230,   8, 243, 161, 191, 162,   3,
+194, 189, 237, 227,  15, 223, 185, 161, 232,   4, 153, 233, 249, 155,   8, 240,
+147, 199, 249,   5, 135, 205, 250, 160,  11, 252, 183, 238, 210,  10, 244, 146,
+156, 160,   5, 196, 252, 142,  22, 191, 148, 222, 231,  11, 182, 201, 163, 219,
+ 13, 199, 238, 233, 179,   4, 180, 199, 255, 249,   4, 254, 237, 180, 213,   2,
+211, 221, 157, 151,   7, 178, 192, 158, 241,  13, 133, 212, 252,  51, 146, 221,
+241, 177,   7, 137, 246, 204, 171,   5, 138, 209, 144, 231,   2, 153, 213, 230,
+179,   4, 171, 244, 213, 172,   3, 191, 201, 249, 129,   3, 184, 184, 186, 243,
+ 11, 145, 176, 183, 103, 145, 131, 206, 147,  14, 136, 134, 191, 173,   9, 180,
+164, 241, 245,   5, 172, 133, 212, 167,   8, 198, 162, 158, 244,   4, 232, 175,
+222, 231,  12, 146, 246, 134, 196,   8, 147, 248, 177, 230,   8, 145, 216, 180,
+139,  12, 224, 254, 191, 222,   1, 182, 145, 213, 232,  10, 178, 139, 143, 237,
+  9, 253, 230, 172, 181,  13, 225, 218, 252, 132,   6, 141, 175, 159, 197,  14,
+185, 222, 237, 246,   3, 154, 184, 245, 228,  11, 193, 198, 235, 204,  10, 182,
+239, 253, 136,  15, 205, 143, 161, 211,   7, 164, 207, 235, 220,   4, 158, 235,
+183, 187,   9, 203, 201, 147, 139,   3, 169, 181, 153, 201,  11, 222, 206, 192,
+251,  12, 221, 253, 242, 152,  11, 128, 249, 190, 248, 152,   5, 151, 199, 221,
+227,  14, 209, 246, 133, 200,   7, 246, 181, 176, 131,   9, 165, 219, 139, 171,
+  4, 254, 130, 187, 208,   8, 144, 221, 189, 192,  10, 163, 146, 139, 166,  12,
+231, 177, 223, 205,   9, 229, 179, 214, 227,   2, 132, 153, 150, 154,   5, 242,
+250, 159, 171,   9, 144, 228, 238, 120, 168, 206, 130, 107, 145, 144, 235, 248,
+  1, 254, 218, 166, 129,   4, 237, 129, 235,   7, 150, 199, 251, 175,   9, 252,
+199, 200, 168,   9, 172, 147, 153, 151,   5, 168, 129, 129, 188,  13, 200, 166,
+192, 192,   8, 154, 184, 218, 232,   2, 155, 202, 193, 156,  12, 182, 241, 250,
+153,  13, 180, 141, 206, 141,   6, 206, 129, 157, 153,  12, 132, 158, 212, 247,
+ 14, 160, 135, 203, 238,  12, 216, 173, 204, 156,   9, 166, 214, 242, 138,  14,
+178, 248, 246, 135,   1, 244, 219, 210, 155,   3, 208, 155, 189, 180,   3, 156,
+189, 171, 174,  13, 162, 161, 233, 108, 231, 134, 177, 255,  11, 216, 159, 226,
+244,   2, 130, 227, 211, 185,   6, 169, 146, 187, 143,   2, 153, 225, 150, 187,
+  9, 210, 153, 211, 181,  13, 147, 216, 152, 173,   3, 246, 236, 142,  33, 222,
+140, 194, 241,  10, 171, 251, 248, 210,  11, 239, 197, 137, 242,   8, 132, 179,
+189, 209,   6, 181, 221, 179, 161,   2, 168, 172, 241, 163,   2, 203, 241, 250,
+226,  12, 184, 188, 237, 210,   8, 228, 163, 153, 234,   5, 222, 162, 216, 142,
+ 13, 235, 251, 186, 239,   1, 199, 133, 166, 158,   9, 134, 241, 161, 174,   3,
+183, 248, 214, 158,   7, 181, 248, 184, 143,  11, 234, 152, 151, 169,   9, 198,
+134, 159, 251,   7, 144, 176, 211, 121, 199, 255, 166, 132,   5, 201, 243, 215,
+189,  14, 213, 240, 205, 223,  10, 205, 191, 234, 185,   9, 240, 221, 255, 234,
+  5, 210, 250, 179, 148,   7, 185, 162, 155, 243,   3, 140, 197, 165, 222,  12,
+150, 143, 215, 241,  13, 138, 138, 246,  30, 236, 151, 243, 235,  12, 232, 222,
+197, 223,   5, 177, 198, 228, 194,   4, 130, 172, 242, 221,  11, 208, 235, 221,
+161,   1, 254, 141, 148, 144,  14, 168, 251, 185, 179,   9, 247, 144, 244, 178,
+ 12, 209, 235, 151, 183,   9, 131, 208, 184, 182,  13, 135, 245, 

<TRUNCATED>