You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/06/10 02:19:38 UTC

[GitHub] [hudi] KnightChess opened a new pull request, #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

KnightChess opened a new pull request, #5825:
URL: https://github.com/apache/hudi/pull/5825

   ## What is the purpose of the pull request
   
   improve repeat init object in ExpressionPayload, it cose lots of cpu time
   
   ## Brief change log
   
   *(for example:)*
     - *Modify AnnotationLocation checkstyle rule in checkstyle.xml*
   
   ## Verify this pull request
   
   *(Please pick either of the following options)*
   
   This pull request is a trivial rework / code cleanup without any test coverage.
   
   *(or)*
   
   This pull request is already covered by existing tests, such as *(please describe tests)*.
   
   (or)
   
   This change added tests and can be verified as follows:
   
   *(example:)*
   
     - *Added integration tests for end-to-end.*
     - *Added HoodieClientWriteTest to verify the change.*
     - *Manually verified the change by running a job locally.*
   
   ## Committer checklist
   
    - [ ] Has a corresponding JIRA in PR title & commit
    
    - [ ] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.
   


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] KnightChess commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
KnightChess commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1151848950

   before:
   <img width="1164" alt="image" src="https://user-images.githubusercontent.com/20125927/172977646-ccbab8e2-8252-48bc-913d-6e2ba48ddeb0.png">
   
   after:
   <img width="1128" alt="image" src="https://user-images.githubusercontent.com/20125927/172977678-38836991-49fa-4fc8-9aa1-0493fee307b7.png">
   


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1156203384

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203",
       "triggerID" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "248ea4739704297450d11dd09b063257caf98ef2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155428842",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155827502",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155875714",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c327f086923c91f1bcdc445250c296996ef1c721",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9309",
       "triggerID" : "c327f086923c91f1bcdc445250c296996ef1c721",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da138ce4883eac60270d7aaea672fd0462722731",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9313",
       "triggerID" : "da138ce4883eac60270d7aaea672fd0462722731",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 248ea4739704297450d11dd09b063257caf98ef2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291) 
   * c327f086923c91f1bcdc445250c296996ef1c721 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9309) 
   * da138ce4883eac60270d7aaea672fd0462722731 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9313) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1155266086

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203",
       "triggerID" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "248ea4739704297450d11dd09b063257caf98ef2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 526a45bec3eb95921d25a59203d10f8bf9d9e2ce Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203) 
   * 248ea4739704297450d11dd09b063257caf98ef2 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1156140935

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203",
       "triggerID" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "248ea4739704297450d11dd09b063257caf98ef2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155428842",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155827502",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155875714",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c327f086923c91f1bcdc445250c296996ef1c721",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9309",
       "triggerID" : "c327f086923c91f1bcdc445250c296996ef1c721",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da138ce4883eac60270d7aaea672fd0462722731",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "da138ce4883eac60270d7aaea672fd0462722731",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 248ea4739704297450d11dd09b063257caf98ef2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291) 
   * c327f086923c91f1bcdc445250c296996ef1c721 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9309) 
   * da138ce4883eac60270d7aaea672fd0462722731 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] KnightChess commented on a diff in pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
KnightChess commented on code in PR #5825:
URL: https://github.com/apache/hudi/pull/5825#discussion_r894120312


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/SqlTypedRecord.scala:
##########
@@ -17,27 +17,59 @@
 
 package org.apache.spark.sql.hudi.command.payload
 
+import com.google.common.cache.CacheBuilder
 import org.apache.avro.Schema
 import org.apache.avro.generic.IndexedRecord
-import org.apache.hudi.{AvroConversionUtils, SparkAdapterSupport}
+import org.apache.hudi.HoodieSparkUtils.sparkAdapter
+import org.apache.hudi.AvroConversionUtils
+import org.apache.spark.sql.avro.HoodieAvroDeserializer
 import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.hudi.command.payload.SqlTypedRecord.{getAvroDeserializer, getSqlType}
+import org.apache.spark.sql.types.StructType
+
+import java.util.concurrent.Callable
 
 /**
  * A sql typed record which will convert the avro field to sql typed value.
  */
