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 2021/11/07 02:22:42 UTC

[GitHub] [hudi] YannByron opened a new pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

YannByron opened a new pull request #3936:
URL: https://github.com/apache/hudi/pull/3936


   ## What is the purpose of the pull request
   
   1. refactor `CreateHoodieTableCommand`;
   2. use `TBLPROPERTIES` syntax to pass config instead of `OPTIONS`, and sync hudi's config to properties of the table rather other properties of the storage of the table in hive metastore, also keep compatible with `OPTIONS` syntax;
   3. force to provide `PrimaryKey`, so that make Update/Delete available;
   4. hudi spark-sql decouple from metastore as far as possible, get config from local hoodie.properties first;
   5. modify operation and related configs when insert/merge, to make consistent with dataframe api.
   6. add parameter validation for recordKey, preCombineKey, keyGenerator, even if those parameters are defined by different keys.
   
   ## 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] YannByron commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
YannByron commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r747940349



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
##########
@@ -38,92 +42,125 @@ import org.apache.spark.sql.hudi.HoodieOptionConfig
 import org.apache.spark.sql.hudi.HoodieSqlUtils._
 import org.apache.spark.sql.hudi.command.CreateHoodieTableCommand.{initTableIfNeed, isEmptyPath}
 import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD
-import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.types.{StructField, StructType}
 import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
 import org.apache.spark.{SPARK_VERSION, SparkConf}
-import java.util.{Locale, Properties}
 
