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

[1/2] orc git commit: HIVE-14566: LLAP IO reads timestamp wrongly (Prasanth Jayachandran reviewed by Sergey Shelukhin)

Repository: orc
Updated Branches:
  refs/heads/master 4509a8b2d -> 7118e968b


HIVE-14566: LLAP IO reads timestamp wrongly (Prasanth Jayachandran reviewed by Sergey Shelukhin)

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


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

Branch: refs/heads/master
Commit: 195181fc7017f29931e7a7fc20d2f0099b8901e0
Parents: 4509a8b
Author: Owen O'Malley <om...@apache.org>
Authored: Sat Aug 27 19:34:35 2016 -0700
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Sep 6 21:06:59 2016 -0700

----------------------------------------------------------------------
 .../org/apache/orc/impl/TreeReaderFactory.java  |   5 +-
 .../test/org/apache/orc/TestOrcTimezone3.java   | 126 +++++++++++++++++++
 2 files changed, 129 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/orc/blob/195181fc/java/core/src/java/org/apache/orc/impl/TreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/java/core/src/java/org/apache/orc/impl/TreeReaderFactory.java b/java/core/src/java/org/apache/orc/impl/TreeReaderFactory.java
index 0c94f0e..1e1ae01 100644
--- a/java/core/src/java/org/apache/orc/impl/TreeReaderFactory.java
+++ b/java/core/src/java/org/apache/orc/impl/TreeReaderFactory.java
@@ -863,7 +863,7 @@ public class TreeReaderFactory {
     protected IntegerReader nanos = null;
     private final boolean skipCorrupt;
     private Map<String, Long> baseTimestampMap;
-    private long base_timestamp;
+    protected long base_timestamp;
     private final TimeZone readerTimeZone;
     private TimeZone writerTimeZone;
     private boolean hasSameTZRules;
@@ -896,6 +896,7 @@ public class TreeReaderFactory {
         if (nanosStream != null) {
           this.nanos = createIntegerReader(encoding.getKind(), nanosStream, false, context);
         }
+        base_timestamp = getBaseTimestamp(context.getWriterTimezone());
       }
     }
 
@@ -922,7 +923,7 @@ public class TreeReaderFactory {
       base_timestamp = getBaseTimestamp(stripeFooter.getWriterTimezone());
     }
 
-    private long getBaseTimestamp(String timeZoneId) throws IOException {
+    protected long getBaseTimestamp(String timeZoneId) throws IOException {
       // to make sure new readers read old files in the same way
       if (timeZoneId == null || timeZoneId.isEmpty()) {
         timeZoneId = readerTimeZone.getID();

http://git-wip-us.apache.org/repos/asf/orc/blob/195181fc/java/core/src/test/org/apache/orc/TestOrcTimezone3.java
----------------------------------------------------------------------
diff --git a/java/core/src/test/org/apache/orc/TestOrcTimezone3.java b/java/core/src/test/org/apache/orc/TestOrcTimezone3.java
new file mode 100644
index 0000000..40ab0c9
--- /dev/null
+++ b/java/core/src/test/org/apache/orc/TestOrcTimezone3.java
@@ -0,0 +1,126 @@
+/**
+ * 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.
+ */
+package org.apache.orc;
+
+import static junit.framework.Assert.assertEquals;
+
+import java.io.File;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.TimeZone;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import com.google.common.collect.Lists;
+
+import junit.framework.Assert;
+
+/**
+ *
+ */
+@RunWith(Parameterized.class)
+public class TestOrcTimezone3 {
+  Path workDir = new Path(System.getProperty("test.tmp.dir",
+      "target" + File.separator + "test" + File.separator + "tmp"));
+  Configuration conf;
+  FileSystem fs;
+  Path testFilePath;
+  String writerTimeZone;
+  String readerTimeZone;
+  static TimeZone defaultTimeZone = TimeZone.getDefault();
+
+  public TestOrcTimezone3(String writerTZ, String readerTZ) {
+    this.writerTimeZone = writerTZ;
+    this.readerTimeZone = readerTZ;
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    List<Object[]> result = Arrays.asList(new Object[][]{
+        {"America/Chicago", "America/Los_Angeles"},
+    });
+    return result;
+  }
+
+  @Rule
+  public TestName testCaseName = new TestName();
+
+  @Before
+  public void openFileSystem() throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    testFilePath = new Path(workDir, "TestOrcTimezone3." +
+        testCaseName.getMethodName() + ".orc");
+    fs.delete(testFilePath, false);
+  }
+
+  @After
+  public void restoreTimeZone() {
+    TimeZone.setDefault(defaultTimeZone);
+  }
+
+  @Test
+  public void testTimestampWriter() throws Exception {
+    TypeDescription schema = TypeDescription.createTimestamp();
+
+    TimeZone.setDefault(TimeZone.getTimeZone(writerTimeZone));
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf).setSchema(schema).stripeSize(100000)
+            .bufferSize(10000));
+    assertEquals(writerTimeZone, TimeZone.getDefault().getID());
+    List<String> ts = Lists.newArrayList();
+    ts.add("1969-12-31 16:00:14.007");
+    ts.add("1969-12-31 16:00:06.021");
+    ts.add("1969-12-31 16:00:03.963");
+    VectorizedRowBatch batch = schema.createRowBatch();
+    TimestampColumnVector times = (TimestampColumnVector) batch.cols[0];
+    for (String t : ts) {
+      times.set(batch.size++, Timestamp.valueOf(t));
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    TimeZone.setDefault(TimeZone.getTimeZone(readerTimeZone));
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    assertEquals(readerTimeZone, TimeZone.getDefault().getID());
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    times = (TimestampColumnVector) batch.cols[0];
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(ts.get(idx++), times.asScratchTimestamp(r).toString());
+      }
+    }
+    rows.close();
+  }
+}


