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 2019/10/22 23:15:53 UTC

[orc] branch branch-1.6 updated (a5c7b16 -> 1b24716)

This is an automated email from the ASF dual-hosted git repository.

omalley pushed a change to branch branch-1.6
in repository https://gitbox.apache.org/repos/asf/orc.git.


    from a5c7b16  ORC-528: Add unittest for timestamp off-by-one issues
     new 1328fd8  ORC-552: Fix c++ compilation for centos 6 and 8, and MacOs 10.14.
     new f789ca2  ORC-560: Improve docker scripts and add centos 8 and debian 10.
     new 0635bb3  ORC-414: [C++] Check root type existence before reading files
     new 1b24716  ORC-557 Fix problem reading large header with uncompressed streams.

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 CMakeLists.txt                                     |  6 +-
 c++/src/ColumnPrinter.cc                           | 56 +++++++++---------
 c++/src/Reader.cc                                  | 12 ++--
 c++/src/Vector.cc                                  | 35 +++++------
 c++/test/TestColumnReader.cc                       |  3 +
 c++/test/TestType.cc                               |  2 +
 cmake_modules/FindSnappy.cmake                     |  2 +-
 cmake_modules/ThirdpartyToolchain.cmake            |  2 +-
 docker/{centos7 => centos8}/Dockerfile             | 12 +---
 docker/{debian9 => debian10}/Dockerfile            |  8 +--
 docker/os-list.txt                                 |  2 +
 docker/run-all.sh                                  | 31 +++++-----
 docker/{run-all.sh => run-one.sh}                  | 48 +++++++--------
 .../src/java/org/apache/orc/impl/InStream.java     | 69 ++++++++++------------
 .../src/test/org/apache/orc/impl/TestInStream.java | 52 ++++++++++++++++
 15 files changed, 194 insertions(+), 146 deletions(-)
 copy docker/{centos7 => centos8}/Dockerfile (83%)
 copy docker/{debian9 => debian10}/Dockerfile (90%)
 copy docker/{run-all.sh => run-one.sh} (63%)


[orc] 04/04: ORC-557 Fix problem reading large header with uncompressed streams.

Posted by om...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

omalley pushed a commit to branch branch-1.6
in repository https://gitbox.apache.org/repos/asf/orc.git

commit 1b2471666f6a41341bf6a9961a6a251c0f0246e0
Author: Owen O'Malley <om...@apache.org>
AuthorDate: Fri Oct 18 16:31:29 2019 -0700

    ORC-557 Fix problem reading large header with uncompressed streams.
    
    Fixes #439 #435
    
    Signed-off-by: Owen O'Malley <om...@apache.org>
---
 .../src/java/org/apache/orc/impl/InStream.java     | 69 ++++++++++------------
 .../src/test/org/apache/orc/impl/TestInStream.java | 52 ++++++++++++++++
 2 files changed, 83 insertions(+), 38 deletions(-)