-class SqlTypedRecord(val record: IndexedRecord) extends IndexedRecord with SparkAdapterSupport {
+class SqlTypedRecord(val record: IndexedRecord) extends IndexedRecord {
 
-  private lazy val sqlType = AvroConversionUtils.convertAvroSchemaToStructType(getSchema)
-  private lazy val avroDeserializer = sparkAdapter.createAvroDeserializer(record.getSchema, sqlType)
-  private lazy val sqlRow = avroDeserializer.deserialize(record).get.asInstanceOf[InternalRow]
+  private lazy val sqlRow = getAvroDeserializer(getSchema).deserialize(record).get.asInstanceOf[InternalRow]
 
   override def put(i: Int, v: Any): Unit = {
     record.put(i, v)
   }
 
   override def get(i: Int): AnyRef = {
-    sqlRow.get(i, sqlType(i).dataType)
+    sqlRow.get(i, getSqlType(getSchema)(i).dataType)
   }
 
   override def getSchema: Schema = record.getSchema
 }
+
+object SqlTypedRecord {
+
+  private val sqlTypeCache = CacheBuilder.newBuilder().maximumSize(16).build[Schema, StructType]()

Review Comment:
   16 just the guava cache default size. In batch model, I think only one source and target schema will be cache, but considering streaming or spark thrift server( or livy kyuubi), the target or source schema will diff. For long process task, may be someone can give a appropriate suggestion for it
   
   >I also met this issue recently, but I'm not sure it's a good idea if we don't have a way to invalidate the schema if we don't use it anymore.
   
   that's why I use LRU cache to cache it, just for long process task. And we also can set some expiration strategy if its necessary, but I think LRU cache will enough



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] KnightChess commented on a diff in pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
KnightChess commented on code in PR #5825:
URL: https://github.com/apache/hudi/pull/5825#discussion_r930545666


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/SqlTypedRecord.scala:
##########
@@ -17,27 +17,59 @@
 
 package org.apache.spark.sql.hudi.command.payload
 
+import com.google.common.cache.CacheBuilder

Review Comment:
   @xushiyan I will fix it this week



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1151920089

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203",
       "triggerID" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 526a45bec3eb95921d25a59203d10f8bf9d9e2ce Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] xushiyan commented on a diff in pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
xushiyan commented on code in PR #5825:
URL: https://github.com/apache/hudi/pull/5825#discussion_r930507915


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/SqlTypedRecord.scala:
##########
@@ -17,27 +17,59 @@
 
 package org.apache.spark.sql.hudi.command.payload
 
+import com.google.common.cache.CacheBuilder

Review Comment:
   filed
   https://issues.apache.org/jira/browse/HUDI-4482 -> remove guava
   https://issues.apache.org/jira/browse/HUDI-4483 -> fix checkstyle
   
   @KnightChess would you take these?  i see guava heavily used in procedure, @XuQianJin-Stars maybe you can help review?



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1155436603

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203",
       "triggerID" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "248ea4739704297450d11dd09b063257caf98ef2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155428842",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 248ea4739704297450d11dd09b063257caf98ef2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1156126637

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203",
       "triggerID" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "248ea4739704297450d11dd09b063257caf98ef2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155428842",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155827502",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155875714",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c327f086923c91f1bcdc445250c296996ef1c721",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c327f086923c91f1bcdc445250c296996ef1c721",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 248ea4739704297450d11dd09b063257caf98ef2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291) 
   * c327f086923c91f1bcdc445250c296996ef1c721 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1155883880

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203",
       "triggerID" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "248ea4739704297450d11dd09b063257caf98ef2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155428842",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155827502",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155875714",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 248ea4739704297450d11dd09b063257caf98ef2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1155421145

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203",
       "triggerID" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "248ea4739704297450d11dd09b063257caf98ef2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 248ea4739704297450d11dd09b063257caf98ef2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 merged pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
danny0405 merged PR #5825:
URL: https://github.com/apache/hudi/pull/5825


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] KnightChess commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
KnightChess commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1156039664

   @danny0405 does it need new commit to re build azure? It look like run azure can't trigger it success,the completion time has not changed
   ![image](https://user-images.githubusercontent.com/20125927/173757264-099e3bf1-519e-4700-bcd8-d8472fb1b5c6.png)
   


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
danny0405 commented on code in PR #5825:
URL: https://github.com/apache/hudi/pull/5825#discussion_r897629951


##########
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala:
##########
@@ -318,5 +305,26 @@ object ExpressionPayload {
         }
       })
   }