-import org.apache.hudi.exception.HoodieException
-import org.apache.hudi.keygen.ComplexKeyGenerator
-import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
+import java.util.{Locale, Properties}
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable
+import scala.util.control.NonFatal
 
 /**
  * Command for create hoodie table.
  */
 case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean)
   extends RunnableCommand with SparkAdapterSupport {
 
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    val tableName = table.identifier.unquotedString
+  val tableName = formatName(table.identifier.table)
+
+  val tblProperties = table.storage.properties ++ table.properties
 
+  override def run(sparkSession: SparkSession): Seq[Row] = {
     val tableIsExists = sparkSession.sessionState.catalog.tableExists(table.identifier)
     if (tableIsExists) {
       if (ignoreIfExists) {
         // scalastyle:off
         return Seq.empty[Row]
         // scalastyle:on
       } else {
-        throw new IllegalArgumentException(s"Table $tableName already exists.")
+        throw new IllegalArgumentException(s"Table ${table.identifier.unquotedString} already exists.")
       }
     }
-    // Create table in the catalog
-    val createTable = createTableInCatalog(sparkSession)
+
+    // get schema with meta fields, table config if hudi table exists, options including
+    // table configs and properties of the catalog table
+    val path = getTableLocation(table, sparkSession)
+    val (finalSchema, existingTableConfig, tableSqlOptions) = parseSchemaAndConfigs(sparkSession, path)
+
     // Init the hoodie.properties
-    initTableIfNeed(sparkSession, createTable)
+    initTableIfNeed(sparkSession, tableName, path, finalSchema,

Review comment:
       i'll optimize this.




-- 
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] YannByron commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
YannByron commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r748677982



##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala
##########
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hudi
+
+import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, OverwriteWithLatestAvroPayload}
+import org.apache.hudi.common.table.HoodieTableConfig
+import org.apache.hudi.testutils.HoodieClientTestBase
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.types._
+
+import org.junit.jupiter.api.{Assertions, BeforeEach, Test}
+
+import org.scalatest.Matchers.intercept
+
+class TestHoodieOptionConfig extends HoodieClientTestBase {
+
+  var spark: SparkSession = _
+
+  /**
+   * Setup method running before each test.
+   */
+  @BeforeEach override def setUp() {
+    initSparkContexts()
+    spark = sqlContext.sparkSession
+  }
+
+  @Test
+  def testWithDefaultSqlOptions(): Unit = {
+    val ops1 = Map("primaryKey" -> "id")
+    val with1 = HoodieOptionConfig.withDefaultSqlOptions(ops1)
+    Assertions.assertTrue(with1.size == 3)

Review comment:
       done.




-- 
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] leesf commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
leesf commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r747552833



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
##########
@@ -38,92 +42,125 @@ import org.apache.spark.sql.hudi.HoodieOptionConfig
 import org.apache.spark.sql.hudi.HoodieSqlUtils._
 import org.apache.spark.sql.hudi.command.CreateHoodieTableCommand.{initTableIfNeed, isEmptyPath}
 import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD
-import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.types.{StructField, StructType}
 import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
 import org.apache.spark.{SPARK_VERSION, SparkConf}
-import java.util.{Locale, Properties}
 
-import org.apache.hudi.exception.HoodieException
-import org.apache.hudi.keygen.ComplexKeyGenerator
-import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
+import java.util.{Locale, Properties}
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable
+import scala.util.control.NonFatal
 
 /**
  * Command for create hoodie table.
  */
 case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean)
   extends RunnableCommand with SparkAdapterSupport {
 
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    val tableName = table.identifier.unquotedString
+  val tableName = formatName(table.identifier.table)
+
+  val tblProperties = table.storage.properties ++ table.properties
 
+  override def run(sparkSession: SparkSession): Seq[Row] = {
     val tableIsExists = sparkSession.sessionState.catalog.tableExists(table.identifier)
     if (tableIsExists) {
       if (ignoreIfExists) {
         // scalastyle:off
         return Seq.empty[Row]
         // scalastyle:on
       } else {
-        throw new IllegalArgumentException(s"Table $tableName already exists.")
+        throw new IllegalArgumentException(s"Table ${table.identifier.unquotedString} already exists.")
       }
     }
-    // Create table in the catalog
-    val createTable = createTableInCatalog(sparkSession)
+
+    // get schema with meta fields, table config if hudi table exists, options including
+    // table configs and properties of the catalog table
+    val path = getTableLocation(table, sparkSession)
+    val (finalSchema, existingTableConfig, tableSqlOptions) = parseSchemaAndConfigs(sparkSession, path)
+
     // Init the hoodie.properties
-    initTableIfNeed(sparkSession, createTable)
+    initTableIfNeed(sparkSession, tableName, path, finalSchema,
+      table.partitionColumnNames, existingTableConfig, tableSqlOptions)
+
+    try {
+      // Create table in the catalog
+      createTableInCatalog(sparkSession, finalSchema, tableSqlOptions)
+    } catch {
+      case NonFatal(e) =>
+        logWarning(s"Failed to create catalog table in metastore: ${e.getMessage}")
+        e.printStackTrace()

Review comment:
       remove this line?

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala
##########
@@ -102,4 +106,75 @@ object HoodieWriterUtils {
     properties.putAll(mapAsJavaMap(parameters))
     new HoodieConfig(properties)
   }
+
+  def getRealKeyGenerator(hoodieConfig: HoodieConfig): String = {
+    val kg = hoodieConfig.getString(KEYGENERATOR_CLASS_NAME.key())
+    if (classOf[SqlKeyGenerator].getCanonicalName == kg) {
+      hoodieConfig.getString(SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME)
+    } else {
+      kg
+    }
+  }
+
+  /**
+   * Detects conflicts between new parameters and existing table configurations
+   */
+  def validateTableConfig(spark: SparkSession, params: Map[String, String],
+      tableConfig: HoodieConfig): Unit = {
+    val resolver = spark.sessionState.conf.resolver
+    val diffConfigs = StringBuilder.newBuilder
+    params.foreach { case (key, value) =>
+      val existingValue = getStringFromTableConfigWithAlternatives(tableConfig, key)
+      if (null != existingValue && !resolver(existingValue, value)) {
+        diffConfigs.append(s"$key:\t$value\t${tableConfig.getString(key)}\n")
+      }
+    }
+
+    if (null != tableConfig) {
+      val datasourceRecordKey = params.getOrElse(RECORDKEY_FIELD.key(), null)
+      val tableConfigRecordKey = tableConfig.getString(HoodieTableConfig.RECORDKEY_FIELDS)
+      if (null != datasourceRecordKey && null != tableConfigRecordKey
+          && datasourceRecordKey != tableConfigRecordKey) {
+        diffConfigs.append(s"RecordKey:\t$datasourceRecordKey\t$tableConfigRecordKey\n")
+      }
+
+      val datasourcePreCombineKey = params.getOrElse(PRECOMBINE_FIELD.key(), null)
+      val tableConfigPreCombineKey = tableConfig.getString(HoodieTableConfig.PRECOMBINE_FIELD)
+      if (null != datasourcePreCombineKey && null != tableConfigPreCombineKey
+          && datasourcePreCombineKey != tableConfigPreCombineKey) {
+        diffConfigs.append(s"PreCombineKey:\t$datasourcePreCombineKey\t$tableConfigPreCombineKey\n")
+      }
+
+      val datasourceKeyGen = {
+        val kg = params.getOrElse(KEYGENERATOR_CLASS_NAME.key(), null)
+        if (classOf[SqlKeyGenerator].getCanonicalName == kg) {

Review comment:
       duplicate code above in `getRealKeyGenerator `

##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala
##########
@@ -38,7 +38,7 @@ class TestAlterTable extends TestHoodieSqlBase {
              |  ts long
              |) using hudi
              | location '$tablePath'
-             | options (
+             | tblproperties (

Review comment:
       here means users would not use options? 

##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala
##########
@@ -62,6 +62,53 @@ class TestCreateTable extends TestHoodieSqlBase {
     )(table.schema.fields)
   }
 
+  test("Test Create Hoodie Table With Options") {
+    val tableName = generateTableName
+    spark.sql(
+      s"""
+         | create table $tableName (
+         |  id int,
+         |  name string,
+         |  price double,
+         |  ts long,
+         |  dt string
+         | ) using hudi
+         | partitioned by (dt)
+         | options (

Review comment:
       we would change here to tblproperties and rename the test description?

##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala
##########
@@ -475,8 +478,8 @@ class TestDataSourceForBootstrap {
   }
 
   def runMetadataBootstrapAndVerifyCommit(tableType: String,
-                                          partitionColumns: Option[String] = None,
-                                          extraOpts: Map[String, String] = Map.empty): String = {
+      partitionColumns: Option[String] = None,

Review comment:
       no need to change the format?

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
##########
@@ -38,92 +42,125 @@ import org.apache.spark.sql.hudi.HoodieOptionConfig
 import org.apache.spark.sql.hudi.HoodieSqlUtils._
 import org.apache.spark.sql.hudi.command.CreateHoodieTableCommand.{initTableIfNeed, isEmptyPath}
 import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD
-import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.types.{StructField, StructType}
 import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
 import org.apache.spark.{SPARK_VERSION, SparkConf}
-import java.util.{Locale, Properties}
 
-import org.apache.hudi.exception.HoodieException
-import org.apache.hudi.keygen.ComplexKeyGenerator
-import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
+import java.util.{Locale, Properties}
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable
+import scala.util.control.NonFatal
 
 /**
  * Command for create hoodie table.
  */
 case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean)
   extends RunnableCommand with SparkAdapterSupport {
 
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    val tableName = table.identifier.unquotedString
+  val tableName = formatName(table.identifier.table)
+
+  val tblProperties = table.storage.properties ++ table.properties
 
+  override def run(sparkSession: SparkSession): Seq[Row] = {
     val tableIsExists = sparkSession.sessionState.catalog.tableExists(table.identifier)
     if (tableIsExists) {
       if (ignoreIfExists) {
         // scalastyle:off
         return Seq.empty[Row]
         // scalastyle:on
       } else {
-        throw new IllegalArgumentException(s"Table $tableName already exists.")
+        throw new IllegalArgumentException(s"Table ${table.identifier.unquotedString} already exists.")
       }
     }
-    // Create table in the catalog
-    val createTable = createTableInCatalog(sparkSession)
+
+    // get schema with meta fields, table config if hudi table exists, options including
+    // table configs and properties of the catalog table
+    val path = getTableLocation(table, sparkSession)
+    val (finalSchema, existingTableConfig, tableSqlOptions) = parseSchemaAndConfigs(sparkSession, path)
+
     // Init the hoodie.properties
-    initTableIfNeed(sparkSession, createTable)
+    initTableIfNeed(sparkSession, tableName, path, finalSchema,

Review comment:
       too many args 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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-967774322


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330",
       "triggerID" : "966919685",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3346",
       "triggerID" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 905d341133715a792d8d7073df14acfbf29860bd Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330) 
   * 701268085d201d24c1eefef115079ac4e4b4d40a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3346) 
   
   <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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-962555036


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d003fdb9a2f781de5a20f7550a6d17fb8351339a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188) 
   
   <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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 58abfc536ba3fc11a00f65226ad5574c324c4496 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187) 
   * d003fdb9a2f781de5a20f7550a6d17fb8351339a 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] xushiyan commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
xushiyan commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r745241470



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala
##########
@@ -102,6 +107,8 @@ object HoodieOptionConfig {
 
   private lazy val reverseValueMapping = valueMapping.map(f => f._2 -> f._1)
 
+  def withDefaultSqlOption(options: Map[String, String]): Map[String, String] = defaultSqlOption ++ options

Review comment:
       ```suggestion
     def withDefaultSqlOptions(options: Map[String, String]): Map[String, String] = defaultSqlOptions ++ options
   ```

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala
##########
@@ -104,10 +104,6 @@ extends RunnableCommand {
         PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp
       )
     }
-
-    val parameters = HoodieWriterUtils.parametersWithWriteDefaults(optParams)
-    val translatedOptions = DataSourceWriteOptions.translateSqlOptions(parameters)
-    translatedOptions

Review comment:
       can you clarify the reason why these no longer needed?

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala
##########
@@ -136,16 +142,19 @@ object HoodieOptionConfig {
     options.map(kv => tableConfigKeyToSqlKey.getOrElse(kv._1, kv._1) -> reverseValueMapping.getOrElse(kv._2, kv._2))
   }
 
-  private lazy val defaultTableConfig: Map[String, String] = {
+  private lazy val defaultSqlOption: Map[String, String] = {

Review comment:
       ```suggestion
     private lazy val defaultSqlOptions: Map[String, String] = {
   ```

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala
##########
@@ -102,4 +106,73 @@ object HoodieWriterUtils {
     properties.putAll(mapAsJavaMap(parameters))
     new HoodieConfig(properties)
   }
+
+  def getRealKeyGenerator(hoodieConfig: HoodieConfig): String = {
+    val kg = hoodieConfig.getString(KEYGENERATOR_CLASS_NAME.key())
+    if (classOf[SqlKeyGenerator].getCanonicalName == kg) {
+      hoodieConfig.getString(SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME)
+    } else {
+      kg
+    }
+  }
+
+  // Detects conflicts between new parameters and existing table configurations

Review comment:
       ```suggestion
     /**
      * Detects conflicts between new parameters and existing table configurations
      */ 
   ```

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala
##########
@@ -73,9 +75,10 @@ case class CreateHoodieTableAsSelectCommand(
 
     // Execute the insert query
     try {
+      val tblProperties = table.storage.properties ++ table.properties

Review comment:
       have seen this needed repeated. can we consider making a `HoodieCatalogTable` to encapsulate this and other hudi specific logic inside, e.g. validation, options transform, etc

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java
##########
@@ -113,7 +113,7 @@ protected boolean needUpdatingPersistedRecord(IndexedRecord currentValue,
     Object persistedOrderingVal = getNestedFieldVal((GenericRecord) currentValue,
         properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), true);
     Comparable incomingOrderingVal = (Comparable) getNestedFieldVal((GenericRecord) incomingRecord,
-        properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), false);
+        properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), true);

Review comment:
       so we return null if not found ordering/precombine key. Shall we make its parent class behave in similar way?




-- 
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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330",
       "triggerID" : "966919685",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 905d341133715a792d8d7073df14acfbf29860bd Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330) 
   * 701268085d201d24c1eefef115079ac4e4b4d40a 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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-962560746


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * d003fdb9a2f781de5a20f7550a6d17fb8351339a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189) 
   
   <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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 02efbd6f34723b31c013b600f0ad41f4b2989fbb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310) 
   
   <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] YannByron commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
YannByron commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r747939916



##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala
##########
@@ -62,6 +62,53 @@ class TestCreateTable extends TestHoodieSqlBase {
     )(table.schema.fields)
   }
 
+  test("Test Create Hoodie Table With Options") {
+    val tableName = generateTableName
+    spark.sql(
+      s"""
+         | create table $tableName (
+         |  id int,
+         |  name string,
+         |  price double,
+         |  ts long,
+         |  dt string
+         | ) using hudi
+         | partitioned by (dt)
+         | options (

Review comment:
       Here is a UT for test the compatibility to use `options`. We should show a standard example for users in official document.




-- 
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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-962541026


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * de33f59e26e56e2e5e23e0a72a26587ecdbaacbb 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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-962541193


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * de33f59e26e56e2e5e23e0a72a26587ecdbaacbb Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186) 
   
   <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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-962556539


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * d003fdb9a2f781de5a20f7550a6d17fb8351339a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189) 
   
   <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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * de33f59e26e56e2e5e23e0a72a26587ecdbaacbb 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] YannByron commented on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
YannByron commented on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-962556410


   @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] YannByron commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
YannByron commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r745325819



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala
##########
@@ -104,10 +104,6 @@ extends RunnableCommand {
         PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp
       )
     }
-
-    val parameters = HoodieWriterUtils.parametersWithWriteDefaults(optParams)
-    val translatedOptions = DataSourceWriteOptions.translateSqlOptions(parameters)
-    translatedOptions

Review comment:
       Appending default parameters and translating sql options will be executed in `HoodieSparkSqlWriter.mergeParamsAndGetHoodieConfig`. In Sql Command, the raw parameters passing to `HoodieSparkSqlWriter.write` will be used to check conflicts between parameters.




-- 
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 change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
xushiyan commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r747943133



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
##########
@@ -141,7 +145,7 @@ object HoodieSparkSqlWriter {
           .setPartitionFields(partitionColumns)
           .setPopulateMetaFields(populateMetaFields)
           .setRecordKeyFields(hoodieConfig.getString(RECORDKEY_FIELD))
-          .setKeyGeneratorClassProp(hoodieConfig.getString(KEYGENERATOR_CLASS_NAME))
+          .setKeyGeneratorClassProp(HoodieWriterUtils.getRealKeyGenerator(hoodieConfig))

Review comment:
       Guess @leesf was suggesting "real" sounds too vague. What about `getUserProvidedKeygenerator()`




-- 
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 merged pull request #3936: [HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
xushiyan merged pull request #3936:
URL: https://github.com/apache/hudi/pull/3936


   


-- 
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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330",
       "triggerID" : "966919685",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3346",
       "triggerID" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 905d341133715a792d8d7073df14acfbf29860bd Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330) 
   * 701268085d201d24c1eefef115079ac4e4b4d40a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3346) 
   
   <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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330",
       "triggerID" : "966919685",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3346",
       "triggerID" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2930c00a4375e00d2f12e8e663da805eb89259fd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3348",
       "triggerID" : "2930c00a4375e00d2f12e8e663da805eb89259fd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 701268085d201d24c1eefef115079ac4e4b4d40a Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3346) 
   * 2930c00a4375e00d2f12e8e663da805eb89259fd Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3348) 
   
   <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] YannByron commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
YannByron commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r748683959



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala
##########
@@ -102,4 +106,75 @@ object HoodieWriterUtils {
     properties.putAll(mapAsJavaMap(parameters))
     new HoodieConfig(properties)
   }
+
+  def getRealKeyGenerator(hoodieConfig: HoodieConfig): String = {
+    val kg = hoodieConfig.getString(KEYGENERATOR_CLASS_NAME.key())
+    if (classOf[SqlKeyGenerator].getCanonicalName == kg) {
+      hoodieConfig.getString(SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME)
+    } else {
+      kg
+    }
+  }
+
+  /**
+   * Detects conflicts between new parameters and existing table configurations
+   */
+  def validateTableConfig(spark: SparkSession, params: Map[String, String],
+      tableConfig: HoodieConfig): Unit = {
+    val resolver = spark.sessionState.conf.resolver
+    val diffConfigs = StringBuilder.newBuilder
+    params.foreach { case (key, value) =>
+      val existingValue = getStringFromTableConfigWithAlternatives(tableConfig, key)
+      if (null != existingValue && !resolver(existingValue, value)) {
+        diffConfigs.append(s"$key:\t$value\t${tableConfig.getString(key)}\n")
+      }
+    }
+
+    if (null != tableConfig) {
+      val datasourceRecordKey = params.getOrElse(RECORDKEY_FIELD.key(), null)
+      val tableConfigRecordKey = tableConfig.getString(HoodieTableConfig.RECORDKEY_FIELDS)
+      if (null != datasourceRecordKey && null != tableConfigRecordKey
+          && datasourceRecordKey != tableConfigRecordKey) {
+        diffConfigs.append(s"RecordKey:\t$datasourceRecordKey\t$tableConfigRecordKey\n")
+      }
+
+      val datasourcePreCombineKey = params.getOrElse(PRECOMBINE_FIELD.key(), null)
+      val tableConfigPreCombineKey = tableConfig.getString(HoodieTableConfig.PRECOMBINE_FIELD)
+      if (null != datasourcePreCombineKey && null != tableConfigPreCombineKey
+          && datasourcePreCombineKey != tableConfigPreCombineKey) {
+        diffConfigs.append(s"PreCombineKey:\t$datasourcePreCombineKey\t$tableConfigPreCombineKey\n")
+      }
+
+      val datasourceKeyGen = {
+        val kg = params.getOrElse(KEYGENERATOR_CLASS_NAME.key(), null)
+        if (classOf[SqlKeyGenerator].getCanonicalName == kg) {

Review comment:
       done.




-- 
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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-966920520


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330",
       "triggerID" : "966919685",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 905d341133715a792d8d7073df14acfbf29860bd Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330) 
   
   <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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-967774091


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330",
       "triggerID" : "966919685",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 905d341133715a792d8d7073df14acfbf29860bd Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330) 
   * 701268085d201d24c1eefef115079ac4e4b4d40a 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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d326b7b41a1d71492c1de795afa172e0a3c20c59 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313) 
   * 7b38fc2781a74beb0db37de1f7c5e970eb44dd41 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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-967779933


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330",
       "triggerID" : "966919685",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3346",
       "triggerID" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 701268085d201d24c1eefef115079ac4e4b4d40a Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3346) 
   
   <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] YannByron commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
YannByron commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r747357446



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala
##########
@@ -171,7 +180,48 @@ object HoodieOptionConfig {
 
   def getPreCombineField(options: Map[String, String]): Option[String] = {
     val params = mappingSqlOptionToHoodieParam(options)
-    params.get(DataSourceWriteOptions.PRECOMBINE_FIELD.key)
+    params.get(DataSourceWriteOptions.PRECOMBINE_FIELD.key).filter(_.nonEmpty)
+  }
+
+  def deleteHooideOptions(options: Map[String, String]): Map[String, String] = {

Review comment:
       OK, i'll add some for this.




-- 
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] leesf commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
leesf commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r747544865



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
##########
@@ -141,7 +145,7 @@ object HoodieSparkSqlWriter {
           .setPartitionFields(partitionColumns)
           .setPopulateMetaFields(populateMetaFields)
           .setRecordKeyFields(hoodieConfig.getString(RECORDKEY_FIELD))
-          .setKeyGeneratorClassProp(hoodieConfig.getString(KEYGENERATOR_CLASS_NAME))
+          .setKeyGeneratorClassProp(HoodieWriterUtils.getRealKeyGenerator(hoodieConfig))

Review comment:
       what does get `Real` means?




-- 
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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-966334795


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 02efbd6f34723b31c013b600f0ad41f4b2989fbb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310) 
   * d326b7b41a1d71492c1de795afa172e0a3c20c59 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313) 
   
   <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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d326b7b41a1d71492c1de795afa172e0a3c20c59 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313) 
   
   <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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330",
       "triggerID" : "966919685",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 905d341133715a792d8d7073df14acfbf29860bd Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330) 
   
   <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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 58abfc536ba3fc11a00f65226ad5574c324c4496 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187) 
   * d003fdb9a2f781de5a20f7550a6d17fb8351339a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188) 
   
   <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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-962547177


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * de33f59e26e56e2e5e23e0a72a26587ecdbaacbb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186) 
   * 58abfc536ba3fc11a00f65226ad5574c324c4496 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187) 
   
   <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] YannByron commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
