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

    https://github.com/apache/spark/pull/993#discussion_r15333352
  
    --- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
    @@ -0,0 +1,458 @@
    +/*
    + * 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.catalyst.expressions.codegen
    +
    +import com.google.common.cache.{CacheLoader, CacheBuilder}
    +
    +import scala.language.existentials
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.sql.catalyst.expressions
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.types._
    +
    +/**
    + * A base class for generators of byte code to perform expression 
evaluation.  Includes a set of
    + * helpers for referring to Catalyst types and building trees that perform 
evaluation of individual
    + * expressions.
    + */
    +abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends 
Logging {
    +  import scala.reflect.runtime.{universe => ru}
    +  import scala.reflect.runtime.universe._
    +
    +  import scala.tools.reflect.ToolBox
    +
    +  protected val toolBox = 
runtimeMirror(getClass.getClassLoader).mkToolBox()
    +
    +  protected val rowType = typeOf[Row]
    +  protected val mutableRowType = typeOf[MutableRow]
    +  protected val genericRowType = typeOf[GenericRow]
    +  protected val genericMutableRowType = typeOf[GenericMutableRow]
    +
    +  protected val projectionType = typeOf[Projection]
    +  protected val mutableProjectionType = typeOf[MutableProjection]
    +
    +  private val curId = new java.util.concurrent.atomic.AtomicInteger()
    +  private val javaSeparator = "$"
    +
    +  /**
    +   * Generates a class for a given input expression.  Called when there is 
not cached code
    +   * already available.
    +   */
    +  protected def create(in: InType): OutType
    +
    +  /**
    +   * Canonicalizes an input expression. Used to avoid double caching 
expressions that differ only
    +   * cosmetically.
    +   */
    +  protected def canonicalize(in: InType): InType
    +
    +  /** Binds an input expression to a given input schema */
    +  protected def bind(in: InType, inputSchema: Seq[Attribute]): InType
    +
    +  protected val cache = CacheBuilder.newBuilder()
    +    .maximumSize(1000)
    +    .build(
    +      new CacheLoader[InType, OutType]() {
    +        override def load(in: InType): OutType = globalLock.synchronized {
    +           create(in)
    +        }
    +      })
    +
    +  def apply(expressions: InType, inputSchema: Seq[Attribute]): OutType=
    +    apply(bind(expressions, inputSchema))
    +
    +  def apply(expressions: InType): OutType = 
cache.get(canonicalize(expressions))
    +
    +  /**
    +   * Returns a term name that is unique within this instance of a 
`CodeGenerator`.
    +   *
    +   * (Since we aren't in a macro context we do not seem to have access to 
the built in `freshName`
    +   * function.)
    +   */
    +  protected def freshName(prefix: String): TermName = {
    +    newTermName(s"$prefix$javaSeparator${curId.getAndIncrement}")
    +  }
    +
    +  /**
    +   * Scala ASTs for evaluating an [[Expression]] given a [[Row]] of input.
    +   *
    +   * @param code The sequence of statements required to evaluate the 
expression.
    +   * @param nullTerm A term that holds a boolean value representing 
whether the expression evaluated
    +   *                 to null.
    +   * @param primitiveTerm A term for a possible primitive value of the 
result of the evaluation. Not
    +   *                      valid if `nullTerm` is set to `false`.
    +   * @param objectTerm A possibly boxed version of the result of 
evaluating this expression.
    +   */
    +  protected case class EvaluatedExpression(
    +      code: Seq[Tree],
    +      nullTerm: TermName,
    +      primitiveTerm: TermName,
    +      objectTerm: TermName)
    +
    +  /**
    +   * Given an expression tree returns an [[EvaluatedExpression]], which 
contains Scala trees that
    +   * can be used to determine the result of evaluating the expression on 
an input row.
    +   */
    +  def expressionEvaluator(e: Expression): EvaluatedExpression = {
    +    val primitiveTerm = freshName("primitiveTerm")
    +    val nullTerm = freshName("nullTerm")
    +    val objectTerm = freshName("objectTerm")
    +
    +    implicit class Evaluate1(e: Expression) {
    +      def castOrNull(f: TermName => Tree, dataType: DataType): Seq[Tree] = 
{
    +        val eval = expressionEvaluator(e)
    +        eval.code ++
    +        q"""
    +          val $nullTerm = ${eval.nullTerm}
    +          val $primitiveTerm =
    +            if($nullTerm)
    +              ${defaultPrimitive(dataType)}
    +            else
    +              ${f(eval.primitiveTerm)}
    +        """.children
    +      }
    +    }
    +
    +    implicit class Evaluate2(expressions: (Expression, Expression)) {
    +
    +      /**
    +       * Short hand for generating binary evaluation code, which depends 
on two sub-evaluations of
    +       * the same type.  If either of the sub-expressions is null, the 
result of this computation
    +       * is assumed to be null.
    +       *
    +       * @param f a function from two primitive term names to a tree that 
evaluates them.
    +       */
    +      def evaluate(f: (TermName, TermName) => Tree): Seq[Tree] =
    +        evaluateAs(expressions._1.dataType)(f)
    +
    +      def evaluateAs(resultType: DataType)(f: (TermName, TermName) => 
Tree): Seq[Tree] = {
    +        // TODO: Right now some timestamp tests fail if we enforce this...
    +        if (expressions._1.dataType != expressions._2.dataType) {
    +          log.warn(s"${expressions._1.dataType} != 
${expressions._2.dataType}")
    +        }
    +
    +        val eval1 = expressionEvaluator(expressions._1)
    +        val eval2 = expressionEvaluator(expressions._2)
    +        val resultCode = f(eval1.primitiveTerm, eval2.primitiveTerm)
    +
    +        eval1.code ++ eval2.code ++
    +        q"""
    +          val $nullTerm = ${eval1.nullTerm} || ${eval2.nullTerm}
    +          val $primitiveTerm: ${termForType(resultType)} =
    +            if($nullTerm) {
    +              ${defaultPrimitive(resultType)}
    +            } else {
    +              $resultCode.asInstanceOf[${termForType(resultType)}]
    +            }
    +        """.children : Seq[Tree]
    +      }
    +    }
    +
    +    val inputTuple = newTermName(s"i")
    +
    +    // TODO: Skip generation of null handling code when expression are not 
nullable.
    +    val primitiveEvaluation: PartialFunction[Expression, Seq[Tree]] = {
    +      case b @ BoundReference(ordinal, dataType, nullable) =>
    +        val nullValue = q"$inputTuple.isNullAt($ordinal)"
    +        q"""
    +          val $nullTerm: Boolean = $nullValue
    +          val $primitiveTerm: ${termForType(dataType)} =
    +            if($nullTerm)
    +              ${defaultPrimitive(dataType)}
    +            else
    +              ${getColumn(inputTuple, dataType, ordinal)}
    +         """.children
    +
    +      case expressions.Literal(null, dataType) =>
    +        q"""
    +          val $nullTerm = true
    +          val $primitiveTerm: ${termForType(dataType)} = 
null.asInstanceOf[${termForType(dataType)}]
    +         """.children
    +
    +      case expressions.Literal(value: Boolean, dataType) =>
    +        q"""
    +          val $nullTerm = ${value == null}
    +          val $primitiveTerm: ${termForType(dataType)} = $value
    +         """.children
    +
    +      case expressions.Literal(value: String, dataType) =>
    +        q"""
    +          val $nullTerm = ${value == null}
    +          val $primitiveTerm: ${termForType(dataType)} = $value
    +         """.children
    +
    +      case expressions.Literal(value: Int, dataType) =>
    +        q"""
    +          val $nullTerm = ${value == null}
    +          val $primitiveTerm: ${termForType(dataType)} = $value
    +         """.children
    +
    +      case expressions.Literal(value: Long, dataType) =>
    +        q"""
    +          val $nullTerm = ${value == null}
    +          val $primitiveTerm: ${termForType(dataType)} = $value
    +         """.children
    +
    +      case Cast(e @ BinaryType(), StringType) =>
    +        val eval = expressionEvaluator(e)
    +        eval.code ++
    +        q"""
    +          val $nullTerm = ${eval.nullTerm}
    +          val $primitiveTerm =
    +            if($nullTerm)
    +              ${defaultPrimitive(StringType)}
    +            else
    +              new String(${eval.primitiveTerm}.asInstanceOf[Array[Byte]])
    +        """.children
    +
    +      case Cast(child @ NumericType(), IntegerType) =>
    +        child.castOrNull(c => q"$c.toInt", IntegerType)
    +
    +      case Cast(child @ NumericType(), LongType) =>
    +        child.castOrNull(c => q"$c.toLong", LongType)
    +
    +      case Cast(child @ NumericType(), DoubleType) =>
    +        child.castOrNull(c => q"$c.toDouble", DoubleType)
    +
    +      case Cast(child @ NumericType(), FloatType) =>
    +        child.castOrNull(c => q"$c.toFloat", IntegerType)
    +
    +      // Special handling required for timestamps in hive test cases.
    --- End diff --
    
    can you also explain inline what the problem is


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to