+
+  private val mergedSchemaCache = CacheBuilder.newBuilder().build[TupleSchema, Schema]()
+
+  def getMergedSchema(source: Schema, target: Schema): Schema = {
+
+    mergedSchemaCache.get(TupleSchema(source, target), new Callable[Schema] {
+      override def call(): Schema = {
+        val noMetaRight = HoodieAvroUtils.removeMetadataFields(target)
+        val mergedFields =

Review Comment:
   `noMetaRight` -> `rightSchema`
   
   I would suggest you keep the `mergeSchema` method, and `getMergedSchema` can call this method to make the logic more clear.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1155881704

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203",
       "triggerID" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "248ea4739704297450d11dd09b063257caf98ef2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155428842",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155827502",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1155875714",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 248ea4739704297450d11dd09b063257caf98ef2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1155855228

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203",
       "triggerID" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "248ea4739704297450d11dd09b063257caf98ef2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155428842",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155827502",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 248ea4739704297450d11dd09b063257caf98ef2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1155852628

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203",
       "triggerID" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "248ea4739704297450d11dd09b063257caf98ef2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155428842",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1155827502",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 248ea4739704297450d11dd09b063257caf98ef2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] KnightChess commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
KnightChess commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1155875714

   @hudi-bot run azure


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1156264113

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203",
       "triggerID" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "248ea4739704297450d11dd09b063257caf98ef2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155428842",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155827502",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155875714",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c327f086923c91f1bcdc445250c296996ef1c721",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9309",
       "triggerID" : "c327f086923c91f1bcdc445250c296996ef1c721",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da138ce4883eac60270d7aaea672fd0462722731",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9313",
       "triggerID" : "da138ce4883eac60270d7aaea672fd0462722731",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c327f086923c91f1bcdc445250c296996ef1c721 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9309) 
   * da138ce4883eac60270d7aaea672fd0462722731 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9313) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
danny0405 commented on code in PR #5825:
URL: https://github.com/apache/hudi/pull/5825#discussion_r896359970


##########
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala:
##########
@@ -318,5 +316,26 @@ object ExpressionPayload {
         }
       })
   }
+
+  private val mergeCache = CacheBuilder.newBuilder().maximumSize(16).build[TupleSchema, Schema]()
+

Review Comment:
   `mergeCache` -> `mergedSchemaCache`



##########
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala:
##########
@@ -318,5 +316,26 @@ object ExpressionPayload {
         }
       })
   }
+
+  private val mergeCache = CacheBuilder.newBuilder().maximumSize(16).build[TupleSchema, Schema]()
+
+  def getMergeSchema(source: Schema, target: Schema): Schema = {
+

Review Comment:
   `getMergeSchema` -> `getMergedSchema`



##########
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala:
##########
@@ -318,5 +316,26 @@ object ExpressionPayload {
         }
       })
   }
+
+  private val mergeCache = CacheBuilder.newBuilder().maximumSize(16).build[TupleSchema, Schema]()
+
+  def getMergeSchema(source: Schema, target: Schema): Schema = {
+
+    mergeCache.get(TupleSchema(source, target), new Callable[Schema] {
+      override def call(): Schema = {
+        val noMetaRight = HoodieAvroUtils.removeMetadataFields(target)
+        val mergedFields =
+          source.getFields.asScala.map(field =>

Review Comment:
   Can we reuse the `mergeSchema` method ?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/SqlTypedRecord.scala:
##########
@@ -17,27 +17,59 @@
 
 package org.apache.spark.sql.hudi.command.payload
 
+import com.google.common.cache.CacheBuilder
 import org.apache.avro.Schema
 import org.apache.avro.generic.IndexedRecord
-import org.apache.hudi.{AvroConversionUtils, SparkAdapterSupport}
+import org.apache.hudi.HoodieSparkUtils.sparkAdapter
+import org.apache.hudi.AvroConversionUtils
+import org.apache.spark.sql.avro.HoodieAvroDeserializer
 import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.hudi.command.payload.SqlTypedRecord.{getAvroDeserializer, getSqlType}
+import org.apache.spark.sql.types.StructType
+
+import java.util.concurrent.Callable
 
 /**
  * A sql typed record which will convert the avro field to sql typed value.
  */
-class SqlTypedRecord(val record: IndexedRecord) extends IndexedRecord with SparkAdapterSupport {
+class SqlTypedRecord(val record: IndexedRecord) extends IndexedRecord {
 
-  private lazy val sqlType = AvroConversionUtils.convertAvroSchemaToStructType(getSchema)
-  private lazy val avroDeserializer = sparkAdapter.createAvroDeserializer(record.getSchema, sqlType)
-  private lazy val sqlRow = avroDeserializer.deserialize(record).get.asInstanceOf[InternalRow]
+  private lazy val sqlRow = getAvroDeserializer(getSchema).deserialize(record).get.asInstanceOf[InternalRow]
 
   override def put(i: Int, v: Any): Unit = {
     record.put(i, v)
   }
 
   override def get(i: Int): AnyRef = {
-    sqlRow.get(i, sqlType(i).dataType)
+    sqlRow.get(i, getSqlType(getSchema)(i).dataType)
   }
 
   override def getSchema: Schema = record.getSchema
 }
+
+object SqlTypedRecord {
+
+  private val sqlTypeCache = CacheBuilder.newBuilder().maximumSize(16).build[Schema, StructType]()

Review Comment:
   If 16 is a default size, why we still configure it here ?



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1151852794

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203",
       "triggerID" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 526a45bec3eb95921d25a59203d10f8bf9d9e2ce Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] KnightChess commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
KnightChess commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1156128336

   @danny0405 yes, I will rebase and force push it when any review be solved


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] KnightChess commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
KnightChess commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1156415445

   @danny0405 thanks for review :)


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] KnightChess commented on a diff in pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
KnightChess commented on code in PR #5825:
URL: https://github.com/apache/hudi/pull/5825#discussion_r897640266


##########
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala:
##########
@@ -318,5 +305,26 @@ object ExpressionPayload {
         }
       })
   }