YannByron commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r745326932



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala
##########
@@ -73,9 +75,10 @@ case class CreateHoodieTableAsSelectCommand(
 
     // Execute the insert query
     try {
+      val tblProperties = table.storage.properties ++ table.properties

Review comment:
       good idea.




-- 
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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-962551399


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 58abfc536ba3fc11a00f65226ad5574c324c4496 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187) 
   * d003fdb9a2f781de5a20f7550a6d17fb8351339a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188) 
   
   <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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d003fdb9a2f781de5a20f7550a6d17fb8351339a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188) 
   
   <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] melin commented on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
melin commented on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-962556577


   The primary key syntax is consistent with other relational databases
   ```
   create table test_hudi_demo ( 
       id int, 
       name string, 
       price double,
       ds date)
   stored as hudi    
   primary key (id)
   lifeCycle 300
   ```


-- 
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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-965470204


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c49f4cd59f44d0a033c88de9391c587fde73e06b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284) 
   
   <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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d326b7b41a1d71492c1de795afa172e0a3c20c59 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313) 
   * 7b38fc2781a74beb0db37de1f7c5e970eb44dd41 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327) 
   
   <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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-966375732


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d326b7b41a1d71492c1de795afa172e0a3c20c59 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313) 
   
   <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] YannByron commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
YannByron commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r748677921



