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

[GitHub] [hudi] waywtdcc opened a new pull request, #7009: [HUDI-5058]Fix read spark table error : primary key col cat not nullable

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

   ### Change Logs
   
   Fix read spark table error : primary key col cat not nullable
   
   ### Impact
   
   
   ### Risk level (write none, low medium or high below)
   
   
   ### Documentation Update
   
   Fix read spark table error : primary key col cat not nullable
   
   
   ### Contributor's checklist
   
   - [Y] Read through [contributor's guide](https://hudi.apache.org/contribute/how-to-contribute)
   - [ Y] Change Logs and Impact were stated clearly
   - [ Y] Adequate tests were added if applicable
   - [ Y] CI passed
   


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on pull request #7009: [HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable

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

   The failed test case `testInsertAndCleanByCommits` has no relationship with this patch, would merge it 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] alexeykudinkin commented on a diff in pull request #7009: [HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java:
##########
@@ -397,17 +399,22 @@ public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistExcep
     String path = hiveTable.getSd().getLocation();
     Map<String, String> parameters = hiveTable.getParameters();
     Schema latestTableSchema = StreamerUtil.getLatestTableSchema(path, hiveConf);
+    String pkColumnsStr = parameters.get(FlinkOptions.RECORD_KEY_FIELD.key());
+    List<String> pkColumns = StringUtils.isNullOrEmpty(pkColumnsStr)
+        ? null : StringUtils.split(pkColumnsStr, ",");
     org.apache.flink.table.api.Schema schema;
     if (latestTableSchema != null) {
+      // if the table is initialized from spark, the write schema is nullable for pk columns.
+      DataType tableDataType = DataTypeUtils.ensureColumnsAsNonNullable(

Review Comment:
   Yeah, i don't think this in Spark DS we enforce invariant that primary-keys has to be non-null



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/DataTypeUtils.java:
##########
@@ -123,4 +128,39 @@ public static Object resolvePartition(String partition, DataType type) {
                 "Can not convert %s to type %s for partition value", partition, type));
     }
   }
+
+  /**
+   * Ensures the give columns of the row data type are not nullable(for example, the primary keys).
+   *
+   * @param dataType  The row data type
+   * @param pkColumns The primary keys
+   * @return a new row data type if any column nullability is tweaked or the original data type
+   */
+  public static DataType ensureColumnsAsNonNullable(DataType dataType, @Nullable List<String> pkColumns) {
+    if (pkColumns == null || pkColumns.isEmpty()) {
+      return dataType;
+    }
+    RowType rowType = (RowType) dataType.getLogicalType();
+    List<DataType> oriFieldTypes = dataType.getChildren();

Review Comment:
   nit: `originalFieldTypes`



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/DataTypeUtils.java:
##########
@@ -123,4 +128,39 @@ public static Object resolvePartition(String partition, DataType type) {
                 "Can not convert %s to type %s for partition value", partition, type));
     }
   }
+
+  /**
+   * Ensures the give columns of the row data type are not nullable(for example, the primary keys).
+   *
+   * @param dataType  The row data type
+   * @param pkColumns The primary keys
+   * @return a new row data type if any column nullability is tweaked or the original data type
+   */
+  public static DataType ensureColumnsAsNonNullable(DataType dataType, @Nullable List<String> pkColumns) {
+    if (pkColumns == null || pkColumns.isEmpty()) {
+      return dataType;
+    }
+    RowType rowType = (RowType) dataType.getLogicalType();
+    List<DataType> oriFieldTypes = dataType.getChildren();
+    List<String> fieldNames = rowType.getFieldNames();
+    List<DataType> fieldTypes = new ArrayList<>();
+    boolean tweaked = false;
+    for (int i = 0; i < fieldNames.size(); i++) {
+      if (pkColumns.contains(fieldNames.get(i)) && rowType.getTypeAt(i).isNullable()) {
+        fieldTypes.add(oriFieldTypes.get(i).notNull());
+        tweaked = true;
+      } else {
+        fieldTypes.add(oriFieldTypes.get(i));
+      }
+    }
+    if (!tweaked) {
+      return dataType;
+    }
+    List<DataTypes.Field> fields = new ArrayList<>();
+    for (int i = 0; i < fieldNames.size(); i++) {
+      fields.add(DataTypes.FIELD(fieldNames.get(i), fieldTypes.get(i)));
+    }
+    return DataTypes.ROW(fields.toArray(new DataTypes.Field[fields.size()])).notNull();

Review Comment:
   Better to do: `fields.toArray(DataTypes:Field[]::new)`



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/DataTypeUtils.java:
##########
@@ -123,4 +128,39 @@ public static Object resolvePartition(String partition, DataType type) {
                 "Can not convert %s to type %s for partition value", partition, type));
     }
   }