[2/2] orc git commit: ORC-97 Add support for nested column id selection. (Chunyang Wen reviewed by omalley)

Posted by om...@apache.org.
ORC-97 Add support for nested column id selection. (Chunyang Wen reviewed by
omalley)

Fixes #59

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


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

Branch: refs/heads/master
Commit: 7118e968bc6a20ced8fbf8b75f68fdb919edc81d
Parents: 195181f
Author: wenchunyang <we...@baidu.com>
Authored: Wed Aug 10 10:04:21 2016 +0800
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Sep 6 21:08:00 2016 -0700

----------------------------------------------------------------------
 c++/src/Reader.cc       | 72 +++++++++++++++++++++++++++++++++++---------
 tools/test/TestMatch.cc | 25 +++++++++++++++
 2 files changed, 82 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/orc/blob/7118e968/c++/src/Reader.cc
----------------------------------------------------------------------
diff --git a/c++/src/Reader.cc b/c++/src/Reader.cc
index 184adf4..9b1f1b9 100644
--- a/c++/src/Reader.cc
+++ b/c++/src/Reader.cc
@@ -35,6 +35,7 @@
 #include <sstream>
 #include <string>
 #include <vector>
+#include <iterator>
 #include <set>
 
 namespace orc {
@@ -79,9 +80,9 @@ namespace orc {
 
   enum ColumnSelection {
     ColumnSelection_NONE = 0,
-    ColumnSelection_FIELD_NAMES = 1,
+    ColumnSelection_NAMES = 1,
     ColumnSelection_FIELD_IDS = 2,
-    ColumnSelection_TYPE_IDS = 3
+    ColumnSelection_TYPE_IDS = 3,
   };
 
   struct ReaderOptionsPrivate {
@@ -147,7 +148,7 @@ namespace orc {
   }
 
   ReaderOptions& ReaderOptions::include(const std::list<std::string>& include) {
-    privateBits->selection = ColumnSelection_FIELD_NAMES;
+    privateBits->selection = ColumnSelection_NAMES;
     privateBits->includedColumnNames.assign(include.begin(), include.end());
     privateBits->includedColumnIndexes.clear();
     return *this;
@@ -200,7 +201,7 @@ namespace orc {
   }
 
   bool ReaderOptions::getNamesSet() const {
-    return privateBits->selection == ColumnSelection_FIELD_NAMES;
+    return privateBits->selection == ColumnSelection_NAMES;
   }
 
   const std::list<std::string>& ReaderOptions::getIncludeNames() const {
@@ -1122,6 +1123,8 @@ namespace orc {
     proto::StripeInformation currentStripeInfo;
     proto::StripeFooter currentStripeFooter;
     std::unique_ptr<ColumnReader> reader;
+    std::map<std::string, uint64_t> nameIdMap;
+    std::map<uint64_t, const Type*> idTypeMap;
 
     // internal methods
     proto::StripeFooter getStripeFooter(const proto::StripeInformation& info);
@@ -1129,6 +1132,10 @@ namespace orc {
     void checkOrcVersion();
     void readMetadata() const;
 
+    // build map from type name and id, id to Type
+    void buildTypeNameIdMap(const Type* type, std::vector<std::string>& columns);
+    std::string toDotColumnPath(const std::vector<std::string>& columns);
+
     // Select the columns from the options object
     void updateSelected();
 
@@ -1305,6 +1312,8 @@ namespace orc {
     }
 
     schema = convertType(footer->types(0), *footer);
+    std::vector<std::string> columns;
+    buildTypeNameIdMap(schema.get(), columns);
     updateSelected();
   }
 
@@ -1330,6 +1339,7 @@ namespace orc {
       std::fill(selectedColumns.begin(), selectedColumns.end(), true);
     }
     selectParents(*schema);
+    selectedColumns[0] = true; // column 0 is selected by default
   }
 
   std::string ReaderImpl::getSerializedFileTail() const {
@@ -2256,7 +2266,8 @@ namespace orc {
 
   void ReaderImpl::updateSelectedByTypeId(uint64_t typeId) {
     if (typeId < selectedColumns.size()) {
-      selectedColumns[typeId] = true;
+      const Type& type = *idTypeMap[typeId];
+      selectChildren(type);
     } else {
       std::stringstream buffer;
       buffer << "Invalid type id selected " << typeId << " out of "
@@ -2266,13 +2277,12 @@ namespace orc {
   }
 
   void ReaderImpl::updateSelectedByName(const std::string& fieldName) {
-    for(size_t field=0; field < schema->getSubtypeCount(); ++field) {
-      if (schema->getFieldName(field) == fieldName) {
-        selectChildren(*schema->getSubtype(field));
-        return;
-      }
+    std::map<std::string, uint64_t>::const_iterator ite = nameIdMap.find(fieldName);
+    if (ite != nameIdMap.end()) {
+      updateSelectedByTypeId(ite->second);
+    } else {
+      throw ParseError("Invalid column selected " + fieldName);
     }
-    throw ParseError("Invalid column selected " + fieldName);
   }
 
   void ReaderImpl::selectChildren(const Type& type) {
@@ -2291,10 +2301,7 @@ namespace orc {
    */
   bool ReaderImpl::selectParents(const Type& type) {
     size_t id = static_cast<size_t>(type.getColumnId());
-    if (selectedColumns[id]) {
-      return true;
-    }
-    bool result = false;
+    bool result = selectedColumns[id];
     for(uint64_t c=0; c < type.getSubtypeCount(); ++c) {
       result |= selectParents(*type.getSubtype(c));
     }
@@ -2302,4 +2309,39 @@ namespace orc {
     return result;
   }
 
+  /**
+   * Recurses over a type tree and build two maps
+   * map<TypeName, TypeId>, map<TypeId, Type>
+   */
+  void ReaderImpl::buildTypeNameIdMap(const Type* type, std::vector<std::string>& columns) {
+    // map<type_id, Type*>
+    idTypeMap[type->getColumnId()] = type;
+
+    if (orc::STRUCT == type->getKind()) {
+      for (size_t i = 0; i < type->getSubtypeCount(); ++i) {
+        const std::string& fieldName = type->getFieldName(i);
+        columns.push_back(fieldName);
+        nameIdMap[toDotColumnPath(columns)] = type->getSubtype(i)->getColumnId();
+        buildTypeNameIdMap(type->getSubtype(i), columns);
+        columns.pop_back();
+      }
+    } else {
+      // other non-primitive type
+      for (size_t j = 0; j < type->getSubtypeCount(); ++j) {
+        buildTypeNameIdMap(type->getSubtype(j), columns);
+      }
+    }
+  }
+
+  std::string ReaderImpl::toDotColumnPath(const std::vector<std::string>& columns) {
+      if (columns.empty()) {
+          return std::string();
+      }
+      std::ostringstream columnStream;
+      std::copy(columns.begin(), columns.end(),
+              std::ostream_iterator<std::string>(columnStream, "."));
+      std::string columnPath = columnStream.str();
+      return columnPath.substr(0, columnPath.length() - 1);
+  }
+
 }// namespace

http://git-wip-us.apache.org/repos/asf/orc/blob/7118e968/tools/test/TestMatch.cc
----------------------------------------------------------------------
diff --git a/tools/test/TestMatch.cc b/tools/test/TestMatch.cc
index 9f0c297..3f754ca 100644
--- a/tools/test/TestMatch.cc
+++ b/tools/test/TestMatch.cc
@@ -1064,6 +1064,31 @@ TEST(TestMatch, selectColumns) {
         << "\"887336a7\", \"value\": {\"int1\": -941468492, \"string1\": "
         << "\"887336a7\"}}]}";
     EXPECT_EQ(expectedMapWithColumnId.str(), line);
+
+    // Struct column #10, with field name: middle
+    std::list<std::string> colNames;
+    colNames.push_back("middle.list.int1");
+    colNames.push_back("middle.list.string1");
+    opts.include(colNames);
+    reader = orc::createReader(orc::readLocalFile(filename), opts);
+    c = reader->getSelectedColumns();
+    for (unsigned int i=1; i < c.size(); i++) {
+      if (i>=10 && i<=14)
+        EXPECT_TRUE(c[i]);
+      else
+        EXPECT_TRUE(!c[i]);
+    }
+    batch = reader->createRowBatch(1);
+    line.clear();
+    printer = createColumnPrinter(line, &reader->getSelectedType());
+    reader->next(*batch);
+    printer->reset(*batch);
+    printer->printRow(0);
+    std::ostringstream expectedStructWithColumnName;
+    expectedStructWithColumnName << "{\"middle\": {\"list\": "
+        << "[{\"int1\": -941468492, \"string1\": \"887336a7\"}, "
+        << "{\"int1\": -1598014431, \"string1\": \"ba419d35-x\"}]}}";
+    EXPECT_EQ(expectedStructWithColumnName.str(), line);
 }
 
 TEST(TestMatch, memoryUse) {