You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by je...@apache.org on 2019/03/20 17:33:20 UTC

[incubator-pinot] branch orc updated: Copying orc reader to pinot core for now so I don't have to edit the pinot script internally to publish jars

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

jenniferdai pushed a commit to branch orc
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git


The following commit(s) were added to refs/heads/orc by this push:
     new 41c6bf7  Copying orc reader to pinot core for now so I don't have to edit the pinot script internally to publish jars
41c6bf7 is described below

commit 41c6bf7be17191d13a23da6f49c3443a8fda4ca5
Author: Jennifer Dai <jd...@linkedin.com>
AuthorDate: Wed Mar 20 10:33:09 2019 -0700

    Copying orc reader to pinot core for now so I don't have to edit the pinot script internally to publish jars
---
 .../java/org/apache/pinot/common/data/Schema.java  |   2 +-
 pinot-core/pom.xml                                 |   8 +
 .../pinot/orc/data/readers/ORCRecordReader.java    | 186 +++++++++++++++++++++
 3 files changed, 195 insertions(+), 1 deletion(-)

diff --git a/pinot-common/src/main/java/org/apache/pinot/common/data/Schema.java b/pinot-common/src/main/java/org/apache/pinot/common/data/Schema.java
index 2db6475..f06b14c 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/data/Schema.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/data/Schema.java
@@ -137,7 +137,7 @@ public final class Schema {
   public List<MetricFieldSpec> getMetricFieldSpecs() {
     return _metricFieldSpecs;
   }
-  
+
   /**
    * Required by JSON deserializer. DO NOT USE. DO NOT REMOVE.
    * Adding @Deprecated to prevent usage
diff --git a/pinot-core/pom.xml b/pinot-core/pom.xml
index 8f4efe2..82a1fb3 100644
--- a/pinot-core/pom.xml
+++ b/pinot-core/pom.xml
@@ -213,5 +213,13 @@
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-math</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.orc</groupId>
+      <artifactId>orc-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.orc</groupId>
+      <artifactId>orc-mapreduce</artifactId>
+    </dependency>
   </dependencies>
 </project>
diff --git a/pinot-core/src/main/java/org/apache/pinot/orc/data/readers/ORCRecordReader.java b/pinot-core/src/main/java/org/apache/pinot/orc/data/readers/ORCRecordReader.java
new file mode 100644
index 0000000..046571a
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/orc/data/readers/ORCRecordReader.java
@@ -0,0 +1,186 @@
+package org.apache.pinot.orc.data.readers;
+
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.ByteWritable;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.ShortWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.orc.OrcFile;
+import org.apache.orc.Reader;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.mapred.OrcList;
+import org.apache.orc.mapred.OrcMapredRecordReader;
+import org.apache.pinot.common.data.Schema;
+import org.apache.pinot.core.data.GenericRow;
+import org.apache.pinot.core.data.readers.RecordReader;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ORCRecordReader implements RecordReader {
+
+  private Schema _pinotSchema;
+  private TypeDescription _orcSchema;
+  Reader _reader;
+  org.apache.orc.RecordReader _recordReader;
+  VectorizedRowBatch _reusableVectorizedRowBatch = new VectorizedRowBatch(1);
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(ORCRecordReader.class);
+
+  @Override
+  public void init(SegmentGeneratorConfig segmentGeneratorConfig) {
+    Configuration conf = new Configuration();
+    LOGGER.info("Creating segment for {}", segmentGeneratorConfig.getInputFilePath());
+    try {
+      Path orcReaderPath = new Path("file://" + segmentGeneratorConfig.getInputFilePath());
+      LOGGER.info("orc reader path is {}", orcReaderPath);
+      _reader = OrcFile.createReader(orcReaderPath, OrcFile.readerOptions(conf));
+      _orcSchema = _reader.getSchema();
+      LOGGER.info("ORC schema is {}", _orcSchema.toJson());
+
+      _pinotSchema = segmentGeneratorConfig.getSchema();
+      if (_pinotSchema == null) {
+        throw new IllegalArgumentException("ORCRecordReader requires schema");
+      }
+      _recordReader = _reader.rows(_reader.options().schema(_orcSchema));
+    } catch (Exception e) {
+      LOGGER.error("Caught exception initializing record reader at path {}", segmentGeneratorConfig.getInputFilePath());
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public boolean hasNext() {
+    try {
+      return _recordReader.getProgress() != 1;
+    } catch (IOException e) {
+      LOGGER.error("Could not get next record");
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public GenericRow next()
+      throws IOException {
+    return next(new GenericRow());
+  }
+
+  @Override
+  public GenericRow next(GenericRow reuse)
+      throws IOException {
+    _reusableVectorizedRowBatch = _orcSchema.createRowBatch(1);
+    _recordReader.nextBatch(_reusableVectorizedRowBatch);
+    fillGenericRow(reuse, _reusableVectorizedRowBatch);
+    return reuse;
+  }
+
+  private void fillGenericRow(GenericRow genericRow, VectorizedRowBatch rowBatch) throws IOException {
+    // Read the row data
+    TypeDescription schema = _reader.getSchema();
+    // Create a row batch with max size 1
+
+    if (schema.getCategory().equals(TypeDescription.Category.STRUCT)) {
+      for (int i = 0; i < schema.getChildren().size(); i++) {
+        // Get current column in schema
+        TypeDescription currColumn = schema.getChildren().get(i);
+        String currColumnName = currColumn.getFieldNames().get(0);
+        int currColRowIndex = currColumn.getId();
+        ColumnVector vector = rowBatch.cols[currColRowIndex];
+        // Previous value set to null, not used except to save allocation memory in OrcMapredRecordReader
+        WritableComparable writableComparable = OrcMapredRecordReader.nextValue(vector, currColRowIndex, _orcSchema, null);
+        genericRow.putField(currColumnName, getBaseObject(writableComparable));
+      }
+    } else {
+      throw new IllegalArgumentException("Not a valid schema");
+    }
+  }
+
+  /**
+   * A utility method to convert an Orc WritableComparable object to a generic Java object that can
+   * be added to a Pinot GenericRow object
+   *
+   * @param w Orc WritableComparable to convert
+   * @return Object that will be added to the Pinot GenericRow
+   */
+  private Object getBaseObject(WritableComparable w) {
+    Object obj = null;
+
+    if (w == null || NullWritable.class.isAssignableFrom(w.getClass())) {
+      obj = null;
+    } else if (BooleanWritable.class.isAssignableFrom(w.getClass())) {
+      obj = ((BooleanWritable) w).get();
+    } else if (ByteWritable.class.isAssignableFrom(w.getClass())) {
+      obj = ((ByteWritable) w).get();
+    } else if (ShortWritable.class.isAssignableFrom(w.getClass())) {
+      obj = ((ShortWritable) w).get();
+    } else if (IntWritable.class.isAssignableFrom(w.getClass())) {
+      obj = ((IntWritable) w).get();
+    } else if (LongWritable.class.isAssignableFrom(w.getClass())) {
+      obj = ((LongWritable) w).get();
+    } else if (FloatWritable.class.isAssignableFrom(w.getClass())) {
+      obj = ((FloatWritable) w).get();
+    } else if (DoubleWritable.class.isAssignableFrom(w.getClass())) {
+      obj = ((DoubleWritable) w).get();
+    } else if (BytesWritable.class.isAssignableFrom(w.getClass())) {
+      obj = ((BytesWritable) w).getBytes();
+    } else if (Text.class.isAssignableFrom(w.getClass())) {
+      obj = ((Text) w).toString();
+    } else if (OrcList.class.isAssignableFrom(w.getClass())) {
+      // TODO: This is probably multivalue columns
+      LOGGER.info("Skipping unsupported type: list");
+    } else {
+      LOGGER.info("Unknown type found: " + w.getClass().getSimpleName());
+      throw new IllegalArgumentException("Unknown type: " + w.getClass().getSimpleName());
+    }
+
+    return obj;
+  }
+
+  @Override
+  public void rewind()
+      throws IOException {
+    _recordReader = _reader.rows();
+  }
+
+  @Override
+  public Schema getSchema() {
+    return _pinotSchema;
+  }
+
+  @Override
+  public void close()
+      throws IOException {
+    _recordReader.close();
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org