Skip to content
Closed
Show file tree
Hide file tree
Changes from 5 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,93 @@
/*
* 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 org.apache.hadoop.hive.common.`type`.HiveVarchar
import org.apache.spark.sql.hive.{HiveInspectors, HiveShim}
import org.apache.hadoop.hive.serde2.objectinspector._
import org.apache.hadoop.hive.serde2.objectinspector.primitive._
import org.apache.spark.sql.catalyst.expressions.{Row, MutableRow}

import scala.collection.JavaConversions._

/**
* We can consolidate TableReader.unwrappers and HiveInspectors.wrapperFor to use
* this class.
*
*/
private[hive] object HadoopTypeConverter extends HiveInspectors {
/**
* Builds specific unwrappers ahead of time according to object inspector
* types to avoid pattern matching and branching costs per row.
*/
def unwrappers(fieldRefs: Seq[StructField]): Seq[(Any, MutableRow, Int) => Unit] = fieldRefs.map {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Change TableReader.unwrapper method to remove this duplicated code?

_.getFieldObjectInspector match {
case oi: BooleanObjectInspector =>
(value: Any, row: MutableRow, ordinal: Int) => row.setBoolean(ordinal, oi.get(value))
case oi: ByteObjectInspector =>
(value: Any, row: MutableRow, ordinal: Int) => row.setByte(ordinal, oi.get(value))
case oi: ShortObjectInspector =>
(value: Any, row: MutableRow, ordinal: Int) => row.setShort(ordinal, oi.get(value))
case oi: IntObjectInspector =>
(value: Any, row: MutableRow, ordinal: Int) => row.setInt(ordinal, oi.get(value))
case oi: LongObjectInspector =>
(value: Any, row: MutableRow, ordinal: Int) => row.setLong(ordinal, oi.get(value))
case oi: FloatObjectInspector =>
(value: Any, row: MutableRow, ordinal: Int) => row.setFloat(ordinal, oi.get(value))
case oi: DoubleObjectInspector =>
(value: Any, row: MutableRow, ordinal: Int) => row.setDouble(ordinal, oi.get(value))
case oi =>
(value: Any, row: MutableRow, ordinal: Int) => row(ordinal) = unwrap(value, oi)
}
}

/**
* Wraps with Hive types based on object inspector.
*/
def wrappers(oi: ObjectInspector): Any => Any = oi match {
case _: JavaHiveVarcharObjectInspector =>
(o: Any) => new HiveVarchar(o.asInstanceOf[String], o.asInstanceOf[String].size)

case _: JavaHiveDecimalObjectInspector =>
(o: Any) => HiveShim.createDecimal(o.asInstanceOf[BigDecimal].underlying())

case soi: StandardStructObjectInspector =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe use SettableStructObjectInspector instead?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will reuse wrapperFor directly in the next push to remove this part of code totally.

val wrappers = soi.getAllStructFieldRefs.map(ref => wrapperFor(ref.getFieldObjectInspector))
(o: Any) => {
val struct = soi.create()
(soi.getAllStructFieldRefs, wrappers, o.asInstanceOf[Row].toSeq).zipped.foreach {
(field, wrapper, data) => soi.setStructFieldData(struct, field, wrapper(data))
}
struct
}

case loi: ListObjectInspector =>
val wrapper = wrapperFor(loi.getListElementObjectInspector)
(o: Any) => seqAsJavaList(o.asInstanceOf[Seq[_]].map(wrapper))

case moi: MapObjectInspector =>
val keyWrapper = wrapperFor(moi.getMapKeyObjectInspector)
val valueWrapper = wrapperFor(moi.getMapValueObjectInspector)
(o: Any) => mapAsJavaMap(o.asInstanceOf[Map[_, _]].map { case (key, value) =>
keyWrapper(key) -> valueWrapper(value)
})

case _ =>
identity[Any]
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,87 @@
/*
* 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 org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.hive.ql.io.orc.{OrcFile, Reader}
import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector
import org.apache.spark.Logging
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.sql.hive.HiveMetastoreTypes
import org.apache.spark.sql.types.StructType

private[orc] object OrcFileOperator extends Logging{

def getFileReader(pathStr: String, config: Option[Configuration] = None ): Reader = {
var conf = config.getOrElse(new Configuration)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

var => val

val fspath = new Path(pathStr)
val fs = fspath.getFileSystem(conf)
val orcFiles = listOrcFiles(pathStr, conf)
OrcFile.createReader(fs, orcFiles(0))
}

def readSchema(path: String, conf: Option[Configuration]): StructType = {
val reader = getFileReader(path, conf)
val readerInspector: StructObjectInspector = reader.getObjectInspector
.asInstanceOf[StructObjectInspector]
val schema = readerInspector.getTypeName
HiveMetastoreTypes.toDataType(schema).asInstanceOf[StructType]
}

def getObjectInspector(path: String, conf: Option[Configuration]): StructObjectInspector = {
val reader = getFileReader(path, conf)
val readerInspector: StructObjectInspector = reader.getObjectInspector
.asInstanceOf[StructObjectInspector]
readerInspector
}

def deletePath(pathStr: String, conf: Configuration): Unit = {
val fspath = new Path(pathStr)
val fs = fspath.getFileSystem(conf)
try {
fs.delete(fspath, true)
} catch {
case e: IOException =>
throw new IOException(
s"Unable to clear output directory ${fspath.toString} prior"
+ s" to InsertIntoOrcTable:\n${e.toString}")
}
}

def listOrcFiles(pathStr: String, conf: Configuration): Seq[Path] = {
val origPath = new Path(pathStr)
val fs = origPath.getFileSystem(conf)
val path = origPath.makeQualified(fs)
val paths = SparkHadoopUtil.get.listLeafStatuses(fs, origPath)
.filterNot(_.isDir)
.map(_.getPath)
.filterNot(_.getName.startsWith("_"))
.filterNot(_.getName.startsWith("."))

if (paths == null || paths.size == 0) {
throw new IllegalArgumentException(
s"orcFileOperator: path $path does not have valid orc files matching the pattern")
}
logInfo("Qualified file list: ")
paths.foreach{x=>logInfo(x.toString)}
paths
}
}
121 changes: 121 additions & 0 deletions sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,121 @@
/*
* 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 org.apache.hadoop.hive.ql.io.sarg.SearchArgument
import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.Builder
import org.apache.spark.Logging
import org.apache.spark.sql.sources._

private[sql] object OrcFilters extends Logging {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

private[orc]?


def createFilter(expr: Array[Filter]): Option[SearchArgument] = {
if (expr == null || expr.size == 0) return None
var sarg: Option[Builder] = Some(SearchArgument.FACTORY.newBuilder())
sarg.get.startAnd()
expr.foreach {
x => {
sarg match {
case Some(s1) => sarg = createFilter(x, s1)
case _ => None
}
}
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: we prefer

expr.foreach { x =>
  sarg match {
     ...
  }
}

The braces around the match expression is redundant.

sarg match {
case Some(b) => Some(b.end.build)
case _ => None
}
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This method can be simplified to:

  def createFilter(expr: Array[Filter]): Option[SearchArgument] = {
    expr.headOption.map { _ =>
      val andBuilder = SearchArgument.FACTORY.newBuilder()
      expr.foldLeft(andBuilder.startAnd()) { (builder, e) =>
        createFilter(e, builder).getOrElse(builder)
      }.end().build()
    }
  }

(I omitted the null checking since expr shouldn't be nullable along the code path).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually let's make it less cryptic...

  def createFilter(expr: Array[Filter]): Option[SearchArgument] = {
    if (expr.nonEmpty) {
      val and = expr.foldLeft(SearchArgument.FACTORY.newBuilder().startAnd()) {
        (builder, e) => createFilter(e, builder).getOrElse(builder)
      }.end()

      Some(and.build())
    } else {
      None
    }
  }


def createFilter(expression: Filter, builder: Builder): Option[Builder] = {
expression match {
case p@And(left: Filter, right: Filter) => {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: In general we don't use braces for case clauses.

val b1 = builder.startAnd()
val b2 = createFilter(left, b1)
b2 match {
case Some(b) => val b3 = createFilter(right, b)
if (b3.isDefined) {
Some(b3.get.end)
} else {
None
}
case _ => None
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This clause can be simplified to:

        val and = for {
          lhs <- createFilter(left, builder.startAnd())
          rhs <- createFilter(right, lhs)
        } yield rhs.end()

        and
          .orElse(createFilter(right, builder))
          .orElse(createFilter(left, builder))

The rationale behind the second part is that, for filter expression A && B, we can still push down A even if B is not convertible, and vice versa. This is similar to what we've done in ParquetFilters and InMemoryColumnarTableScan.

}
case p@Or(left: Filter, right: Filter) => {
val b1 = builder.startOr()
val b2 = createFilter(left, b1)
b2 match {
case Some(b) => val b3 = createFilter(right, b)
if (b3.isDefined) {
Some(b3.get.end)
} else {
None
}
case _ => None
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Similarly, this clause can be simplified to

        for {
          lhs <- createFilter(left, builder.startOr())
          rhs <- createFilter(right, lhs)
        } yield rhs.end()

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

(For Or expression, we must ensure both sides are convertible.)

}
case p@Not(child: Filter) => {
val b1 = builder.startNot()
val b2 = createFilter(child, b1)
b2 match {
case Some(b) => Some(b.end)
case _ => None
}
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

        createFilter(child, builder.startNot()).map(_.end())

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Or maybe

        for {
          not <- createFilter(child, builder.startNot())
        } yield not.end()

Just to keep the style consistent with the other two clauses above.

case p@EqualTo(attribute: String, value: Any) => {
val b1 = builder.equals(attribute, value)
Some(b1)
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The temporary variable is redundant:

Some(builder.equals(attribute, value))

Also, we can simplify the pattern in case part to

case EqualTo(attribute, value) =>

These also apply to all other clauses in this method.

case p@LessThan(attribute: String, value: Any) => {
val b1 = builder.lessThan(attribute ,value)
Some(b1)
}
case p@LessThanOrEqual(attribute: String, value: Any) => {
val b1 = builder.lessThanEquals(attribute, value)
Some(b1)
}
case p@GreaterThan(attribute: String, value: Any) => {
val b1 = builder.startNot().lessThanEquals(attribute, value).end()
Some(b1)
}
case p@GreaterThanOrEqual(attribute: String, value: Any) => {
val b1 = builder.startNot().lessThan(attribute, value).end()
Some(b1)
}
case p@IsNull(attribute: String) => {
val b1 = builder.isNull(attribute)
Some(b1)
}
case p@IsNotNull(attribute: String) => {
val b1 = builder.startNot().isNull(attribute).end()
Some(b1)
}
case p@In(attribute: String, values: Array[Any]) => {
val b1 = builder.in(attribute, values)
Some(b1)
}
// not supported in filter
// case p@EqualNullSafe(left: String, right: String) => {
// val b1 = builder.nullSafeEquals(left, right)
// Some(b1)
// }
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's remove this comment.

case _ => None
}
}
}
Loading