##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala
##########
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hudi
+
+import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, OverwriteWithLatestAvroPayload}
+import org.apache.hudi.common.table.HoodieTableConfig
+import org.apache.hudi.testutils.HoodieClientTestBase
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.types._
+
+import org.junit.jupiter.api.{Assertions, BeforeEach, Test}
+
+import org.scalatest.Matchers.intercept
+
+class TestHoodieOptionConfig extends HoodieClientTestBase {
+
+  var spark: SparkSession = _
+
+  /**
+   * Setup method running before each test.
+   */
+  @BeforeEach override def setUp() {
+    initSparkContexts()
+    spark = sqlContext.sparkSession
+  }
+
+  @Test
+  def testWithDefaultSqlOptions(): Unit = {
+    val ops1 = Map("primaryKey" -> "id")
+    val with1 = HoodieOptionConfig.withDefaultSqlOptions(ops1)
+    Assertions.assertTrue(with1.size == 3)
+    Assertions.assertTrue(with1("primaryKey") == "id")
+    Assertions.assertTrue(with1("type") == "cow")
+    Assertions.assertTrue(with1("payloadClass") == classOf[DefaultHoodieRecordPayload].getName)
+
+    val ops2 = Map("primaryKey" -> "id",
+      "preCombineField" -> "timestamp",
+      "type" -> "mor",
+      "payloadClass" -> classOf[OverwriteWithLatestAvroPayload].getName
+    )
+    val with2 = HoodieOptionConfig.withDefaultSqlOptions(ops2)
+    Assertions.assertTrue(ops2 == with2)
+  }
+
+  @Test
+  def testMappingSqlOptionToTableConfig(): Unit = {
+    val sqlOptions = Map("primaryKey" -> "id,addr",
+      "preCombineField" -> "timestamp",
+      "type" -> "mor",
+      "hoodie.index.type" -> "INMEMORY",
+      "hoodie.compact.inline" -> "true"
+    )
+    val tableConfigs = HoodieOptionConfig.mappingSqlOptionToTableConfig(sqlOptions)
+
+    Assertions.assertTrue(tableConfigs.size == 5)
+    Assertions.assertTrue(tableConfigs(HoodieTableConfig.RECORDKEY_FIELDS.key) == "id,addr")
+    Assertions.assertTrue(tableConfigs(HoodieTableConfig.PRECOMBINE_FIELD.key) == "timestamp")
+    Assertions.assertTrue(tableConfigs(HoodieTableConfig.TYPE.key) == "MERGE_ON_READ")
+    Assertions.assertTrue(tableConfigs("hoodie.index.type") == "INMEMORY")
+    Assertions.assertTrue(tableConfigs("hoodie.compact.inline") == "true")
+  }
+
+  @Test
+  def testDeleteHooideOptions(): Unit = {
+    val sqlOptions = Map("primaryKey" -> "id,addr",
+      "preCombineField" -> "timestamp",
+      "type" -> "mor",
+      "hoodie.index.type" -> "INMEMORY",
+      "hoodie.compact.inline" -> "true",
+      "key123" -> "value456"
+    )
+    val tableConfigs = HoodieOptionConfig.deleteHooideOptions(sqlOptions)
+    Assertions.assertTrue(tableConfigs.size == 1)
+    Assertions.assertTrue(tableConfigs("key123") == "value456")
+  }
+
+  @Test
+  def testExtractSqlOptions(): Unit = {
+    val sqlOptions = Map("primaryKey" -> "id,addr",
+      "preCombineField" -> "timestamp",
+      "type" -> "mor",
+      "hoodie.index.type" -> "INMEMORY",
+      "hoodie.compact.inline" -> "true",
+      "key123" -> "value456"
+    )
+    val tableConfigs = HoodieOptionConfig.extractSqlOptions(sqlOptions)
+    Assertions.assertTrue(tableConfigs.size == 3)
+    Assertions.assertTrue(tableConfigs.keySet == Set("primaryKey", "preCombineField", "type"))
+  }
+
+  @Test
+  def testValidateTable(): Unit = {
+    val baseSqlOptions = Map(
+      "hoodie.datasource.write.hive_style_partitioning" -> "true",
+      "hoodie.datasource.write.partitionpath.urlencode" -> "false",
+      "hoodie.table.keygenerator.class" -> "org.apache.hudi.keygen.ComplexKeyGenerator"
+    )
+
+    val schema = StructType(
+      Seq(StructField("id", IntegerType, true),
+        StructField("name", StringType, true),
+        StructField("timestamp", TimestampType, true),
+        StructField("dt", StringType, true))
+    )
+
+    // miss primaryKey parameter
+    val sqlOptions1 = baseSqlOptions ++ Map(
+      "type" -> "mor"
+    )
+
+    val e1 = intercept[IllegalArgumentException] {
+      HoodieOptionConfig.validateTable(spark, schema, sqlOptions1)
+    }
+    Assertions.assertTrue(e1.getMessage.contains("No `primaryKey` is specified."))

Review comment:
       There are some problems with using junit5 in a Scala environment, just leave 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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7b38fc2781a74beb0db37de1f7c5e970eb44dd41 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327) 
   * 905d341133715a792d8d7073df14acfbf29860bd Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329) 
   
   <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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-967785259


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330",
       "triggerID" : "966919685",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3346",
       "triggerID" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2930c00a4375e00d2f12e8e663da805eb89259fd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3348",
       "triggerID" : "2930c00a4375e00d2f12e8e663da805eb89259fd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 701268085d201d24c1eefef115079ac4e4b4d40a Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3346) 
   * 2930c00a4375e00d2f12e8e663da805eb89259fd Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3348) 
   
   <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] YannByron commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
YannByron commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r748677927



##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala
##########
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hudi
+
+import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, OverwriteWithLatestAvroPayload}
+import org.apache.hudi.common.table.HoodieTableConfig
+import org.apache.hudi.testutils.HoodieClientTestBase
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.types._
+
+import org.junit.jupiter.api.{Assertions, BeforeEach, Test}
+
+import org.scalatest.Matchers.intercept
+
+class TestHoodieOptionConfig extends HoodieClientTestBase {
+
+  var spark: SparkSession = _
+
+  /**
+   * Setup method running before each test.
+   */
+  @BeforeEach override def setUp() {
+    initSparkContexts()
+    spark = sqlContext.sparkSession
+  }
+
+  @Test
+  def testWithDefaultSqlOptions(): Unit = {
+    val ops1 = Map("primaryKey" -> "id")
+    val with1 = HoodieOptionConfig.withDefaultSqlOptions(ops1)
+    Assertions.assertTrue(with1.size == 3)
+    Assertions.assertTrue(with1("primaryKey") == "id")
+    Assertions.assertTrue(with1("type") == "cow")
+    Assertions.assertTrue(with1("payloadClass") == classOf[DefaultHoodieRecordPayload].getName)
+
+    val ops2 = Map("primaryKey" -> "id",
+      "preCombineField" -> "timestamp",
+      "type" -> "mor",
+      "payloadClass" -> classOf[OverwriteWithLatestAvroPayload].getName
+    )
+    val with2 = HoodieOptionConfig.withDefaultSqlOptions(ops2)
+    Assertions.assertTrue(ops2 == with2)
+  }
+
+  @Test
+  def testMappingSqlOptionToTableConfig(): Unit = {
+    val sqlOptions = Map("primaryKey" -> "id,addr",
+      "preCombineField" -> "timestamp",
+      "type" -> "mor",
+      "hoodie.index.type" -> "INMEMORY",
+      "hoodie.compact.inline" -> "true"
+    )
+    val tableConfigs = HoodieOptionConfig.mappingSqlOptionToTableConfig(sqlOptions)
+
+    Assertions.assertTrue(tableConfigs.size == 5)
+    Assertions.assertTrue(tableConfigs(HoodieTableConfig.RECORDKEY_FIELDS.key) == "id,addr")
+    Assertions.assertTrue(tableConfigs(HoodieTableConfig.PRECOMBINE_FIELD.key) == "timestamp")
+    Assertions.assertTrue(tableConfigs(HoodieTableConfig.TYPE.key) == "MERGE_ON_READ")
+    Assertions.assertTrue(tableConfigs("hoodie.index.type") == "INMEMORY")
+    Assertions.assertTrue(tableConfigs("hoodie.compact.inline") == "true")
+  }
+
+  @Test
+  def testDeleteHooideOptions(): Unit = {
+    val sqlOptions = Map("primaryKey" -> "id,addr",
+      "preCombineField" -> "timestamp",
+      "type" -> "mor",
+      "hoodie.index.type" -> "INMEMORY",
+      "hoodie.compact.inline" -> "true",
+      "key123" -> "value456"
+    )
+    val tableConfigs = HoodieOptionConfig.deleteHooideOptions(sqlOptions)
+    Assertions.assertTrue(tableConfigs.size == 1)
+    Assertions.assertTrue(tableConfigs("key123") == "value456")
+  }
+
+  @Test
+  def testExtractSqlOptions(): Unit = {
+    val sqlOptions = Map("primaryKey" -> "id,addr",
+      "preCombineField" -> "timestamp",
+      "type" -> "mor",
+      "hoodie.index.type" -> "INMEMORY",
+      "hoodie.compact.inline" -> "true",
+      "key123" -> "value456"
+    )
+    val tableConfigs = HoodieOptionConfig.extractSqlOptions(sqlOptions)
+    Assertions.assertTrue(tableConfigs.size == 3)
+    Assertions.assertTrue(tableConfigs.keySet == Set("primaryKey", "preCombineField", "type"))
+  }
+
+  @Test
+  def testValidateTable(): Unit = {
+    val baseSqlOptions = Map(
+      "hoodie.datasource.write.hive_style_partitioning" -> "true",
+      "hoodie.datasource.write.partitionpath.urlencode" -> "false",
+      "hoodie.table.keygenerator.class" -> "org.apache.hudi.keygen.ComplexKeyGenerator"
+    )
+
+    val schema = StructType(
+      Seq(StructField("id", IntegerType, true),
+        StructField("name", StringType, true),
+        StructField("timestamp", TimestampType, true),
+        StructField("dt", StringType, true))
+    )
+
+    // miss primaryKey parameter
+    val sqlOptions1 = baseSqlOptions ++ Map(
+      "type" -> "mor"
+    )
+
+    val e1 = intercept[IllegalArgumentException] {
+      HoodieOptionConfig.validateTable(spark, schema, sqlOptions1)
+    }
+    Assertions.assertTrue(e1.getMessage.contains("No `primaryKey` is specified."))
+
+    // primary field not found
+    val sqlOptions2 = baseSqlOptions ++ Map(
+      "primaryKey" -> "xxx",
+      "type" -> "mor"
+    )
+    val e2 = intercept[IllegalArgumentException] {
+      HoodieOptionConfig.validateTable(spark, schema, sqlOptions2)
+    }
+    Assertions.assertTrue(e2.getMessage.contains("Can't find primary key"))
+
+    // preCombine field not found
+    val sqlOptions3 = baseSqlOptions ++ Map(
+      "primaryKey" -> "id",
+      "preCombineField" -> "ts",
+      "type" -> "mor"
+    )
+    val e3 = intercept[IllegalArgumentException] {
+      HoodieOptionConfig.validateTable(spark, schema, sqlOptions3)
+    }
+    Assertions.assertTrue(e3.getMessage.contains("Can't find precombine key"))
+
+    // miss type parameter
+    val sqlOptions4 = baseSqlOptions ++ Map(
+      "primaryKey" -> "id",
+      "preCombineField" -> "timestamp"
+    )
+    val e4 = intercept[IllegalArgumentException] {
+      HoodieOptionConfig.validateTable(spark, schema, sqlOptions4)
+    }
+    Assertions.assertTrue(e4.getMessage.contains("No `type` is specified."))
+
+    // type is invalid
+    val sqlOptions5 = baseSqlOptions ++ Map(
+      "primaryKey" -> "id",
+      "preCombineField" -> "timestamp",
+      "type" -> "abc"
+    )
+    val e5 = intercept[IllegalArgumentException] {
+      HoodieOptionConfig.validateTable(spark, schema, sqlOptions5)
+    }
+    Assertions.assertTrue(e5.getMessage.contains("'type' must be 'cow' or 'mor'"))
+
+    // right options and schema
+    val sqlOptions6 = baseSqlOptions ++ Map(
+      "primaryKey" -> "id",
+      "preCombineField" -> "timestamp",
+      "type" -> "cow"
+    )
+    HoodieOptionConfig.validateTable(spark, schema, sqlOptions6)

Review comment:
       ditto




-- 
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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-966830825


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d326b7b41a1d71492c1de795afa172e0a3c20c59 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313) 
   * 7b38fc2781a74beb0db37de1f7c5e970eb44dd41 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327) 
   * 905d341133715a792d8d7073df14acfbf29860bd 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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-966828104


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d326b7b41a1d71492c1de795afa172e0a3c20c59 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313) 
   * 7b38fc2781a74beb0db37de1f7c5e970eb44dd41 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327) 
   
   <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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330",
       "triggerID" : "966919685",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3346",
       "triggerID" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2930c00a4375e00d2f12e8e663da805eb89259fd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3348",
       "triggerID" : "2930c00a4375e00d2f12e8e663da805eb89259fd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2930c00a4375e00d2f12e8e663da805eb89259fd Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3348) 
   
   <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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330",
       "triggerID" : "966919685",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 905d341133715a792d8d7073df14acfbf29860bd Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330) 
   
   <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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-966261444


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c49f4cd59f44d0a033c88de9391c587fde73e06b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284) 
   * 02efbd6f34723b31c013b600f0ad41f4b2989fbb 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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 02efbd6f34723b31c013b600f0ad41f4b2989fbb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310) 
   * d326b7b41a1d71492c1de795afa172e0a3c20c59 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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-965351205


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d003fdb9a2f781de5a20f7550a6d17fb8351339a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189) 
   * c49f4cd59f44d0a033c88de9391c587fde73e06b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284) 
   
   <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] melin edited a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
