Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2576#discussion_r18771511
  
    --- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcTableOperations.scala 
---
    @@ -0,0 +1,351 @@
    +/*
    + * 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.spark.sql.hive.orc
    +
    +import java.io.IOException
    +import java.text.SimpleDateFormat
    +import java.util.{Locale, Date}
    +import scala.collection.JavaConversions._
    +
    +import org.apache.hadoop.fs.Path
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat, 
FileOutputCommitter}
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
    +import org.apache.hadoop.io.{Writable, NullWritable}
    +import org.apache.hadoop.mapreduce.{TaskID, TaskAttemptContext, Job}
    +import org.apache.hadoop.hive.ql.io.orc.{OrcSerde, OrcInputFormat, 
OrcOutputFormat}
    +import org.apache.hadoop.hive.serde2.objectinspector._
    +import org.apache.hadoop.hive.serde2.ColumnProjectionUtils
    +import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar}
    +import org.apache.hadoop.mapred.{SparkHadoopMapRedUtil, Reporter, JobConf}
    +
    +import org.apache.spark.sql.execution._
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.parquet.FileSystemHelper
    +import org.apache.spark.{TaskContext, SerializableWritable}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.Utils._
    +
    +/**
    + * orc table scan operator. Imports the file that backs the given
    + * [[org.apache.spark.sql.hive.orc.OrcRelation]] as a ``RDD[Row]``.
    + */
    +case class OrcTableScan(
    +    output: Seq[Attribute],
    +    relation: OrcRelation,
    +    columnPruningPred: Option[Expression])
    +  extends LeafNode {
    +
    +  @transient
    +  lazy val serde: OrcSerde = initSerde
    +
    +  @transient
    +  lazy val getFieldValue: Seq[Product => Any] = {
    +    val inspector = 
serde.getObjectInspector.asInstanceOf[StructObjectInspector]
    +    output.map(attr => {
    +      val ref = 
inspector.getStructFieldRef(attr.name.toLowerCase(Locale.ENGLISH))
    +      row: Product => {
    +        val fieldData = row.productElement(1)
    +        val data = inspector.getStructFieldData(fieldData, ref)
    +        unwrapData(data, ref.getFieldObjectInspector)
    +      }
    +    })
    +  }
    +
    +  private def initSerde(): OrcSerde = {
    +    val serde = new OrcSerde
    +    serde.initialize(null, relation.prop)
    +    serde
    +  }
    +
    +  def unwrapData(data: Any, oi: ObjectInspector): Any = oi match {
    +    case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data)
    +    case li: ListObjectInspector =>
    +      Option(li.getList(data))
    +        .map(_.map(unwrapData(_, li.getListElementObjectInspector)).toSeq)
    +        .orNull
    +    case mi: MapObjectInspector =>
    +      Option(mi.getMap(data)).map(
    +        _.map {
    +          case (k, v) =>
    +            (unwrapData(k, mi.getMapKeyObjectInspector),
    +              unwrapData(v, mi.getMapValueObjectInspector))
    +        }.toMap).orNull
    +    case si: StructObjectInspector =>
    +      val allRefs = si.getAllStructFieldRefs
    +      new GenericRow(
    +        allRefs.map(r =>
    +          unwrapData(si.getStructFieldData(data, r), 
r.getFieldObjectInspector)).toArray)
    +  }
    +
    +  override def execute(): RDD[Row] = {
    +    val sc = sqlContext.sparkContext
    +    val job = new Job(sc.hadoopConfiguration)
    +
    +    val conf: Configuration = job.getConfiguration
    +    val fileList = FileSystemHelper.listFiles(relation.path, conf)
    +
    +    // add all paths in the directory but skip "hidden" ones such
    +    // as "_SUCCESS"
    +    for (path <- fileList if !path.getName.startsWith("_")) {
    +      FileInputFormat.addInputPath(job, path)
    +    }
    +
    +    setColumnIds(output, relation, conf)
    +    val inputClass = classOf[OrcInputFormat].asInstanceOf[
    +      Class[_ <: org.apache.hadoop.mapred.InputFormat[Void, Row]]]
    +
    +    val rowRdd = productToRowRdd(sc.hadoopRDD[Void, Row](
    +      conf.asInstanceOf[JobConf], inputClass, classOf[Void], classOf[Row]))
    +    rowRdd
    +  }
    +
    +  /**
    +   * @param output
    +   * @param relation
    +   * @param conf
    +   */
    +  def setColumnIds(output: Seq[Attribute], relation: OrcRelation, conf: 
Configuration) {
    +    val idBuff = new StringBuilder()
    +
    +    output.map(att => {
    +      val realName = att.name.toLowerCase(Locale.ENGLISH)
    +      val id = relation.fieldIdCache.getOrElse(realName, null)
    +      if (null != id) {
    +        idBuff.append(id)
    +        idBuff.append(",")
    +      }
    +    })
    +    if (idBuff.length > 0) {
    +      idBuff.setLength(idBuff.length - 1)
    +    }
    +    conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, 
idBuff.toString())
    +  }
    +
    +  /**
    +   *
    +   * @param data
    +   * @tparam A
    +   * @return
    +   */
    +  def productToRowRdd[A <: Product](data: RDD[A]): RDD[Row] = {
    +    data.mapPartitions {
    +      iterator =>
    +        if (iterator.isEmpty) {
    +          Iterator.empty
    +        } else {
    +          val bufferedIterator = iterator.buffered
    --- End diff --
    
    `BufferedIterator` is only useful when you need to "peek" the first element 
without consuming it. Here you don't need it and the `if`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to