[GitHub] [spark] cloud-fan commented on a diff in pull request #40394: [SPARK-42771][SQL] Refactor HiveGenericUDF

2023-03-14 Thread via GitHub


cloud-fan commented on code in PR #40394:
URL: https://github.com/apache/spark/pull/40394#discussion_r1136441006


##
sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala:
##
@@ -235,6 +210,36 @@ private[hive] case class HiveGenericUDF(
   }
 }
 
+class HiveGenericUDFEvaluator(
+funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
+  extends HiveInspectors
+  with Serializable {
+
+  @transient
+  private[hive] lazy val function = funcWrapper.createFunction[GenericUDF]()
+
+  @transient
+  private[hive] lazy val argumentInspectors = children.map(toInspector)
+
+  @transient
+  private[hive] lazy val returnInspector = {
+function.initializeAndFoldConstants(argumentInspectors.toArray)
+  }
+
+  @transient
+  private lazy val deferredObjects: Array[DeferredObject] = 
argumentInspectors.zip(children).map {
+case (inspect, child) => new DeferredObjectAdapter(inspect, child.dataType)
+  }.toArray[DeferredObject]
+
+  @transient
+  private lazy val unwrapper: Any => Any = unwrapperFor(returnInspector)
+
+  private[hive] def setArg(index: Int, arg: Any): Unit =

Review Comment:
   ```suggestion
 def setArg(index: Int, arg: Any): Unit =
   ```



##
sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala:
##
@@ -235,6 +210,36 @@ private[hive] case class HiveGenericUDF(
   }
 }
 
+class HiveGenericUDFEvaluator(
+funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
+  extends HiveInspectors
+  with Serializable {
+
+  @transient
+  private[hive] lazy val function = funcWrapper.createFunction[GenericUDF]()
+
+  @transient
+  private[hive] lazy val argumentInspectors = children.map(toInspector)
+
+  @transient
+  private[hive] lazy val returnInspector = {
+function.initializeAndFoldConstants(argumentInspectors.toArray)
+  }
+
+  @transient
+  private lazy val deferredObjects: Array[DeferredObject] = 
argumentInspectors.zip(children).map {
+case (inspect, child) => new DeferredObjectAdapter(inspect, child.dataType)
+  }.toArray[DeferredObject]
+
+  @transient
+  private lazy val unwrapper: Any => Any = unwrapperFor(returnInspector)
+
+  private[hive] def setArg(index: Int, arg: Any): Unit =
+deferredObjects(index).asInstanceOf[DeferredObjectAdapter].set(arg)
+
+  private[hive] def evaluate(): Any = 
unwrapper(function.evaluate(deferredObjects))

Review Comment:
   ```suggestion
 def evaluate(): Any = unwrapper(function.evaluate(deferredObjects))
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a diff in pull request #40394: [SPARK-42771][SQL] Refactor HiveGenericUDF

2023-03-14 Thread via GitHub


cloud-fan commented on code in PR #40394:
URL: https://github.com/apache/spark/pull/40394#discussion_r1136440921


##
sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala:
##
@@ -235,6 +210,36 @@ private[hive] case class HiveGenericUDF(
   }
 }
 
+class HiveGenericUDFEvaluator(
+funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
+  extends HiveInspectors
+  with Serializable {
+
+  @transient
+  private[hive] lazy val function = funcWrapper.createFunction[GenericUDF]()
+
+  @transient
+  private[hive] lazy val argumentInspectors = children.map(toInspector)
+
+  @transient
+  private[hive] lazy val returnInspector = {

Review Comment:
   ```suggestion
 lazy val returnInspector = {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a diff in pull request #40394: [SPARK-42771][SQL] Refactor HiveGenericUDF

2023-03-14 Thread via GitHub


cloud-fan commented on code in PR #40394:
URL: https://github.com/apache/spark/pull/40394#discussion_r1136440774


##
sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala:
##
@@ -235,6 +210,36 @@ private[hive] case class HiveGenericUDF(
   }
 }
 
+class HiveGenericUDFEvaluator(
+funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
+  extends HiveInspectors
+  with Serializable {
+
+  @transient
+  private[hive] lazy val function = funcWrapper.createFunction[GenericUDF]()

Review Comment:
   ```suggestion
 lazy val function = funcWrapper.createFunction[GenericUDF]()
   ```



##
sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala:
##
@@ -235,6 +210,36 @@ private[hive] case class HiveGenericUDF(
   }
 }
 
+class HiveGenericUDFEvaluator(
+funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
+  extends HiveInspectors
+  with Serializable {
+
+  @transient
+  private[hive] lazy val function = funcWrapper.createFunction[GenericUDF]()
+
+  @transient
+  private[hive] lazy val argumentInspectors = children.map(toInspector)

Review Comment:
   ```suggestion
 private lazy val argumentInspectors = children.map(toInspector)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a diff in pull request #40394: [SPARK-42771][SQL] Refactor HiveGenericUDF

2023-03-14 Thread via GitHub


cloud-fan commented on code in PR #40394:
URL: https://github.com/apache/spark/pull/40394#discussion_r1136440410


##
sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala:
##
@@ -130,57 +130,33 @@ private[hive] case class HiveGenericUDF(
 name: String, funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
   extends Expression
   with HiveInspectors
-  with Logging
   with UserDefinedExpression {
 
   override def nullable: Boolean = true
 
-  override lazy val deterministic: Boolean = isUDFDeterministic && 
children.forall(_.deterministic)
+  override lazy val deterministic: Boolean =
+isUDFDeterministic && children.forall(_.deterministic)
 
-  override def foldable: Boolean =
-isUDFDeterministic && returnInspector.isInstanceOf[ConstantObjectInspector]
+  override def foldable: Boolean = isUDFDeterministic &&
+evaluator.returnInspector.isInstanceOf[ConstantObjectInspector]
 
-  @transient
-  lazy val function = funcWrapper.createFunction[GenericUDF]()
-
-  @transient
-  private lazy val argumentInspectors = children.map(toInspector)
-
-  @transient
-  private lazy val returnInspector = {
-function.initializeAndFoldConstants(argumentInspectors.toArray)
-  }
+  override lazy val dataType: DataType = 
inspectorToDataType(evaluator.returnInspector)
 
-  // Visible for codegen
   @transient
-  lazy val unwrapper: Any => Any = unwrapperFor(returnInspector)
+  private lazy val evaluator = new HiveGenericUDFEvaluator(funcWrapper, 
children)
 
   @transient
-  private lazy val isUDFDeterministic = {
-val udfType = function.getClass.getAnnotation(classOf[HiveUDFType])
+  private val isUDFDeterministic = {
+val udfType = 
evaluator.function.getClass.getAnnotation(classOf[HiveUDFType])
 udfType != null && udfType.deterministic() && !udfType.stateful()
   }
 
-  // Visible for codegen
-  @transient
-  lazy val deferredObjects: Array[DeferredObject] = 
argumentInspectors.zip(children).map {
-case (inspect, child) => new DeferredObjectAdapter(inspect, child.dataType)
-  }.toArray[DeferredObject]
-
-  override lazy val dataType: DataType = inspectorToDataType(returnInspector)
-
   override def eval(input: InternalRow): Any = {
-returnInspector // Make sure initialized.
-
-var i = 0
-val length = children.length
-while (i < length) {
-  val idx = i
-  deferredObjects(i).asInstanceOf[DeferredObjectAdapter]
-.set(children(idx).eval(input))
-  i += 1
+children.zipWithIndex.map {
+  case (child, idx) =>
+evaluator.setArg(idx, child.eval(input))

Review Comment:
   ```suggestion
 case (child, idx) => evaluator.setArg(idx, child.eval(input))
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a diff in pull request #40394: [SPARK-42771][SQL] Refactor HiveGenericUDF

2023-03-14 Thread via GitHub


cloud-fan commented on code in PR #40394:
URL: https://github.com/apache/spark/pull/40394#discussion_r1136424404


##
sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala:
##
@@ -235,6 +210,39 @@ private[hive] case class HiveGenericUDF(
   }
 }
 
+class HiveGenericUDFEvaluator(
+funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
+  extends HiveInspectors
+  with Serializable {
+
+  @transient
+  private lazy val function = funcWrapper.createFunction[GenericUDF]()
+
+  @transient
+  private[hive] val getUDFType = 
function.getClass.getAnnotation(classOf[HiveUDFType])

Review Comment:
   This is only called in the expression, we can move it out



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a diff in pull request #40394: [SPARK-42771][SQL] Refactor HiveGenericUDF

2023-03-14 Thread via GitHub


cloud-fan commented on code in PR #40394:
URL: https://github.com/apache/spark/pull/40394#discussion_r1135521409


##
sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala:
##
@@ -235,6 +200,56 @@ private[hive] case class HiveGenericUDF(
   }
 }
 
+class HiveGenericUDFHelper(
+funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
+  extends HiveInspectors
+  with Serializable
+  with Logging {
+
+  @transient
+  private[hive] val deterministic = isUDFDeterministic && 
children.forall(_.deterministic)
+
+  @transient
+  private[hive] val foldable =
+isUDFDeterministic && returnInspector.isInstanceOf[ConstantObjectInspector]
+
+  @transient
+  private[hive] val dataType: DataType = inspectorToDataType(returnInspector)
+
+  @transient
+  private lazy val function = funcWrapper.createFunction[GenericUDF]()
+
+  @transient
+  private lazy val isUDFDeterministic = {
+val udfType = function.getClass.getAnnotation(classOf[HiveUDFType])
+udfType != null && udfType.deterministic() && !udfType.stateful()
+  }
+
+  @transient
+  private lazy val argumentInspectors = children.map(toInspector)
+
+  @transient
+  private lazy val deferredObjects: Array[DeferredObject] = 
argumentInspectors.zip(children).map {

Review Comment:
   This makes me think that we should only keep execution-related things in 
this evaluator: `function` and `deferredObjects`. Can we expose 
`argumentInspectors` so that expression can define `dataType`, `foldable`, etc.?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a diff in pull request #40394: [SPARK-42771][SQL] Refactor HiveGenericUDF

2023-03-13 Thread via GitHub


cloud-fan commented on code in PR #40394:
URL: https://github.com/apache/spark/pull/40394#discussion_r1134995335


##
sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala:
##
@@ -129,58 +129,25 @@ private[hive] class DeferredObjectAdapter(oi: 
ObjectInspector, dataType: DataTyp
 private[hive] case class HiveGenericUDF(
 name: String, funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
   extends Expression
-  with HiveInspectors
-  with Logging
   with UserDefinedExpression {
 
   override def nullable: Boolean = true
 
-  override lazy val deterministic: Boolean = isUDFDeterministic && 
children.forall(_.deterministic)
-
-  override def foldable: Boolean =
-isUDFDeterministic && returnInspector.isInstanceOf[ConstantObjectInspector]
-
-  @transient
-  lazy val function = funcWrapper.createFunction[GenericUDF]()
+  override lazy val deterministic: Boolean = evaluator.deterministic
 
-  @transient
-  private lazy val argumentInspectors = children.map(toInspector)
+  override def foldable: Boolean = evaluator.foldable
 
-  @transient
-  private lazy val returnInspector = {
-function.initializeAndFoldConstants(argumentInspectors.toArray)
-  }
+  override lazy val dataType: DataType = evaluator.dataType
 
-  // Visible for codegen
   @transient
-  lazy val unwrapper: Any => Any = unwrapperFor(returnInspector)
-
-  @transient
-  private lazy val isUDFDeterministic = {
-val udfType = function.getClass.getAnnotation(classOf[HiveUDFType])
-udfType != null && udfType.deterministic() && !udfType.stateful()
-  }
-
-  // Visible for codegen
-  @transient
-  lazy val deferredObjects: Array[DeferredObject] = 
argumentInspectors.zip(children).map {
-case (inspect, child) => new DeferredObjectAdapter(inspect, child.dataType)
-  }.toArray[DeferredObject]
-
-  override lazy val dataType: DataType = inspectorToDataType(returnInspector)
+  private[hive] lazy val evaluator = new HiveGenericUDFEvaluator(funcWrapper, 
children)

Review Comment:
   ```suggestion
 private lazy val evaluator = new HiveGenericUDFEvaluator(funcWrapper, 
children)
   ```
   unless we need to access it somewhere.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a diff in pull request #40394: [SPARK-42771][SQL] Refactor HiveGenericUDF

2023-03-13 Thread via GitHub


cloud-fan commented on code in PR #40394:
URL: https://github.com/apache/spark/pull/40394#discussion_r1134994989


##
sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala:
##
@@ -129,58 +129,25 @@ private[hive] class DeferredObjectAdapter(oi: 
ObjectInspector, dataType: DataTyp
 private[hive] case class HiveGenericUDF(
 name: String, funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
   extends Expression
-  with HiveInspectors
-  with Logging
   with UserDefinedExpression {
 
   override def nullable: Boolean = true
 
-  override lazy val deterministic: Boolean = isUDFDeterministic && 
children.forall(_.deterministic)
-
-  override def foldable: Boolean =
-isUDFDeterministic && returnInspector.isInstanceOf[ConstantObjectInspector]
-
-  @transient
-  lazy val function = funcWrapper.createFunction[GenericUDF]()
+  override lazy val deterministic: Boolean = evaluator.deterministic
 
-  @transient
-  private lazy val argumentInspectors = children.map(toInspector)
+  override def foldable: Boolean = evaluator.foldable
 
-  @transient
-  private lazy val returnInspector = {
-function.initializeAndFoldConstants(argumentInspectors.toArray)
-  }
+  override lazy val dataType: DataType = evaluator.dataType

Review Comment:
   ```suggestion
 override def dataType: DataType = evaluator.dataType
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a diff in pull request #40394: [SPARK-42771][SQL] Refactor HiveGenericUDF

2023-03-13 Thread via GitHub


cloud-fan commented on code in PR #40394:
URL: https://github.com/apache/spark/pull/40394#discussion_r1134829857


##
sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala:
##
@@ -191,18 +157,18 @@ private[hive] case class HiveGenericUDF(
 
   override protected def withNewChildrenInternal(newChildren: 
IndexedSeq[Expression]): Expression =
 copy(children = newChildren)
-  override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): 
ExprCode = {
+
+  protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
 val refTerm = ctx.addReferenceObj("this", this)

Review Comment:
   do we still reference the entire `this`? Can we just reference the new 
evalautor?



##
sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala:
##
@@ -191,18 +157,18 @@ private[hive] case class HiveGenericUDF(
 
   override protected def withNewChildrenInternal(newChildren: 
IndexedSeq[Expression]): Expression =
 copy(children = newChildren)
-  override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): 
ExprCode = {
+
+  protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
 val refTerm = ctx.addReferenceObj("this", this)

Review Comment:
   do we still reference the entire `this`? Can we just reference the new 
evaluator?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a diff in pull request #40394: [SPARK-42771][SQL] Refactor HiveGenericUDF

2023-03-13 Thread via GitHub


cloud-fan commented on code in PR #40394:
URL: https://github.com/apache/spark/pull/40394#discussion_r1134828589


##
sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala:
##
@@ -235,6 +200,56 @@ private[hive] case class HiveGenericUDF(
   }
 }
 
+class HiveGenericUDFHelper(
+funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
+  extends HiveInspectors
+  with Serializable
+  with Logging {
+
+  @transient
+  private[hive] val deterministic = isUDFDeterministic && 
children.forall(_.deterministic)
+
+  @transient
+  private[hive] val foldable =

Review Comment:
   SGTM, we can probably add a base class `HiveUDFEvaluatorBase`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a diff in pull request #40394: [SPARK-42771][SQL] Refactor HiveGenericUDF

2023-03-13 Thread via GitHub


cloud-fan commented on code in PR #40394:
URL: https://github.com/apache/spark/pull/40394#discussion_r1134828589


##
sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala:
##
@@ -235,6 +200,56 @@ private[hive] case class HiveGenericUDF(
   }
 }
 
+class HiveGenericUDFHelper(
+funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
+  extends HiveInspectors
+  with Serializable
+  with Logging {
+
+  @transient
+  private[hive] val deterministic = isUDFDeterministic && 
children.forall(_.deterministic)
+
+  @transient
+  private[hive] val foldable =

Review Comment:
   SGTM



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a diff in pull request #40394: [SPARK-42771][SQL] Refactor HiveGenericUDF

2023-03-13 Thread via GitHub


cloud-fan commented on code in PR #40394:
URL: https://github.com/apache/spark/pull/40394#discussion_r1134034297


##
sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala:
##
@@ -235,6 +200,56 @@ private[hive] case class HiveGenericUDF(
   }
 }
 
+class HiveGenericUDFHelper(
+funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
+  extends HiveInspectors
+  with Serializable
+  with Logging {
+
+  @transient
+  private[hive] val deterministic = isUDFDeterministic && 
children.forall(_.deterministic)
+
+  @transient
+  private[hive] val foldable =
+isUDFDeterministic && returnInspector.isInstanceOf[ConstantObjectInspector]
+
+  @transient
+  private[hive] val dataType: DataType = inspectorToDataType(returnInspector)
+
+  @transient
+  private lazy val function = funcWrapper.createFunction[GenericUDF]()
+
+  @transient
+  private lazy val isUDFDeterministic = {
+val udfType = function.getClass.getAnnotation(classOf[HiveUDFType])
+udfType != null && udfType.deterministic() && !udfType.stateful()
+  }
+
+  @transient
+  private lazy val argumentInspectors = children.map(toInspector)
+
+  @transient
+  private lazy val deferredObjects: Array[DeferredObject] = 
argumentInspectors.zip(children).map {

Review Comment:
   `lazy val` has perf overhead. I don't think this class needs any lazy val, 
but itself should be lazy val in `HiveGenericUDF`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a diff in pull request #40394: [SPARK-42771][SQL] Refactor HiveGenericUDF

2023-03-13 Thread via GitHub


cloud-fan commented on code in PR #40394:
URL: https://github.com/apache/spark/pull/40394#discussion_r1134032217


##
sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala:
##
@@ -235,6 +200,56 @@ private[hive] case class HiveGenericUDF(
   }
 }
 
+class HiveGenericUDFHelper(
+funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
+  extends HiveInspectors
+  with Serializable
+  with Logging {
+
+  @transient
+  private[hive] val deterministic = isUDFDeterministic && 
children.forall(_.deterministic)
+
+  @transient
+  private[hive] val foldable =

Review Comment:
   why do we define these properties here instead of in the expression? are we 
going to reuse it in `HiveSimpleUDF`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a diff in pull request #40394: [SPARK-42771][SQL] Refactor HiveGenericUDF

2023-03-13 Thread via GitHub


cloud-fan commented on code in PR #40394:
URL: https://github.com/apache/spark/pull/40394#discussion_r1134029759


##
sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala:
##
@@ -235,6 +200,56 @@ private[hive] case class HiveGenericUDF(
   }
 }
 
+class HiveGenericUDFHelper(

Review Comment:
   how about `HiveGenericUDFEvaluator`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org