melin edited a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-962556577


   The primary key syntax is consistent with other relational databases
   ```sql
   create table test_hudi_demo ( 
       id int, 
       name string, 
       price double,
       ds date)
   stored as hudi    
   primary key (id)
   lifeCycle 300
   ```


-- 
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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-966827185


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d326b7b41a1d71492c1de795afa172e0a3c20c59 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313) 
   * 7b38fc2781a74beb0db37de1f7c5e970eb44dd41 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] YannByron commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
YannByron commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r747988008



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
##########
@@ -141,7 +145,7 @@ object HoodieSparkSqlWriter {
           .setPartitionFields(partitionColumns)
           .setPopulateMetaFields(populateMetaFields)
           .setRecordKeyFields(hoodieConfig.getString(RECORDKEY_FIELD))
-          .setKeyGeneratorClassProp(hoodieConfig.getString(KEYGENERATOR_CLASS_NAME))
+          .setKeyGeneratorClassProp(HoodieWriterUtils.getRealKeyGenerator(hoodieConfig))

Review comment:
       ok, `getOriginKeyGenerator` is the chosen one.




-- 
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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330",
       "triggerID" : "966919685",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3346",
       "triggerID" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2930c00a4375e00d2f12e8e663da805eb89259fd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2930c00a4375e00d2f12e8e663da805eb89259fd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 701268085d201d24c1eefef115079ac4e4b4d40a Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3346) 
   * 2930c00a4375e00d2f12e8e663da805eb89259fd 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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d003fdb9a2f781de5a20f7550a6d17fb8351339a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189) 
   * c49f4cd59f44d0a033c88de9391c587fde73e06b 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] YannByron commented on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
YannByron commented on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-966919685


   @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] YannByron commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