+
+  /**
+   * Ensures the give columns of the row data type are not nullable(for example, the primary keys).
+   *
+   * @param dataType  The row data type
+   * @param pkColumns The primary keys
+   * @return a new row data type if any column nullability is tweaked or the original data type
+   */
+  public static DataType ensureColumnsAsNonNullable(DataType dataType, @Nullable List<String> pkColumns) {

Review Comment:
   nit: this method should be generic no reason to couple it to primary-keys (we can call it for non-PK use-cases as well)



-- 
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] waywtdcc commented on pull request #7009: [HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable

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

   @danny0405 @alexeykudinkin Hello, can you review this PR?


-- 
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 #7009: [HUDI-5058]Fix read spark table error : primary key col cat not nullable

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12378",
       "triggerID" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12378) 
   
   <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 #7009: [HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12378",
       "triggerID" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96c1c1d926bd2eb31c9bfcc214df2e8a428233d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12417",
       "triggerID" : "96c1c1d926bd2eb31c9bfcc214df2e8a428233d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fac16008e171289419e6477710ac020712b5067a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12547",
       "triggerID" : "fac16008e171289419e6477710ac020712b5067a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77dd5d5a4053a6628c0147f2547e87594084aee6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77dd5d5a4053a6628c0147f2547e87594084aee6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fac16008e171289419e6477710ac020712b5067a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12547) 
   * 77dd5d5a4053a6628c0147f2547e87594084aee6 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 #7009: [HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12378",
       "triggerID" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96c1c1d926bd2eb31c9bfcc214df2e8a428233d0",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12417",
       "triggerID" : "96c1c1d926bd2eb31c9bfcc214df2e8a428233d0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 96c1c1d926bd2eb31c9bfcc214df2e8a428233d0 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12417) 
   
   <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 #7009: [HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12378",
       "triggerID" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96c1c1d926bd2eb31c9bfcc214df2e8a428233d0",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12417",
       "triggerID" : "96c1c1d926bd2eb31c9bfcc214df2e8a428233d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fac16008e171289419e6477710ac020712b5067a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12547",
       "triggerID" : "fac16008e171289419e6477710ac020712b5067a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 96c1c1d926bd2eb31c9bfcc214df2e8a428233d0 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12417) 
   * fac16008e171289419e6477710ac020712b5067a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12547) 
   
   <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 #7009: [HUDI-5058]Fix read spark table error : primary key col cat not nullable

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229 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] alexeykudinkin commented on a diff in pull request #7009: [HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java:
##########
@@ -397,17 +399,22 @@ public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistExcep
     String path = hiveTable.getSd().getLocation();
     Map<String, String> parameters = hiveTable.getParameters();
     Schema latestTableSchema = StreamerUtil.getLatestTableSchema(path, hiveConf);
+    String pkColumnsStr = parameters.get(FlinkOptions.RECORD_KEY_FIELD.key());
+    List<String> pkColumns = StringUtils.isNullOrEmpty(pkColumnsStr)
+        ? null : StringUtils.split(pkColumnsStr, ",");
     org.apache.flink.table.api.Schema schema;
     if (latestTableSchema != null) {
+      // if the table is initialized from spark, the write schema is nullable for pk columns.
+      DataType tableDataType = DataTypeUtils.ensureColumnsAsNonNullable(

Review Comment:
   @danny0405 i actually think i've faced with issue in other place as well. Is there a reason why by default we mark all columns as nullable in Flink?



-- 
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] waywtdcc commented on a diff in pull request #7009: [HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/DataTypeUtils.java:
##########
@@ -123,4 +128,39 @@ public static Object resolvePartition(String partition, DataType type) {
                 "Can not convert %s to type %s for partition value", partition, type));
     }
   }
+
+  /**
+   * Ensures the give columns of the row data type are not nullable(for example, the primary keys).
+   *
+   * @param dataType  The row data type
+   * @param pkColumns The primary keys
+   * @return a new row data type if any column nullability is tweaked or the original data type
+   */
+  public static DataType ensureColumnsAsNonNullable(DataType dataType, @Nullable List<String> pkColumns) {

Review Comment:
   Hello, thank you for your reviews. This method is used to convert the primary key field into not null type.



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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #7009: [HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java:
##########
@@ -397,17 +399,22 @@ public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistExcep
     String path = hiveTable.getSd().getLocation();
     Map<String, String> parameters = hiveTable.getParameters();
     Schema latestTableSchema = StreamerUtil.getLatestTableSchema(path, hiveConf);
+    String pkColumnsStr = parameters.get(FlinkOptions.RECORD_KEY_FIELD.key());
+    List<String> pkColumns = StringUtils.isNullOrEmpty(pkColumnsStr)
+        ? null : StringUtils.split(pkColumnsStr, ",");
     org.apache.flink.table.api.Schema schema;
     if (latestTableSchema != null) {
+      // if the table is initialized from spark, the write schema is nullable for pk columns.
+      DataType tableDataType = DataTypeUtils.ensureColumnsAsNonNullable(

Review Comment:
   It is a common behavior: a column is by default nullable if user does not declare the nullability in DDL. And for primary keys, they must be forced as non-nullable.
   
   Flink would generate correct avro schema if the table was initialized from Flink app, what we fix here is a table created by Spark, so i guess, spark does not take the primary key constraint into nullability somewhere.



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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #7009: [HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/DataTypeUtils.java:
##########
@@ -123,4 +128,39 @@ public static Object resolvePartition(String partition, DataType type) {
                 "Can not convert %s to type %s for partition value", partition, type));
     }
   }