+
+  private val mergedSchemaCache = CacheBuilder.newBuilder().build[TupleSchema, Schema]()
+
+  def getMergedSchema(source: Schema, target: Schema): Schema = {
+
+    mergedSchemaCache.get(TupleSchema(source, target), new Callable[Schema] {
+      override def call(): Schema = {
+        val noMetaRight = HoodieAvroUtils.removeMetadataFields(target)
+        val mergedFields =

Review Comment:
   got it



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1156397274

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203",
       "triggerID" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "248ea4739704297450d11dd09b063257caf98ef2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155428842",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155827502",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155875714",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c327f086923c91f1bcdc445250c296996ef1c721",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9309",
       "triggerID" : "c327f086923c91f1bcdc445250c296996ef1c721",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da138ce4883eac60270d7aaea672fd0462722731",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9313",
       "triggerID" : "da138ce4883eac60270d7aaea672fd0462722731",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * da138ce4883eac60270d7aaea672fd0462722731 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9313) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1151850397

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 526a45bec3eb95921d25a59203d10f8bf9d9e2ce UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
danny0405 commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1156125254

   > @danny0405 does it need new commit to re build azure? It look like run azure can't trigger it success,the completion time has not changed ![image](https://user-images.githubusercontent.com/20125927/173757264-099e3bf1-519e-4700-bcd8-d8472fb1b5c6.png)
   
   You better rebase the latest master and force push the codes.


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] KnightChess commented on a diff in pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
KnightChess commented on code in PR #5825:
URL: https://github.com/apache/hudi/pull/5825#discussion_r896835464


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/SqlTypedRecord.scala:
##########
@@ -17,27 +17,59 @@
 
 package org.apache.spark.sql.hudi.command.payload
 
+import com.google.common.cache.CacheBuilder
 import org.apache.avro.Schema
 import org.apache.avro.generic.IndexedRecord
-import org.apache.hudi.{AvroConversionUtils, SparkAdapterSupport}
+import org.apache.hudi.HoodieSparkUtils.sparkAdapter
+import org.apache.hudi.AvroConversionUtils
+import org.apache.spark.sql.avro.HoodieAvroDeserializer
 import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.hudi.command.payload.SqlTypedRecord.{getAvroDeserializer, getSqlType}
+import org.apache.spark.sql.types.StructType
+
+import java.util.concurrent.Callable
 
 /**
  * A sql typed record which will convert the avro field to sql typed value.
  */
-class SqlTypedRecord(val record: IndexedRecord) extends IndexedRecord with SparkAdapterSupport {
+class SqlTypedRecord(val record: IndexedRecord) extends IndexedRecord {
 
-  private lazy val sqlType = AvroConversionUtils.convertAvroSchemaToStructType(getSchema)
-  private lazy val avroDeserializer = sparkAdapter.createAvroDeserializer(record.getSchema, sqlType)
-  private lazy val sqlRow = avroDeserializer.deserialize(record).get.asInstanceOf[InternalRow]
+  private lazy val sqlRow = getAvroDeserializer(getSchema).deserialize(record).get.asInstanceOf[InternalRow]
 
   override def put(i: Int, v: Any): Unit = {
     record.put(i, v)
   }
 
   override def get(i: Int): AnyRef = {
-    sqlRow.get(i, sqlType(i).dataType)
+    sqlRow.get(i, getSqlType(getSchema)(i).dataType)
   }
 
   override def getSchema: Schema = record.getSchema
 }
+
+object SqlTypedRecord {
+
+  private val sqlTypeCache = CacheBuilder.newBuilder().maximumSize(16).build[Schema, StructType]()

Review Comment:
   remove it



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1155271805

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203",
       "triggerID" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "248ea4739704297450d11dd09b063257caf98ef2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 526a45bec3eb95921d25a59203d10f8bf9d9e2ce Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203) 
   * 248ea4739704297450d11dd09b063257caf98ef2 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1156131355

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203",
       "triggerID" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "248ea4739704297450d11dd09b063257caf98ef2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155428842",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155827502",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "1155875714",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c327f086923c91f1bcdc445250c296996ef1c721",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9309",
       "triggerID" : "c327f086923c91f1bcdc445250c296996ef1c721",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 248ea4739704297450d11dd09b063257caf98ef2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291) 
   * c327f086923c91f1bcdc445250c296996ef1c721 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9309) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] vinothchandar commented on a diff in pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on code in PR #5825:
URL: https://github.com/apache/hudi/pull/5825#discussion_r929279266


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/SqlTypedRecord.scala:
##########
@@ -17,27 +17,59 @@
 
 package org.apache.spark.sql.hudi.command.payload
 
+import com.google.common.cache.CacheBuilder

Review Comment:
   where is this picked up from? We removed guava from deps a while ago. 



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] boneanxs commented on a diff in pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
boneanxs commented on code in PR #5825:
URL: https://github.com/apache/hudi/pull/5825#discussion_r894110805


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/payload/SqlTypedRecord.scala:
##########
@@ -17,27 +17,59 @@
 
 package org.apache.spark.sql.hudi.command.payload
 