YannByron commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r747937610



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
##########
@@ -141,7 +145,7 @@ object HoodieSparkSqlWriter {
           .setPartitionFields(partitionColumns)
           .setPopulateMetaFields(populateMetaFields)
           .setRecordKeyFields(hoodieConfig.getString(RECORDKEY_FIELD))
-          .setKeyGeneratorClassProp(hoodieConfig.getString(KEYGENERATOR_CLASS_NAME))
+          .setKeyGeneratorClassProp(HoodieWriterUtils.getRealKeyGenerator(hoodieConfig))

Review comment:
       For Spark-SQL, we assign `SqlKeyGenerator` to `hoodie.datasource.write.keygenerator.class`, and use `hoodie.sql.origin.keygen.class` to point the real user-defined key generator, for example `ComplexKeyGenerator`. When persist it  to hoodie.properties, we need to convert to the `real`one in spark-sql scene.

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala
##########
@@ -102,4 +106,75 @@ object HoodieWriterUtils {
     properties.putAll(mapAsJavaMap(parameters))
     new HoodieConfig(properties)
   }
+
+  def getRealKeyGenerator(hoodieConfig: HoodieConfig): String = {
+    val kg = hoodieConfig.getString(KEYGENERATOR_CLASS_NAME.key())
+    if (classOf[SqlKeyGenerator].getCanonicalName == kg) {

Review comment:
       ok




-- 
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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-966831627


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7b38fc2781a74beb0db37de1f7c5e970eb44dd41 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327) 
   * 905d341133715a792d8d7073df14acfbf29860bd 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] xushiyan commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
xushiyan commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r748619821



##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala
##########
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hudi
+
+import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, OverwriteWithLatestAvroPayload}
+import org.apache.hudi.common.table.HoodieTableConfig
+import org.apache.hudi.testutils.HoodieClientTestBase
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.types._
+
+import org.junit.jupiter.api.{Assertions, BeforeEach, Test}
+
+import org.scalatest.Matchers.intercept
+
+class TestHoodieOptionConfig extends HoodieClientTestBase {
+
+  var spark: SparkSession = _
+
+  /**
+   * Setup method running before each test.
+   */
+  @BeforeEach override def setUp() {
+    initSparkContexts()
+    spark = sqlContext.sparkSession
+  }
+
+  @Test
+  def testWithDefaultSqlOptions(): Unit = {
+    val ops1 = Map("primaryKey" -> "id")
+    val with1 = HoodieOptionConfig.withDefaultSqlOptions(ops1)
+    Assertions.assertTrue(with1.size == 3)

Review comment:
       suggestion: it'd be better with static import for assertXXX method, which makes test code more concise and cleaner; here Assertions are repeated many times but with no extra info

##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala
##########
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hudi
+
+import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, OverwriteWithLatestAvroPayload}
+import org.apache.hudi.common.table.HoodieTableConfig
+import org.apache.hudi.testutils.HoodieClientTestBase
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.types._
+
+import org.junit.jupiter.api.{Assertions, BeforeEach, Test}
+
+import org.scalatest.Matchers.intercept
+
+class TestHoodieOptionConfig extends HoodieClientTestBase {
+
+  var spark: SparkSession = _
+
+  /**
+   * Setup method running before each test.
+   */
+  @BeforeEach override def setUp() {
+    initSparkContexts()
+    spark = sqlContext.sparkSession
+  }
+
+  @Test
+  def testWithDefaultSqlOptions(): Unit = {
+    val ops1 = Map("primaryKey" -> "id")
+    val with1 = HoodieOptionConfig.withDefaultSqlOptions(ops1)
+    Assertions.assertTrue(with1.size == 3)
+    Assertions.assertTrue(with1("primaryKey") == "id")
+    Assertions.assertTrue(with1("type") == "cow")
+    Assertions.assertTrue(with1("payloadClass") == classOf[DefaultHoodieRecordPayload].getName)
+
+    val ops2 = Map("primaryKey" -> "id",
+      "preCombineField" -> "timestamp",
+      "type" -> "mor",
+      "payloadClass" -> classOf[OverwriteWithLatestAvroPayload].getName
+    )
+    val with2 = HoodieOptionConfig.withDefaultSqlOptions(ops2)
+    Assertions.assertTrue(ops2 == with2)
+  }
+
+  @Test
+  def testMappingSqlOptionToTableConfig(): Unit = {
+    val sqlOptions = Map("primaryKey" -> "id,addr",
+      "preCombineField" -> "timestamp",
+      "type" -> "mor",
+      "hoodie.index.type" -> "INMEMORY",
+      "hoodie.compact.inline" -> "true"
+    )
+    val tableConfigs = HoodieOptionConfig.mappingSqlOptionToTableConfig(sqlOptions)
+
+    Assertions.assertTrue(tableConfigs.size == 5)
+    Assertions.assertTrue(tableConfigs(HoodieTableConfig.RECORDKEY_FIELDS.key) == "id,addr")
+    Assertions.assertTrue(tableConfigs(HoodieTableConfig.PRECOMBINE_FIELD.key) == "timestamp")
+    Assertions.assertTrue(tableConfigs(HoodieTableConfig.TYPE.key) == "MERGE_ON_READ")
+    Assertions.assertTrue(tableConfigs("hoodie.index.type") == "INMEMORY")
+    Assertions.assertTrue(tableConfigs("hoodie.compact.inline") == "true")
+  }
+
+  @Test
+  def testDeleteHooideOptions(): Unit = {
+    val sqlOptions = Map("primaryKey" -> "id,addr",
+      "preCombineField" -> "timestamp",
+      "type" -> "mor",
+      "hoodie.index.type" -> "INMEMORY",
+      "hoodie.compact.inline" -> "true",
+      "key123" -> "value456"
+    )
+    val tableConfigs = HoodieOptionConfig.deleteHooideOptions(sqlOptions)
+    Assertions.assertTrue(tableConfigs.size == 1)
+    Assertions.assertTrue(tableConfigs("key123") == "value456")
+  }
+
+  @Test
+  def testExtractSqlOptions(): Unit = {
+    val sqlOptions = Map("primaryKey" -> "id,addr",
+      "preCombineField" -> "timestamp",
+      "type" -> "mor",
+      "hoodie.index.type" -> "INMEMORY",
+      "hoodie.compact.inline" -> "true",
+      "key123" -> "value456"
+    )
+    val tableConfigs = HoodieOptionConfig.extractSqlOptions(sqlOptions)
+    Assertions.assertTrue(tableConfigs.size == 3)
+    Assertions.assertTrue(tableConfigs.keySet == Set("primaryKey", "preCombineField", "type"))
+  }
+
+  @Test
+  def testValidateTable(): Unit = {
+    val baseSqlOptions = Map(
+      "hoodie.datasource.write.hive_style_partitioning" -> "true",
+      "hoodie.datasource.write.partitionpath.urlencode" -> "false",
+      "hoodie.table.keygenerator.class" -> "org.apache.hudi.keygen.ComplexKeyGenerator"
+    )
+
+    val schema = StructType(
+      Seq(StructField("id", IntegerType, true),
+        StructField("name", StringType, true),
+        StructField("timestamp", TimestampType, true),
+        StructField("dt", StringType, true))
+    )
+
+    // miss primaryKey parameter
+    val sqlOptions1 = baseSqlOptions ++ Map(
+      "type" -> "mor"
+    )
+
+    val e1 = intercept[IllegalArgumentException] {
+      HoodieOptionConfig.validateTable(spark, schema, sqlOptions1)
+    }
+    Assertions.assertTrue(e1.getMessage.contains("No `primaryKey` is specified."))

Review comment:
       you can use assertThrows(() -> {}) here; also applies to a few other occurrences below
   

##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala
##########
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hudi
+
+import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, OverwriteWithLatestAvroPayload}
+import org.apache.hudi.common.table.HoodieTableConfig
+import org.apache.hudi.testutils.HoodieClientTestBase
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.types._
+
+import org.junit.jupiter.api.{Assertions, BeforeEach, Test}
+
+import org.scalatest.Matchers.intercept
+
+class TestHoodieOptionConfig extends HoodieClientTestBase {
+
+  var spark: SparkSession = _
+
+  /**
+   * Setup method running before each test.
+   */
+  @BeforeEach override def setUp() {
+    initSparkContexts()
+    spark = sqlContext.sparkSession
+  }
+
+  @Test
+  def testWithDefaultSqlOptions(): Unit = {
+    val ops1 = Map("primaryKey" -> "id")
+    val with1 = HoodieOptionConfig.withDefaultSqlOptions(ops1)
+    Assertions.assertTrue(with1.size == 3)
+    Assertions.assertTrue(with1("primaryKey") == "id")
+    Assertions.assertTrue(with1("type") == "cow")
+    Assertions.assertTrue(with1("payloadClass") == classOf[DefaultHoodieRecordPayload].getName)
+
+    val ops2 = Map("primaryKey" -> "id",
+      "preCombineField" -> "timestamp",
+      "type" -> "mor",
+      "payloadClass" -> classOf[OverwriteWithLatestAvroPayload].getName
+    )
+    val with2 = HoodieOptionConfig.withDefaultSqlOptions(ops2)
+    Assertions.assertTrue(ops2 == with2)
+  }
+
+  @Test
+  def testMappingSqlOptionToTableConfig(): Unit = {
+    val sqlOptions = Map("primaryKey" -> "id,addr",
+      "preCombineField" -> "timestamp",
+      "type" -> "mor",
+      "hoodie.index.type" -> "INMEMORY",
+      "hoodie.compact.inline" -> "true"
+    )
+    val tableConfigs = HoodieOptionConfig.mappingSqlOptionToTableConfig(sqlOptions)
+
+    Assertions.assertTrue(tableConfigs.size == 5)
+    Assertions.assertTrue(tableConfigs(HoodieTableConfig.RECORDKEY_FIELDS.key) == "id,addr")
+    Assertions.assertTrue(tableConfigs(HoodieTableConfig.PRECOMBINE_FIELD.key) == "timestamp")
+    Assertions.assertTrue(tableConfigs(HoodieTableConfig.TYPE.key) == "MERGE_ON_READ")
+    Assertions.assertTrue(tableConfigs("hoodie.index.type") == "INMEMORY")
+    Assertions.assertTrue(tableConfigs("hoodie.compact.inline") == "true")
+  }
+
+  @Test
+  def testDeleteHooideOptions(): Unit = {
+    val sqlOptions = Map("primaryKey" -> "id,addr",
+      "preCombineField" -> "timestamp",
+      "type" -> "mor",
+      "hoodie.index.type" -> "INMEMORY",
+      "hoodie.compact.inline" -> "true",
+      "key123" -> "value456"
+    )
+    val tableConfigs = HoodieOptionConfig.deleteHooideOptions(sqlOptions)
+    Assertions.assertTrue(tableConfigs.size == 1)
+    Assertions.assertTrue(tableConfigs("key123") == "value456")
+  }
+
+  @Test
+  def testExtractSqlOptions(): Unit = {
+    val sqlOptions = Map("primaryKey" -> "id,addr",
+      "preCombineField" -> "timestamp",
+      "type" -> "mor",
+      "hoodie.index.type" -> "INMEMORY",
+      "hoodie.compact.inline" -> "true",
+      "key123" -> "value456"
+    )
+    val tableConfigs = HoodieOptionConfig.extractSqlOptions(sqlOptions)
+    Assertions.assertTrue(tableConfigs.size == 3)
+    Assertions.assertTrue(tableConfigs.keySet == Set("primaryKey", "preCombineField", "type"))
+  }
+
+  @Test
+  def testValidateTable(): Unit = {
+    val baseSqlOptions = Map(
+      "hoodie.datasource.write.hive_style_partitioning" -> "true",
+      "hoodie.datasource.write.partitionpath.urlencode" -> "false",
+      "hoodie.table.keygenerator.class" -> "org.apache.hudi.keygen.ComplexKeyGenerator"
+    )
+
+    val schema = StructType(
+      Seq(StructField("id", IntegerType, true),
+        StructField("name", StringType, true),
+        StructField("timestamp", TimestampType, true),
+        StructField("dt", StringType, true))
+    )
+
+    // miss primaryKey parameter
+    val sqlOptions1 = baseSqlOptions ++ Map(
+      "type" -> "mor"
+    )
+
+    val e1 = intercept[IllegalArgumentException] {
+      HoodieOptionConfig.validateTable(spark, schema, sqlOptions1)
+    }
+    Assertions.assertTrue(e1.getMessage.contains("No `primaryKey` is specified."))
+
+    // primary field not found
+    val sqlOptions2 = baseSqlOptions ++ Map(
+      "primaryKey" -> "xxx",
+      "type" -> "mor"
+    )
+    val e2 = intercept[IllegalArgumentException] {
+      HoodieOptionConfig.validateTable(spark, schema, sqlOptions2)
+    }
+    Assertions.assertTrue(e2.getMessage.contains("Can't find primary key"))
+
+    // preCombine field not found
+    val sqlOptions3 = baseSqlOptions ++ Map(
+      "primaryKey" -> "id",
+      "preCombineField" -> "ts",
+      "type" -> "mor"
+    )
+    val e3 = intercept[IllegalArgumentException] {
+      HoodieOptionConfig.validateTable(spark, schema, sqlOptions3)
+    }
+    Assertions.assertTrue(e3.getMessage.contains("Can't find precombine key"))
+
+    // miss type parameter
+    val sqlOptions4 = baseSqlOptions ++ Map(
+      "primaryKey" -> "id",
+      "preCombineField" -> "timestamp"
+    )
+    val e4 = intercept[IllegalArgumentException] {
+      HoodieOptionConfig.validateTable(spark, schema, sqlOptions4)
+    }
+    Assertions.assertTrue(e4.getMessage.contains("No `type` is specified."))
+
+    // type is invalid
+    val sqlOptions5 = baseSqlOptions ++ Map(
+      "primaryKey" -> "id",
+      "preCombineField" -> "timestamp",
+      "type" -> "abc"
+    )
+    val e5 = intercept[IllegalArgumentException] {
+      HoodieOptionConfig.validateTable(spark, schema, sqlOptions5)
+    }
+    Assertions.assertTrue(e5.getMessage.contains("'type' must be 'cow' or 'mor'"))
+
+    // right options and schema
+    val sqlOptions6 = baseSqlOptions ++ Map(
+      "primaryKey" -> "id",
+      "preCombineField" -> "timestamp",
+      "type" -> "cow"
+    )
+    HoodieOptionConfig.validateTable(spark, schema, sqlOptions6)

Review comment:
       use assertNotThrow(() -> {}) to provide assertion failure message?
   
   

##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/HoodieSparkSqlWriterSuite2.scala
##########
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi
+
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.keygen.{ComplexKeyGenerator, SimpleKeyGenerator}
+
+import org.apache.spark.sql.hudi.command.SqlKeyGenerator
+
+import org.junit.jupiter.api.Assertions.assertTrue
+import org.junit.jupiter.api.Test
+
+class HoodieSparkSqlWriterSuite2 {
+
+  @Test
+  def testGetRealKeyGenerator(): Unit = {

Review comment:
       test name also update?




-- 
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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * de33f59e26e56e2e5e23e0a72a26587ecdbaacbb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186) 
   * 58abfc536ba3fc11a00f65226ad5574c324c4496 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187) 
   
   <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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * de33f59e26e56e2e5e23e0a72a26587ecdbaacbb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186) 
   
   <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 pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
xushiyan commented on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-963824070


   @YannByron just did a rough pass over the changes. will do another round on details soon. 


-- 
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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c49f4cd59f44d0a033c88de9391c587fde73e06b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284) 
   * 02efbd6f34723b31c013b600f0ad41f4b2989fbb 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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-967785015


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330",
       "triggerID" : "966919685",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3346",
       "triggerID" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2930c00a4375e00d2f12e8e663da805eb89259fd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2930c00a4375e00d2f12e8e663da805eb89259fd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 701268085d201d24c1eefef115079ac4e4b4d40a Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3346) 
   * 2930c00a4375e00d2f12e8e663da805eb89259fd 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] xushiyan commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