diff --git a/java/core/src/java/org/apache/orc/impl/InStream.java b/java/core/src/java/org/apache/orc/impl/InStream.java
index 5508e76..170d101 100644
--- a/java/core/src/java/org/apache/orc/impl/InStream.java
+++ b/java/core/src/java/org/apache/orc/impl/InStream.java
@@ -45,6 +45,9 @@ public abstract class InStream extends InputStream {
   protected final Object name;
   protected final long offset;
   protected final long length;
+  protected DiskRangeList bytes;
+  // position in the stream (0..length)
+  protected long position;
 
   public InStream(Object name, long offset, long length) {
     this.name = name;
@@ -59,6 +62,32 @@ public abstract class InStream extends InputStream {
   @Override
   public abstract void close();
 
+  /**
+   * Set the current range
+   * @param newRange the block that is current
+   * @param isJump if this was a seek instead of a natural read
+   */
+  abstract protected void setCurrent(DiskRangeList newRange,
+                                     boolean isJump);
+    /**
+     * Reset the input to a new set of data.
+     * @param input the input data
+     */
+  protected void reset(DiskRangeList input) {
+    bytes = input;
+    while (input != null &&
+               (input.getEnd() <= offset ||
+                    input.getOffset() > offset + length)) {
+      input = input.next;
+    }
+    if (input == null || input.getOffset() <= offset) {
+      position = 0;
+    } else {
+      position = input.getOffset() - offset;
+    }
+    setCurrent(input, true);
+  }
+
   public abstract void changeIv(Consumer<byte[]> modifier);
 
   static int getRangeNumber(DiskRangeList list, DiskRangeList current) {
@@ -75,9 +104,6 @@ public abstract class InStream extends InputStream {
    * Implements a stream over an uncompressed stream.
    */
   public static class UncompressedStream extends InStream {
-    private DiskRangeList bytes;
-    // position in the stream (0..length)
-    protected long position;
     protected ByteBuffer decrypted;
     protected DiskRangeList currentRange;
     protected long currentOffset;
@@ -100,16 +126,6 @@ public abstract class InStream extends InputStream {
       reset(input);
     }
 
-    protected void reset(DiskRangeList input) {
-      this.bytes = input;
-      if (input == null || input.getOffset() <= offset) {
-        position = 0;
-      } else {
-        position = input.getOffset() - offset;
-      }
-      setCurrent(input, true);
-    }
-
     @Override
     public int read() {
       if (decrypted == null || decrypted.remaining() == 0) {
@@ -230,7 +246,6 @@ public abstract class InStream extends InputStream {
    */
   static class EncryptionState {
     private final Object name;
-    private final EncryptionAlgorithm algorithm;
     private final Key key;
     private final byte[] iv;
     private final Cipher cipher;
@@ -240,7 +255,7 @@ public abstract class InStream extends InputStream {
     EncryptionState(Object name, long offset, StreamOptions options) {
       this.name = name;
       this.offset = offset;
-      algorithm = options.getAlgorithm();
+      EncryptionAlgorithm algorithm = options.getAlgorithm();
       key = options.getKey();
       iv = options.getIv();
       cipher = algorithm.createCipher();
@@ -347,9 +362,8 @@ public abstract class InStream extends InputStream {
         // what is the position of the start of the newRange?
         currentOffset = newRange.getOffset();
         ByteBuffer encrypted = newRange.getData().slice();
-        int ignoreBytes = 0;
         if (currentOffset < offset) {
-          ignoreBytes = (int) (offset - currentOffset);
+          int ignoreBytes = (int) (offset - currentOffset);
           encrypted.position(ignoreBytes);
           currentOffset = offset;
         }
@@ -382,12 +396,10 @@ public abstract class InStream extends InputStream {
   }
 
   private static class CompressedStream extends InStream {
-    private DiskRangeList bytes;
     private final int bufferSize;
     private ByteBuffer uncompressed;
     private final CompressionCodec codec;
     protected ByteBuffer compressed;
-    protected long position;
     protected DiskRangeList currentRange;
     private boolean isUncompressedOriginal;
 
@@ -426,25 +438,6 @@ public abstract class InStream extends InputStream {
       reset(input);
     }
 
-    /**
-     * Reset the input to a new set of data.
-     * @param input the input data
-     */
-    void reset(DiskRangeList input) {
-      bytes = input;
-      while (input != null &&
-                 (input.getEnd() <= offset ||
-                      input.getOffset() > offset + length)) {
-        input = input.next;
-      }
-      if (input == null || input.getOffset() <= offset) {
-        position = 0;
-      } else {
-        position = input.getOffset() - offset;
-      }
-      setCurrent(input, true);
-    }
-
     private void allocateForUncompressed(int size, boolean isDirect) {
       uncompressed = allocateBuffer(size, isDirect);
     }
diff --git a/java/core/src/test/org/apache/orc/impl/TestInStream.java b/java/core/src/test/org/apache/orc/impl/TestInStream.java
index 9c6490c..7b7cbf1 100644
--- a/java/core/src/test/org/apache/orc/impl/TestInStream.java
+++ b/java/core/src/test/org/apache/orc/impl/TestInStream.java
@@ -738,6 +738,58 @@ public class TestInStream {
   }
 
   @Test
+  public void testExtraFrontUncompressed() throws IOException {
+    // Set up a stream that starts at START, which is divided in to regions
+    // of CHUNK_LENGTH. There are two EXTRA_FRONT byte buffers in front of the
+    // stream.
+    final long START = 1_000_000_000;
+    final int EXTRA_FRONT = 3_000;
+    final int CHUNK_LENGTH = 100;
+    final int STREAM_LENGTH = 4096;
+
+    BufferChunkList list = new BufferChunkList();
+    list.add(new BufferChunk(ByteBuffer.allocate(EXTRA_FRONT),
+        START - 2 * EXTRA_FRONT));
+    byte[] extraFront = new byte[EXTRA_FRONT + CHUNK_LENGTH];
+    Arrays.fill(extraFront, (byte) -1);
+    for(int i=0; i < CHUNK_LENGTH; ++i) {
+      extraFront[EXTRA_FRONT + i] = (byte) i;
+    }
+    list.add(new BufferChunk(ByteBuffer.wrap(extraFront), START - EXTRA_FRONT));
+    byte[] expected = new byte[STREAM_LENGTH];
+    for(int i=CHUNK_LENGTH; i < expected.length; ++i) {
+      expected[i] = (byte) i;
+    }
+    int posn = CHUNK_LENGTH;
+    while (posn <= expected.length) {
+      list.add(new BufferChunk(
+          ByteBuffer.wrap(expected, posn,
+              Math.min(CHUNK_LENGTH, expected.length - posn)),
+          START + posn));
+      posn += CHUNK_LENGTH;
+    }
+
+    // now set up the stream to read it
+    InStream.StreamOptions options = InStream.options();
+    InStream inStream = InStream.create("test", list.get(), START, STREAM_LENGTH,
+        options);
+
+    // ensure the data is correct
+    byte[] inBuffer = new byte[STREAM_LENGTH];
+    posn = 0;
+    int read = inStream.read(inBuffer);
+    while (read != -1) {
+      assertEquals("Read length at " + posn,
+          Math.min(STREAM_LENGTH - posn, CHUNK_LENGTH), read);
+      for(int i=0; i < read; ++i) {
+        assertEquals("posn " + posn + " + " + i, (byte)(posn + i), inBuffer[i]);
+      }
+      posn += read;
+      read = inStream.read(inBuffer);
+    }
+  }
+
+  @Test
   public void testExtraFrontCompressed() throws IOException {
     // Set up a stream that starts at START, which is divided in to regions
     // of CHUNK_LENGTH. There are two EXTRA_FRONT byte buffers in front of the


[orc] 02/04: ORC-560: Improve docker scripts and add centos 8 and debian 10.

Posted by om...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

omalley pushed a commit to branch branch-1.6
in repository https://gitbox.apache.org/repos/asf/orc.git

commit f789ca2597f82281e37276f5b2328e2f31bb6010
Author: Owen O'Malley <om...@apache.org>
AuthorDate: Thu Oct 17 06:57:54 2019 -0700

    ORC-560: Improve docker scripts and add centos 8 and debian 10.
    
    Fixes #437
    
    Signed-off-by: Owen O'Malley <om...@apache.org>
---
 docker/centos8/Dockerfile         | 53 +++++++++++++++++++++++++++++++++++++++
 docker/debian10/Dockerfile        | 43 +++++++++++++++++++++++++++++++
 docker/os-list.txt                |  2 ++
 docker/run-all.sh                 | 31 ++++++++++++-----------
 docker/{run-all.sh => run-one.sh} | 48 ++++++++++++++++-------------------
 5 files changed, 135 insertions(+), 42 deletions(-)

diff --git a/docker/centos8/Dockerfile b/docker/centos8/Dockerfile
new file mode 100644
index 0000000..4a6d2c6
--- /dev/null
+++ b/docker/centos8/Dockerfile
@@ -0,0 +1,53 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# ORC compile for CentOS 8
+#
+
+FROM centos:8
+MAINTAINER Owen O'Malley <ow...@hortonworks.com>
+
+RUN yum check-update || true
+RUN yum install -y \
+  cmake \
+  curl-devel \
+  cyrus-sasl-devel \
+  expat-devel \
+  gcc \
+  gcc-c++ \
+  gettext-devel \
+  git \
+  java-1.8.0-openjdk \
+  java-1.8.0-openjdk-devel \
+  libtool \
+  make \
+  maven \
+  openssl-devel \
+  tar \
+  wget \
+  which \
+  zlib-devel
+
+ENV TZ=America/Los_Angeles
+WORKDIR /root
+
+VOLUME /root/.m2/repository
+
+CMD git clone https://github.com/apache/orc.git -b master && \
+  mkdir orc/build && \
+  cd orc/build && \
+  cmake .. && \ 
+  make package test-out
diff --git a/docker/debian10/Dockerfile b/docker/debian10/Dockerfile
new file mode 100644
index 0000000..fb73d5b
--- /dev/null
+++ b/docker/debian10/Dockerfile
@@ -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.
+
+# ORC compile for Debian 10
+#
+
+FROM debian:buster
+MAINTAINER Owen O'Malley <om...@apache.org>
+
+RUN apt-get update
+RUN apt-get install -y \
+  cmake \
+  gcc \
+  g++ \
+  git \
+  libsasl2-dev \
+  libssl-dev \
+  openjdk-11-jdk \
+  make \
+  maven
+
+WORKDIR /root
+
+VOLUME /root/.m2/repository
+
+CMD git clone https://github.com/apache/orc.git -b master && \
+  mkdir orc/build && \
+  cd orc/build && \
+  cmake .. && \
+  make package test-out
diff --git a/docker/os-list.txt b/docker/os-list.txt
index 7d38a34..2343bb2 100644
--- a/docker/os-list.txt
+++ b/docker/os-list.txt
@@ -1,7 +1,9 @@
 centos6
 centos7
+centos8
 debian8
 debian9
+debian10
 ubuntu14
 ubuntu16
 ubuntu18
diff --git a/docker/run-all.sh b/docker/run-all.sh
index d770bae..8f1436b 100755
--- a/docker/run-all.sh
+++ b/docker/run-all.sh
@@ -24,31 +24,32 @@ MAKEDIR="mkdir orc/build && cd orc/build"
 VOLUME="--volume m2cache:/root/.m2/repository"
 mkdir -p logs
 
+function failure {
+    echo "Failed tests"
+    grep -h "FAILED " logs/*-test.log
+    exit 1
+}
+
 start=`date`
 for os in `cat os-list.txt`; do
   echo "Building $os"
   ( cd $os && docker build -t "orc-$os" . ) > logs/$os-build.log 2>&1 || exit 1
 done
 testStart=`date`
+rm -f logs/pids.txt
 
 for os in `cat os-list.txt`; do
-  echo "Testing $os"
-  case $os in
-  centos6|ubuntu12)
-     OPTS="-DSNAPPY_HOME=/usr/local -DPROTOBUF_HOME=/usr/local"
-     ;;
-  centos7|debian8|ubuntu14)
-     OPTS="-DSNAPPY_HOME=/usr/local"
-     ;;
-  *)
-     OPTS=""
-     ;;
-  esac
-  docker run $VOLUME "orc-$os" /bin/bash -c \
-	 "$CLONE && $MAKEDIR && cmake $OPTS .. && make package test-out" \
-         > logs/$os-test.log 2>&1 || exit 1
+    ./run-one.sh $1 $2 $os > logs/$os-test.log 2>&1 &
+    echo "$!" >> logs/pids.txt
+    echo "Launching $os as $!"
+done
+
+for job in `cat logs/pids.txt`; do
+    echo "Waiting for $job"
+    wait $job || failure
 done
 
+echo ""
 echo "Build start: $start"
 echo "Test start: $testStart"
 echo "End:" `date`
diff --git a/docker/run-all.sh b/docker/run-one.sh
similarity index 63%
copy from docker/run-all.sh
copy to docker/run-one.sh
index d770bae..7688560 100755
--- a/docker/run-all.sh
+++ b/docker/run-one.sh
@@ -18,37 +18,31 @@
 GITHUB_USER=$1
 URL=https://github.com/$GITHUB_USER/orc.git
 BRANCH=$2
+OS=$3
+
+function failure {
+    echo "FAILED $OS"
+    exit 1
+}
 
 CLONE="git clone $URL -b $BRANCH"
 MAKEDIR="mkdir orc/build && cd orc/build"
 VOLUME="--volume m2cache:/root/.m2/repository"
-mkdir -p logs
 
-start=`date`
-for os in `cat os-list.txt`; do
-  echo "Building $os"
-  ( cd $os && docker build -t "orc-$os" . ) > logs/$os-build.log 2>&1 || exit 1
-done
-testStart=`date`
+echo "Started $GITHUB_USER/$BRANCH on $OS at $(date)"
 
-for os in `cat os-list.txt`; do
-  echo "Testing $os"
-  case $os in
-  centos6|ubuntu12)
-     OPTS="-DSNAPPY_HOME=/usr/local -DPROTOBUF_HOME=/usr/local"
-     ;;
-  centos7|debian8|ubuntu14)
-     OPTS="-DSNAPPY_HOME=/usr/local"
-     ;;
-  *)
-     OPTS=""
-     ;;
-  esac
-  docker run $VOLUME "orc-$os" /bin/bash -c \
+case $OS in
+centos6|ubuntu12)
+   OPTS="-DSNAPPY_HOME=/usr/local -DPROTOBUF_HOME=/usr/local"
+   ;;
+centos7|debian8|ubuntu14)
+   OPTS="-DSNAPPY_HOME=/usr/local"
+   ;;
+*)
+   OPTS=""
+   ;;
+esac
+docker run $VOLUME "orc-$OS" /bin/bash -c \
 	 "$CLONE && $MAKEDIR && cmake $OPTS .. && make package test-out" \
-         > logs/$os-test.log 2>&1 || exit 1
-done
-
-echo "Build start: $start"
-echo "Test start: $testStart"
-echo "End:" `date`
+         || failure
+echo "Finished $OS at $(date)"


[orc] 01/04: ORC-552: Fix c++ compilation for centos 6 and 8, and MacOs 10.14.

Posted by om...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

omalley pushed a commit to branch branch-1.6
in repository https://gitbox.apache.org/repos/asf/orc.git

commit 1328fd8afb3082aa4ed39efd560eef0bac884bd7
Author: Owen O'Malley <om...@apache.org>
AuthorDate: Mon Oct 14 14:40:06 2019 -0700

    ORC-552: Fix c++ compilation for centos 6 and 8, and MacOs 10.14.
    
    Fixes #436
    
    Signed-off-by: Owen O'Malley <om...@apache.org>
---
 CMakeLists.txt                          |  6 +++-
 c++/src/ColumnPrinter.cc                | 56 ++++++++++++++++-----------------
 c++/src/Reader.cc                       |  9 +++---
 c++/src/Vector.cc                       | 35 +++++++++++----------
 c++/test/TestColumnReader.cc            |  3 ++
 cmake_modules/FindSnappy.cmake          |  2 +-
 cmake_modules/ThirdpartyToolchain.cmake |  2 +-
 7 files changed, 61 insertions(+), 52 deletions(-)

diff --git a/CMakeLists.txt b/CMakeLists.txt
index 15c6ecd..6102daf 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -81,7 +81,11 @@ if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
   set (WARN_FLAGS "${WARN_FLAGS} -Wno-covered-switch-default")
   set (WARN_FLAGS "${WARN_FLAGS} -Wno-missing-noreturn -Wno-unknown-pragmas")
   set (WARN_FLAGS "${WARN_FLAGS} -Wno-gnu-zero-variadic-macro-arguments")
-  set (WARN_FLAGS "${WARN_FLAGS} -Wconversion -Werror")
+  set (WARN_FLAGS "${WARN_FLAGS} -Wconversion")
+  if (CMAKE_HOST_APPLE AND CMAKE_CXX_COMPILER_VERSION VERSION_GREATER "11.0")
+    set (WARN_FLAGS "${WARN_FLAGS} -Wno-c++2a-compat")
+  endif ()
+  set (WARN_FLAGS "${WARN_FLAGS} -Werror")
 elseif (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
   set (WARN_FLAGS "-Wall -Wno-unknown-pragmas -Wconversion -Werror")
   if (CMAKE_CXX_COMPILER_VERSION STREQUAL "" OR
diff --git a/c++/src/ColumnPrinter.cc b/c++/src/ColumnPrinter.cc
index 379ef0b..b4b5860 100644
--- a/c++/src/ColumnPrinter.cc
+++ b/c++/src/ColumnPrinter.cc
@@ -289,8 +289,8 @@ namespace orc {
     return std::unique_ptr<ColumnPrinter>(result);
   }
 
-  VoidColumnPrinter::VoidColumnPrinter(std::string& buffer
-                                       ): ColumnPrinter(buffer) {
+  VoidColumnPrinter::VoidColumnPrinter(std::string& _buffer
+                                       ): ColumnPrinter(_buffer) {
     // PASS
   }
 
@@ -302,8 +302,8 @@ namespace orc {
     writeString(buffer, "null");
   }
 
-  LongColumnPrinter::LongColumnPrinter(std::string& buffer
-                                       ): ColumnPrinter(buffer),
+  LongColumnPrinter::LongColumnPrinter(std::string& _buffer
+                                       ): ColumnPrinter(_buffer),
                                           data(nullptr) {
     // PASS
   }
@@ -324,9 +324,9 @@ namespace orc {
     }
   }
 
-  DoubleColumnPrinter::DoubleColumnPrinter(std::string& buffer,
+  DoubleColumnPrinter::DoubleColumnPrinter(std::string& _buffer,
                                            const Type& type
-                                           ): ColumnPrinter(buffer),
+                                           ): ColumnPrinter(_buffer),
                                               data(nullptr),
                                               isFloat(type.getKind() == FLOAT){
     // PASS
@@ -348,8 +348,8 @@ namespace orc {
     }
   }
 
-  Decimal64ColumnPrinter::Decimal64ColumnPrinter(std::string& buffer
-                                                 ): ColumnPrinter(buffer),
+  Decimal64ColumnPrinter::Decimal64ColumnPrinter(std::string& _buffer
+                                                 ): ColumnPrinter(_buffer),
                                                     data(nullptr),
                                                     scale(0) {
     // PASS
@@ -398,8 +398,8 @@ namespace orc {
     }
   }
 
-  Decimal128ColumnPrinter::Decimal128ColumnPrinter(std::string& buffer
-                                                   ): ColumnPrinter(buffer),
+  Decimal128ColumnPrinter::Decimal128ColumnPrinter(std::string& _buffer
+                                                   ): ColumnPrinter(_buffer),
                                                       data(nullptr),
                                                       scale(0) {
      // PASS
@@ -419,8 +419,8 @@ namespace orc {
      }
    }
 
-  StringColumnPrinter::StringColumnPrinter(std::string& buffer
-                                           ): ColumnPrinter(buffer),
+  StringColumnPrinter::StringColumnPrinter(std::string& _buffer
+                                           ): ColumnPrinter(_buffer),
                                               start(nullptr),
                                               length(nullptr) {
     // PASS
@@ -470,9 +470,9 @@ namespace orc {
     }
   }
 
-  ListColumnPrinter::ListColumnPrinter(std::string& buffer,
+  ListColumnPrinter::ListColumnPrinter(std::string& _buffer,
                                        const Type& type
-                                       ): ColumnPrinter(buffer),
+                                       ): ColumnPrinter(_buffer),
                                           offsets(nullptr) {
     elementPrinter = createColumnPrinter(buffer, type.getSubtype(0));
   }
@@ -499,9 +499,9 @@ namespace orc {
     }
   }
 
-  MapColumnPrinter::MapColumnPrinter(std::string& buffer,
+  MapColumnPrinter::MapColumnPrinter(std::string& _buffer,
                                      const Type& type
-                                     ): ColumnPrinter(buffer),
+                                     ): ColumnPrinter(_buffer),
                                         offsets(nullptr) {
     keyPrinter = createColumnPrinter(buffer, type.getSubtype(0));
     elementPrinter = createColumnPrinter(buffer, type.getSubtype(1));
@@ -534,9 +534,9 @@ namespace orc {
     }
   }
 
-  UnionColumnPrinter::UnionColumnPrinter(std::string& buffer,
+  UnionColumnPrinter::UnionColumnPrinter(std::string& _buffer,
                                            const Type& type
-                                         ): ColumnPrinter(buffer),
+                                         ): ColumnPrinter(_buffer),
                                             tags(nullptr),
                                             offsets(nullptr) {
     for(unsigned int i=0; i < type.getSubtypeCount(); ++i) {
@@ -577,9 +577,9 @@ namespace orc {
     }
   }
 
-  StructColumnPrinter::StructColumnPrinter(std::string& buffer,
+  StructColumnPrinter::StructColumnPrinter(std::string& _buffer,
                                            const Type& type
-                                           ): ColumnPrinter(buffer) {
+                                           ): ColumnPrinter(_buffer) {
     for(unsigned int i=0; i < type.getSubtypeCount(); ++i) {
       fieldNames.push_back(type.getFieldName(i));
       fieldPrinter.push_back(createColumnPrinter(buffer,
@@ -621,8 +621,8 @@ namespace orc {
     }
   }
 
-  DateColumnPrinter::DateColumnPrinter(std::string& buffer
-                                       ): ColumnPrinter(buffer),
+  DateColumnPrinter::DateColumnPrinter(std::string& _buffer
+                                       ): ColumnPrinter(_buffer),
                                           data(nullptr) {
     // PASS
   }
@@ -647,8 +647,8 @@ namespace orc {
     data = dynamic_cast<const LongVectorBatch&>(batch).data.data();
   }
 
-  BooleanColumnPrinter::BooleanColumnPrinter(std::string& buffer
-                                             ): ColumnPrinter(buffer),
+  BooleanColumnPrinter::BooleanColumnPrinter(std::string& _buffer
+                                             ): ColumnPrinter(_buffer),
                                                 data(nullptr) {
     // PASS
   }
@@ -666,8 +666,8 @@ namespace orc {
     data = dynamic_cast<const LongVectorBatch&>(batch).data.data();
   }
 
-  BinaryColumnPrinter::BinaryColumnPrinter(std::string& buffer
-                                           ): ColumnPrinter(buffer),
+  BinaryColumnPrinter::BinaryColumnPrinter(std::string& _buffer
+                                           ): ColumnPrinter(_buffer),
                                               start(nullptr),
                                               length(nullptr) {
     // PASS
@@ -697,8 +697,8 @@ namespace orc {
     length = dynamic_cast<const StringVectorBatch&>(batch).length.data();
   }
 
-  TimestampColumnPrinter::TimestampColumnPrinter(std::string& buffer
-                                                 ): ColumnPrinter(buffer),
+  TimestampColumnPrinter::TimestampColumnPrinter(std::string& _buffer
+                                                 ): ColumnPrinter(_buffer),
                                                     seconds(nullptr),
                                                     nanoseconds(nullptr) {
     // PASS
diff --git a/c++/src/Reader.cc b/c++/src/Reader.cc
index a814fbc..ad87c15 100644
--- a/c++/src/Reader.cc
+++ b/c++/src/Reader.cc
@@ -346,10 +346,11 @@ namespace orc {
     // store position providers for selected colimns
     std::unordered_map<uint64_t, PositionProvider> positionProviders;
 
-    for (const auto& rowIndex : rowIndexes) {
-      uint64_t colId = rowIndex.first;
+    for (auto rowIndex = rowIndexes.cbegin();
+         rowIndex != rowIndexes.cend(); ++rowIndex) {
+      uint64_t colId = rowIndex->first;
       const proto::RowIndexEntry& entry =
-        rowIndex.second.entry(static_cast<int32_t>(rowGroupEntryId));
+        rowIndex->second.entry(static_cast<int32_t>(rowGroupEntryId));
 
       // copy index positions for a specific column
       positions.push_back({});
@@ -357,7 +358,7 @@ namespace orc {
       for (int pos = 0; pos != entry.positions_size(); ++pos) {
         position.push_back(entry.positions(pos));
       }
-      positionProviders.emplace(std::make_pair(colId, PositionProvider(position)));
+      positionProviders.insert(std::make_pair(colId, PositionProvider(position)));
     }
 
     reader->seekToRowGroup(positionProviders);
diff --git a/c++/src/Vector.cc b/c++/src/Vector.cc
index afb7cfa..14c0ded 100644
--- a/c++/src/Vector.cc
+++ b/c++/src/Vector.cc
@@ -61,9 +61,9 @@ namespace orc {
     return false;
   }
 
-  LongVectorBatch::LongVectorBatch(uint64_t capacity, MemoryPool& pool
-                     ): ColumnVectorBatch(capacity, pool),
-                        data(pool, capacity) {
+  LongVectorBatch::LongVectorBatch(uint64_t _capacity, MemoryPool& pool
+                     ): ColumnVectorBatch(_capacity, pool),
+                        data(pool, _capacity) {
     // PASS
   }
 
@@ -93,9 +93,9 @@ namespace orc {
         static_cast<uint64_t>(data.capacity() * sizeof(int64_t));
   }
 
-  DoubleVectorBatch::DoubleVectorBatch(uint64_t capacity, MemoryPool& pool
-                   ): ColumnVectorBatch(capacity, pool),
-                      data(pool, capacity) {
+  DoubleVectorBatch::DoubleVectorBatch(uint64_t _capacity, MemoryPool& pool
+                   ): ColumnVectorBatch(_capacity, pool),
+                      data(pool, _capacity) {
     // PASS
   }
 
@@ -131,10 +131,11 @@ namespace orc {
     // PASS
   }
 
-  EncodedStringVectorBatch::EncodedStringVectorBatch(uint64_t capacity, MemoryPool& pool)
-                      : StringVectorBatch(capacity, pool),
+  EncodedStringVectorBatch::EncodedStringVectorBatch(uint64_t _capacity,
+                                                     MemoryPool& pool)
+                      : StringVectorBatch(_capacity, pool),
                         dictionary(),
-                        index(pool, capacity) {
+                        index(pool, _capacity) {
     // PASS
   }
 
@@ -148,10 +149,10 @@ namespace orc {
     return buffer.str();
   }
 
-  StringVectorBatch::StringVectorBatch(uint64_t capacity, MemoryPool& pool
-               ): ColumnVectorBatch(capacity, pool),
-                  data(pool, capacity),
-                  length(pool, capacity),
+  StringVectorBatch::StringVectorBatch(uint64_t _capacity, MemoryPool& pool
+               ): ColumnVectorBatch(_capacity, pool),
+                  data(pool, _capacity),
+                  length(pool, _capacity),
                   blob(pool) {
     // PASS
   }
@@ -478,12 +479,12 @@ namespace orc {
     return value.toDecimalString(scale);
   }
 
-  TimestampVectorBatch::TimestampVectorBatch(uint64_t capacity,
+  TimestampVectorBatch::TimestampVectorBatch(uint64_t _capacity,
                                              MemoryPool& pool
-                                             ): ColumnVectorBatch(capacity,
+                                             ): ColumnVectorBatch(_capacity,
                                                                   pool),
-                                                data(pool, capacity),
-                                                nanoseconds(pool, capacity) {
+                                                data(pool, _capacity),
+                                                nanoseconds(pool, _capacity) {
     // PASS
   }
 
diff --git a/c++/test/TestColumnReader.cc b/c++/test/TestColumnReader.cc
index 9444616..9f8d8b8 100644
--- a/c++/test/TestColumnReader.cc
+++ b/c++/test/TestColumnReader.cc
@@ -33,6 +33,9 @@
   DIAGNOSTIC_IGNORE("-Winconsistent-missing-override")
   DIAGNOSTIC_IGNORE("-Wmissing-variable-declarations")
 #endif
+#ifdef __GNUC__
+  DIAGNOSTIC_IGNORE("-Wparentheses")
+#endif
 
 namespace orc {
   using ::testing::TestWithParam;
diff --git a/cmake_modules/FindSnappy.cmake b/cmake_modules/FindSnappy.cmake
index 6cf64be..5aeb88e 100644
--- a/cmake_modules/FindSnappy.cmake
+++ b/cmake_modules/FindSnappy.cmake
@@ -30,7 +30,7 @@ find_path (SNAPPY_INCLUDE_DIR snappy.h HINTS
 
 find_library (SNAPPY_LIBRARIES NAMES snappy HINTS
   ${_snappy_path}
-  PATH_SUFFIXES "lib")
+  PATH_SUFFIXES "lib" "lib64")
 
 if (SNAPPY_INCLUDE_DIR AND SNAPPY_LIBRARIES)
   set (SNAPPY_FOUND TRUE)
diff --git a/cmake_modules/ThirdpartyToolchain.cmake b/cmake_modules/ThirdpartyToolchain.cmake
index 74f0e61..b70540b 100644
--- a/cmake_modules/ThirdpartyToolchain.cmake
+++ b/cmake_modules/ThirdpartyToolchain.cmake
@@ -67,7 +67,7 @@ else ()
   set(SNAPPY_INCLUDE_DIR "${SNAPPY_HOME}/include")
   set(SNAPPY_STATIC_LIB "${SNAPPY_HOME}/lib/${CMAKE_STATIC_LIBRARY_PREFIX}snappy${CMAKE_STATIC_LIBRARY_SUFFIX}")
   set(SNAPPY_CMAKE_ARGS -DCMAKE_INSTALL_PREFIX=${SNAPPY_HOME}
-                        -DBUILD_SHARED_LIBS=OFF)
+                        -DBUILD_SHARED_LIBS=OFF -DCMAKE_INSTALL_LIBDIR=lib)
 
   ExternalProject_Add (snappy_ep
     URL "https://github.com/google/snappy/archive/${SNAPPY_VERSION}.tar.gz"


[orc] 03/04: ORC-414: [C++] Check root type existence before reading files

Posted by om...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

omalley pushed a commit to branch branch-1.6
in repository https://gitbox.apache.org/repos/asf/orc.git

commit 0635bb34332e2abaed1fbe3673b35506463453b5
Author: stiga-huang <hu...@gmail.com>
AuthorDate: Sat Oct 19 21:51:10 2019 +0800

    ORC-414: [C++] Check root type existence before reading files
    
    Fixes #438
    
    Signed-off-by: Owen O'Malley <om...@apache.org>
---
 c++/src/Reader.cc    | 3 +++
 c++/test/TestType.cc | 2 ++
 2 files changed, 5 insertions(+)

diff --git a/c++/src/Reader.cc b/c++/src/Reader.cc
index ad87c15..ef39a58 100644
--- a/c++/src/Reader.cc
+++ b/c++/src/Reader.cc
@@ -1014,6 +1014,9 @@ namespace orc {
   void checkProtoTypeIds(const proto::Footer &footer) {
     std::stringstream msg;
     int maxId = footer.types_size();
+    if (maxId <= 0) {
+      throw ParseError("Footer is corrupt: no types found");
+    }
     for (int i = 0; i < maxId; ++i) {
       const proto::Type& type = footer.types(i);
       for (int j = 0; j < type.subtypes_size(); ++j) {
diff --git a/c++/test/TestType.cc b/c++/test/TestType.cc
index 4abfaf1..5d61f6f 100644
--- a/c++/test/TestType.cc
+++ b/c++/test/TestType.cc
@@ -355,6 +355,8 @@ namespace orc {
   TEST(TestType, testCheckProtoTypeIds) {
     proto::Footer footer;
     proto::Type rootType;
+    expectParseError(footer, "Footer is corrupt: no types found");
+
     rootType.set_kind(proto::Type_Kind_STRUCT);
     rootType.add_subtypes(1); // add a non existent type id
     *(footer.add_types()) = rootType;