+import com.google.common.cache.CacheBuilder
 import org.apache.avro.Schema
 import org.apache.avro.generic.IndexedRecord
-import org.apache.hudi.{AvroConversionUtils, SparkAdapterSupport}
+import org.apache.hudi.HoodieSparkUtils.sparkAdapter
+import org.apache.hudi.AvroConversionUtils
+import org.apache.spark.sql.avro.HoodieAvroDeserializer
 import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.hudi.command.payload.SqlTypedRecord.{getAvroDeserializer, getSqlType}
+import org.apache.spark.sql.types.StructType
+
+import java.util.concurrent.Callable
 
 /**
  * A sql typed record which will convert the avro field to sql typed value.
  */
-class SqlTypedRecord(val record: IndexedRecord) extends IndexedRecord with SparkAdapterSupport {
+class SqlTypedRecord(val record: IndexedRecord) extends IndexedRecord {
 
-  private lazy val sqlType = AvroConversionUtils.convertAvroSchemaToStructType(getSchema)
-  private lazy val avroDeserializer = sparkAdapter.createAvroDeserializer(record.getSchema, sqlType)
-  private lazy val sqlRow = avroDeserializer.deserialize(record).get.asInstanceOf[InternalRow]
+  private lazy val sqlRow = getAvroDeserializer(getSchema).deserialize(record).get.asInstanceOf[InternalRow]
 
   override def put(i: Int, v: Any): Unit = {
     record.put(i, v)
   }
 
   override def get(i: Int): AnyRef = {
-    sqlRow.get(i, sqlType(i).dataType)
+    sqlRow.get(i, getSqlType(getSchema)(i).dataType)
   }
 
   override def getSchema: Schema = record.getSchema
 }
+
+object SqlTypedRecord {
+
+  private val sqlTypeCache = CacheBuilder.newBuilder().maximumSize(16).build[Schema, StructType]()

Review Comment:
   Just curious, why max size is 16, Is there any special meaning?
   
   I also met this issue recently, but I'm not sure it's a good idea if we don't have a way to invalidate the schema if we don't use it anymore.



-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] KnightChess commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
KnightChess commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1155827502

   @hudi-bot run azure


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1155431480

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9203",
       "triggerID" : "526a45bec3eb95921d25a59203d10f8bf9d9e2ce",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291",
       "triggerID" : "248ea4739704297450d11dd09b063257caf98ef2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248ea4739704297450d11dd09b063257caf98ef2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1155428842",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 248ea4739704297450d11dd09b063257caf98ef2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9291) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] KnightChess commented on pull request #5825: [HUDI-4217] improve repeat init object in ExpressionPayload

Posted by GitBox <gi...@apache.org>.
KnightChess commented on PR #5825:
URL: https://github.com/apache/hudi/pull/5825#issuecomment-1155428842

   @hudi-bot run azure


-- 
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: commits-unsubscribe@hudi.apache.org

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