xushiyan commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r748680477



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala
##########
@@ -102,4 +106,75 @@ object HoodieWriterUtils {
     properties.putAll(mapAsJavaMap(parameters))
     new HoodieConfig(properties)
   }
+
+  def getRealKeyGenerator(hoodieConfig: HoodieConfig): String = {
+    val kg = hoodieConfig.getString(KEYGENERATOR_CLASS_NAME.key())
+    if (classOf[SqlKeyGenerator].getCanonicalName == kg) {
+      hoodieConfig.getString(SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME)
+    } else {
+      kg
+    }
+  }
+
+  /**
+   * Detects conflicts between new parameters and existing table configurations
+   */
+  def validateTableConfig(spark: SparkSession, params: Map[String, String],
+      tableConfig: HoodieConfig): Unit = {
+    val resolver = spark.sessionState.conf.resolver
+    val diffConfigs = StringBuilder.newBuilder
+    params.foreach { case (key, value) =>
+      val existingValue = getStringFromTableConfigWithAlternatives(tableConfig, key)
+      if (null != existingValue && !resolver(existingValue, value)) {
+        diffConfigs.append(s"$key:\t$value\t${tableConfig.getString(key)}\n")
+      }
+    }
+
+    if (null != tableConfig) {
+      val datasourceRecordKey = params.getOrElse(RECORDKEY_FIELD.key(), null)
+      val tableConfigRecordKey = tableConfig.getString(HoodieTableConfig.RECORDKEY_FIELDS)
+      if (null != datasourceRecordKey && null != tableConfigRecordKey
+          && datasourceRecordKey != tableConfigRecordKey) {
+        diffConfigs.append(s"RecordKey:\t$datasourceRecordKey\t$tableConfigRecordKey\n")
+      }
+
+      val datasourcePreCombineKey = params.getOrElse(PRECOMBINE_FIELD.key(), null)
+      val tableConfigPreCombineKey = tableConfig.getString(HoodieTableConfig.PRECOMBINE_FIELD)
+      if (null != datasourcePreCombineKey && null != tableConfigPreCombineKey
+          && datasourcePreCombineKey != tableConfigPreCombineKey) {
+        diffConfigs.append(s"PreCombineKey:\t$datasourcePreCombineKey\t$tableConfigPreCombineKey\n")
+      }
+
+      val datasourceKeyGen = {
+        val kg = params.getOrElse(KEYGENERATOR_CLASS_NAME.key(), null)
+        if (classOf[SqlKeyGenerator].getCanonicalName == kg) {

Review comment:
       @YannByron this duplicate code not addressed yet.




-- 
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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-966952634


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330",
       "triggerID" : "966919685",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 905d341133715a792d8d7073df14acfbf29860bd Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330) 
   
   <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 change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
xushiyan commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r747299001



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala
##########
@@ -242,8 +244,13 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab
     // Append the table schema to the parameters. In the case of merge into, the schema of sourceDF
     // may be different from the target table, because the are transform logical in the update or
     // insert actions.
+    val opertion = if (StringUtils.isNullOrEmpty(parameters.getOrElse(PRECOMBINE_FIELD.key, ""))) {

Review comment:
       ```suggestion
       val operation = if (StringUtils.isNullOrEmpty(parameters.getOrElse(PRECOMBINE_FIELD.key, ""))) {
   ```

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
##########
@@ -38,92 +42,125 @@ import org.apache.spark.sql.hudi.HoodieOptionConfig
 import org.apache.spark.sql.hudi.HoodieSqlUtils._
 import org.apache.spark.sql.hudi.command.CreateHoodieTableCommand.{initTableIfNeed, isEmptyPath}
 import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD
-import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.types.{StructField, StructType}
 import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
 import org.apache.spark.{SPARK_VERSION, SparkConf}
-import java.util.{Locale, Properties}
 
-import org.apache.hudi.exception.HoodieException
-import org.apache.hudi.keygen.ComplexKeyGenerator
-import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
+import java.util.{Locale, Properties}
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable
+import scala.util.control.NonFatal
 
 /**
  * Command for create hoodie table.
  */
 case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean)
   extends RunnableCommand with SparkAdapterSupport {
 
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    val tableName = table.identifier.unquotedString
+  val tableName = formatName(table.identifier.table)
+
+  val tblProperties = table.storage.properties ++ table.properties
 
+  override def run(sparkSession: SparkSession): Seq[Row] = {
     val tableIsExists = sparkSession.sessionState.catalog.tableExists(table.identifier)
     if (tableIsExists) {
       if (ignoreIfExists) {
         // scalastyle:off
         return Seq.empty[Row]
         // scalastyle:on
       } else {
-        throw new IllegalArgumentException(s"Table $tableName already exists.")
+        throw new IllegalArgumentException(s"Table ${table.identifier.unquotedString} already exists.")
       }
     }
-    // Create table in the catalog
-    val createTable = createTableInCatalog(sparkSession)
+
+    // get schema with meta fields, table config if hudi table exists, options including
+    // table configs and properties of the catalog tabe

Review comment:
       ```suggestion
       // table configs and properties of the catalog table
   ```

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala
##########
@@ -171,7 +180,48 @@ object HoodieOptionConfig {
 
   def getPreCombineField(options: Map[String, String]): Option[String] = {
     val params = mappingSqlOptionToHoodieParam(options)
-    params.get(DataSourceWriteOptions.PRECOMBINE_FIELD.key)
+    params.get(DataSourceWriteOptions.PRECOMBINE_FIELD.key).filter(_.nonEmpty)
+  }
+
+  def deleteHooideOptions(options: Map[String, String]): Map[String, String] = {

Review comment:
       think now it's a good time to add UT for HoodieOptionConfig, at least to cover the new and changed methods.
   
   

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala
##########
@@ -242,8 +244,13 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab
     // Append the table schema to the parameters. In the case of merge into, the schema of sourceDF
     // may be different from the target table, because the are transform logical in the update or
     // insert actions.
+    val opertion = if (StringUtils.isNullOrEmpty(parameters.getOrElse(PRECOMBINE_FIELD.key, ""))) {

Review comment:
       `PRECOMBINE_FIELD` is meant to determine ordering. If no ordering specified, then we perform insert. This looks like a new behavior, which I'm ok with. Just want to make sure this will be documented somewhere for users to look up.




-- 
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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d326b7b41a1d71492c1de795afa172e0a3c20c59 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313) 
   * 7b38fc2781a74beb0db37de1f7c5e970eb44dd41 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327) 
   * 905d341133715a792d8d7073df14acfbf29860bd 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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3330",
       "triggerID" : "966919685",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3346",
       "triggerID" : "701268085d201d24c1eefef115079ac4e4b4d40a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 701268085d201d24c1eefef115079ac4e4b4d40a Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3346) 
   
   <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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * de33f59e26e56e2e5e23e0a72a26587ecdbaacbb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186) 
   * 58abfc536ba3fc11a00f65226ad5574c324c4496 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187) 
   * d003fdb9a2f781de5a20f7550a6d17fb8351339a 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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-962545529


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * de33f59e26e56e2e5e23e0a72a26587ecdbaacbb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186) 
   
   <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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * d003fdb9a2f781de5a20f7550a6d17fb8351339a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189) 
   
   <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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-966264669


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 02efbd6f34723b31c013b600f0ad41f4b2989fbb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310) 
   
   <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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 02efbd6f34723b31c013b600f0ad41f4b2989fbb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310) 
   * d326b7b41a1d71492c1de795afa172e0a3c20c59 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313) 
   
   <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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-966332584


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 02efbd6f34723b31c013b600f0ad41f4b2989fbb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310) 
   * d326b7b41a1d71492c1de795afa172e0a3c20c59 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] leesf commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