+
+  /**
+   * Ensures the give columns of the row data type are not nullable(for example, the primary keys).
+   *
+   * @param dataType  The row data type
+   * @param pkColumns The primary keys
+   * @return a new row data type if any column nullability is tweaked or the original data type
+   */
+  public static DataType ensureColumnsAsNonNullable(DataType dataType, @Nullable List<String> pkColumns) {

Review Comment:
   @waywtdcc i understand that. What i'm saying though is that there's no reason for us to couple it to primary keys (it should be generic to be able to handle any columns)



-- 
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] waywtdcc commented on pull request #7009: [HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable

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

   > [5058.patch.zip](https://github.com/apache/hudi/files/9835480/5058.patch.zip) Thanks for the contribution, i have reviewed and applied a patch.
   
   I have merged patches and modified a small exception and push. Please review again.


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on pull request #7009: [HUDI-5058]Fix read spark table error : primary key col can not be nullable

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

   [5058.patch.zip](https://github.com/apache/hudi/files/9835480/5058.patch.zip)
   Thanks for the contribution, i have reviewed and applied a patch.


-- 
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] waywtdcc commented on pull request #7009: [HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable

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

   > [5058.patch.zip](https://github.com/apache/hudi/files/9835480/5058.patch.zip) Thanks for the contribution, i have reviewed and applied a patch.
   
   Thank you for your reviews. Do you mean to let me merge this patch into my own branch?


-- 
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 #7009: [HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12378",
       "triggerID" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96c1c1d926bd2eb31c9bfcc214df2e8a428233d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12417",
       "triggerID" : "96c1c1d926bd2eb31c9bfcc214df2e8a428233d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fac16008e171289419e6477710ac020712b5067a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12547",
       "triggerID" : "fac16008e171289419e6477710ac020712b5067a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "77dd5d5a4053a6628c0147f2547e87594084aee6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12580",
       "triggerID" : "77dd5d5a4053a6628c0147f2547e87594084aee6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 77dd5d5a4053a6628c0147f2547e87594084aee6 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12580) 
   
   <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 #7009: [HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12378",
       "triggerID" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96c1c1d926bd2eb31c9bfcc214df2e8a428233d0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12417",
       "triggerID" : "96c1c1d926bd2eb31c9bfcc214df2e8a428233d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fac16008e171289419e6477710ac020712b5067a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12547",
       "triggerID" : "fac16008e171289419e6477710ac020712b5067a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fac16008e171289419e6477710ac020712b5067a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12547) 
   
   <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 #7009: [HUDI-5058]Fix read spark table error : primary key col can not be nullable

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12378",
       "triggerID" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12378) 
   
   <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 #7009: [HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12378",
       "triggerID" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96c1c1d926bd2eb31c9bfcc214df2e8a428233d0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12417",
       "triggerID" : "96c1c1d926bd2eb31c9bfcc214df2e8a428233d0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12378) 
   * 96c1c1d926bd2eb31c9bfcc214df2e8a428233d0 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12417) 
   
   <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 #7009: [HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12378",
       "triggerID" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96c1c1d926bd2eb31c9bfcc214df2e8a428233d0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "96c1c1d926bd2eb31c9bfcc214df2e8a428233d0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12378) 
   * 96c1c1d926bd2eb31c9bfcc214df2e8a428233d0 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 #7009: [HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12378",
       "triggerID" : "da6ea1d7ad1d84eeb8ea2b0935711f1dd0a07229",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96c1c1d926bd2eb31c9bfcc214df2e8a428233d0",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12417",
       "triggerID" : "96c1c1d926bd2eb31c9bfcc214df2e8a428233d0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fac16008e171289419e6477710ac020712b5067a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "fac16008e171289419e6477710ac020712b5067a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 96c1c1d926bd2eb31c9bfcc214df2e8a428233d0 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12417) 
   * fac16008e171289419e6477710ac020712b5067a 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 #7009: [HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable

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

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


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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 merged pull request #7009: [HUDI-5058]Fix flink catalog read spark table error : primary key col can not be nullable

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


-- 
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