leesf commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r747547228



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala
##########
@@ -102,4 +106,75 @@ object HoodieWriterUtils {
     properties.putAll(mapAsJavaMap(parameters))
     new HoodieConfig(properties)
   }
+
+  def getRealKeyGenerator(hoodieConfig: HoodieConfig): String = {
+    val kg = hoodieConfig.getString(KEYGENERATOR_CLASS_NAME.key())
+    if (classOf[SqlKeyGenerator].getCanonicalName == kg) {

Review comment:
       is there uts cover the logic?




-- 
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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-962548134


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * de33f59e26e56e2e5e23e0a72a26587ecdbaacbb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186) 
   * 58abfc536ba3fc11a00f65226ad5574c324c4496 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187) 
   * d003fdb9a2f781de5a20f7550a6d17fb8351339a 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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * de33f59e26e56e2e5e23e0a72a26587ecdbaacbb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186) 
   * 58abfc536ba3fc11a00f65226ad5574c324c4496 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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d003fdb9a2f781de5a20f7550a6d17fb8351339a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189) 
   * c49f4cd59f44d0a033c88de9391c587fde73e06b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284) 
   
   <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] YannByron commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
YannByron commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r747937757



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
##########
@@ -38,92 +42,125 @@ import org.apache.spark.sql.hudi.HoodieOptionConfig
 import org.apache.spark.sql.hudi.HoodieSqlUtils._
 import org.apache.spark.sql.hudi.command.CreateHoodieTableCommand.{initTableIfNeed, isEmptyPath}
 import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD
-import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.types.{StructField, StructType}
 import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
 import org.apache.spark.{SPARK_VERSION, SparkConf}
-import java.util.{Locale, Properties}
 
-import org.apache.hudi.exception.HoodieException
-import org.apache.hudi.keygen.ComplexKeyGenerator
-import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
+import java.util.{Locale, Properties}
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable
+import scala.util.control.NonFatal
 
 /**
  * Command for create hoodie table.
  */
 case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean)
   extends RunnableCommand with SparkAdapterSupport {
 
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    val tableName = table.identifier.unquotedString
+  val tableName = formatName(table.identifier.table)
+
+  val tblProperties = table.storage.properties ++ table.properties
 
+  override def run(sparkSession: SparkSession): Seq[Row] = {
     val tableIsExists = sparkSession.sessionState.catalog.tableExists(table.identifier)
     if (tableIsExists) {
       if (ignoreIfExists) {
         // scalastyle:off
         return Seq.empty[Row]
         // scalastyle:on
       } else {
-        throw new IllegalArgumentException(s"Table $tableName already exists.")
+        throw new IllegalArgumentException(s"Table ${table.identifier.unquotedString} already exists.")
       }
     }
-    // Create table in the catalog
-    val createTable = createTableInCatalog(sparkSession)
+
+    // get schema with meta fields, table config if hudi table exists, options including
+    // table configs and properties of the catalog table
+    val path = getTableLocation(table, sparkSession)
+    val (finalSchema, existingTableConfig, tableSqlOptions) = parseSchemaAndConfigs(sparkSession, path)
+
     // Init the hoodie.properties
-    initTableIfNeed(sparkSession, createTable)
+    initTableIfNeed(sparkSession, tableName, path, finalSchema,
+      table.partitionColumnNames, existingTableConfig, tableSqlOptions)
+
+    try {
+      // Create table in the catalog
+      createTableInCatalog(sparkSession, finalSchema, tableSqlOptions)
+    } catch {
+      case NonFatal(e) =>
+        logWarning(s"Failed to create catalog table in metastore: ${e.getMessage}")
+        e.printStackTrace()

Review comment:
       ok




-- 
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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-966854747


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7b38fc2781a74beb0db37de1f7c5e970eb44dd41 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327) 
   * 905d341133715a792d8d7073df14acfbf29860bd Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329) 
   
   <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] YannByron commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
YannByron commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r748677825



##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/HoodieSparkSqlWriterSuite2.scala
##########
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi
+
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.keygen.{ComplexKeyGenerator, SimpleKeyGenerator}
+
+import org.apache.spark.sql.hudi.command.SqlKeyGenerator
+
+import org.junit.jupiter.api.Assertions.assertTrue
+import org.junit.jupiter.api.Test
+
+class HoodieSparkSqlWriterSuite2 {
+
+  @Test
+  def testGetRealKeyGenerator(): Unit = {

Review comment:
       done.




-- 
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] YannByron commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
YannByron commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r747941171



##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala
##########
@@ -475,8 +478,8 @@ class TestDataSourceForBootstrap {
   }
 
   def runMetadataBootstrapAndVerifyCommit(tableType: String,
-                                          partitionColumns: Option[String] = None,
-                                          extraOpts: Map[String, String] = Map.empty): String = {
+      partitionColumns: Option[String] = None,

Review comment:
       this's a personal practice that format codes involved in this commit. i recover this.




-- 
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] YannByron commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
YannByron commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r747939188



##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala
##########
@@ -38,7 +38,7 @@ class TestAlterTable extends TestHoodieSqlBase {
              |  ts long
              |) using hudi
              | location '$tablePath'
-             | options (
+             | tblproperties (

Review comment:
       `options` will be saved to `storage.properties` in hive metastore. and `tblproperties` to `properties`.
   Using `tblproperties` is a more proper way. Also for compatibility, we  still support `options`.




-- 
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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-965345186


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d003fdb9a2f781de5a20f7550a6d17fb8351339a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189) 
   * c49f4cd59f44d0a033c88de9391c587fde73e06b 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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c49f4cd59f44d0a033c88de9391c587fde73e06b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284) 
   
   <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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-962548315


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 58abfc536ba3fc11a00f65226ad5574c324c4496 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187) 
   * d003fdb9a2f781de5a20f7550a6d17fb8351339a 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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-962547021


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * de33f59e26e56e2e5e23e0a72a26587ecdbaacbb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186) 
   * 58abfc536ba3fc11a00f65226ad5574c324c4496 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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * d003fdb9a2f781de5a20f7550a6d17fb8351339a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189) 
   
   <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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c49f4cd59f44d0a033c88de9391c587fde73e06b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284) 
   * 02efbd6f34723b31c013b600f0ad41f4b2989fbb Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310) 
   
   <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] YannByron commented on a change in pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
YannByron commented on a change in pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#discussion_r747356716



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala
##########
@@ -242,8 +244,13 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab
     // Append the table schema to the parameters. In the case of merge into, the schema of sourceDF
     // may be different from the target table, because the are transform logical in the update or
     // insert actions.
+    val opertion = if (StringUtils.isNullOrEmpty(parameters.getOrElse(PRECOMBINE_FIELD.key, ""))) {

Review comment:
       Yes, this behavior keeps consistent with dataframe api.
   if no PRECOMBINE_FIELD is specified, will throw exception if use `upsert`, but will work if use `insert`.




-- 
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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-966263150


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c49f4cd59f44d0a033c88de9391c587fde73e06b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284) 
   * 02efbd6f34723b31c013b600f0ad41f4b2989fbb Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310) 
   
   <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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7b38fc2781a74beb0db37de1f7c5e970eb44dd41 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327) 
   * 905d341133715a792d8d7073df14acfbf29860bd 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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 905d341133715a792d8d7073df14acfbf29860bd Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329) 
   
   <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 removed a comment on pull request #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #3936:
URL: https://github.com/apache/hudi/pull/3936#issuecomment-966874831


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3187",
       "triggerID" : "58abfc536ba3fc11a00f65226ad5574c324c4496",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3188",
       "triggerID" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d003fdb9a2f781de5a20f7550a6d17fb8351339a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3189",
       "triggerID" : "962556410",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3284",
       "triggerID" : "c49f4cd59f44d0a033c88de9391c587fde73e06b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3310",
       "triggerID" : "02efbd6f34723b31c013b600f0ad41f4b2989fbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3313",
       "triggerID" : "d326b7b41a1d71492c1de795afa172e0a3c20c59",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3327",
       "triggerID" : "7b38fc2781a74beb0db37de1f7c5e970eb44dd41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "905d341133715a792d8d7073df14acfbf29860bd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329",
       "triggerID" : "905d341133715a792d8d7073df14acfbf29860bd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 905d341133715a792d8d7073df14acfbf29860bd Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3329) 
   
   <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 #3936: [WIP][HUDI-2706] refactor spark-sql to make consistent with DataFrame api

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186",
       "triggerID" : "de33f59e26e56e2e5e23e0a72a26587ecdbaacbb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * de33f59e26e56e2e5e23e0a72a26587ecdbaacbb Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=3186) 
   
   <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