Uploaded image for project: 'Apache Hudi'
  1. Apache Hudi
  2. HUDI-4966

Meta sync throws exception if TimestampBasedKeyGenerator is used to generate partition path containing slashes

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Critical
    • Resolution: Fixed
    • None
    • 0.12.2
    • None

    Description

      For Deltastreamer, when using TimestampBasedKeyGenerator with the output format of partition path containing slashes, e.g., "yyyy/MM/dd", and hive-style partitioning disabled (by default), the meta sync fails.

      --hoodie-conf hoodie.datasource.write.partitionpath.field=createdDate
      --hoodie-conf hoodie.datasource.write.keygenerator.class=org.apache.hudi.keygen.TimestampBasedKeyGenerator
      --hoodie-conf hoodie.deltastreamer.keygen.timebased.timezone=GMT
      --hoodie-conf hoodie.deltastreamer.keygen.timebased.output.dateformat=yyyy/MM/dd
      --hoodie-conf hoodie.deltastreamer.keygen.timebased.timestamp.type=EPOCHMILLISECONDS 

      Hive Sync exception:

      Exception in thread "main" org.apache.hudi.exception.HoodieException: Could not sync using the meta sync class org.apache.hudi.hive.HiveSyncTool
          at org.apache.hudi.sync.common.util.SyncUtilHelpers.runHoodieMetaSync(SyncUtilHelpers.java:58)
          at org.apache.hudi.utilities.deltastreamer.DeltaSync.runMetaSync(DeltaSync.java:719)
          at org.apache.hudi.utilities.deltastreamer.DeltaSync.writeToSink(DeltaSync.java:637)
          at org.apache.hudi.utilities.deltastreamer.DeltaSync.syncOnce(DeltaSync.java:337)
          at org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.lambda$sync$2(HoodieDeltaStreamer.java:204)
          at org.apache.hudi.common.util.Option.ifPresent(Option.java:97)
          at org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.sync(HoodieDeltaStreamer.java:202)
          at org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.main(HoodieDeltaStreamer.java:571)
          at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
          at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
          at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
          at java.lang.reflect.Method.invoke(Method.java:498)
          at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
          at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:955)
          at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)
          at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)
          at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)
          at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1043)
          at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1052)
          at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
      Caused by: org.apache.hudi.exception.HoodieException: Got runtime exception when hive syncing test_table
          at org.apache.hudi.hive.HiveSyncTool.syncHoodieTable(HiveSyncTool.java:145)
          at org.apache.hudi.sync.common.util.SyncUtilHelpers.runHoodieMetaSync(SyncUtilHelpers.java:56)
          ... 19 more
      Caused by: org.apache.hudi.hive.HoodieHiveSyncException: Failed to sync partitions for table test_table
          at org.apache.hudi.hive.HiveSyncTool.syncPartitions(HiveSyncTool.java:341)
          at org.apache.hudi.hive.HiveSyncTool.syncHoodieTable(HiveSyncTool.java:232)
          at org.apache.hudi.hive.HiveSyncTool.doSync(HiveSyncTool.java:154)
          at org.apache.hudi.hive.HiveSyncTool.syncHoodieTable(HiveSyncTool.java:142)
          ... 20 more
      Caused by: org.apache.hudi.hive.HoodieHiveSyncException: default.test_table add partition failed
          at org.apache.hudi.hive.ddl.HMSDDLExecutor.addPartitionsToTable(HMSDDLExecutor.java:217)
          at org.apache.hudi.hive.HoodieHiveSyncClient.addPartitionsToTable(HoodieHiveSyncClient.java:107)
          at org.apache.hudi.hive.HiveSyncTool.syncPartitions(HiveSyncTool.java:324)
          ... 23 more
      Caused by: MetaException(message:Invalid partition key & values; keys [createddate, ], values [2022, 10, 02, ])
          at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$add_partitions_req_result$add_partitions_req_resultStandardScheme.read(ThriftHiveMetastore.java)
          at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$add_partitions_req_result$add_partitions_req_resultStandardScheme.read(ThriftHiveMetastore.java)
          at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$add_partitions_req_result.read(ThriftHiveMetastore.java)
          at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:88)
          at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_add_partitions_req(ThriftHiveMetastore.java:1911)
          at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.add_partitions_req(ThriftHiveMetastore.java:1898)
          at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.add_partitions(HiveMetaStoreClient.java:625)
          at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
          at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
          at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
          at java.lang.reflect.Method.invoke(Method.java:498)
          at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.invoke(RetryingMetaStoreClient.java:173)
          at com.sun.proxy.$Proxy44.add_partitions(Unknown Source)
          at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
          at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
          at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
          at java.lang.reflect.Method.invoke(Method.java:498)
          at org.apache.hadoop.hive.metastore.HiveMetaStoreClient$SynchronizedHandler.invoke(HiveMetaStoreClient.java:2327)
          at com.sun.proxy.$Proxy44.add_partitions(Unknown Source)
          at org.apache.hudi.hive.ddl.HMSDDLExecutor.addPartitionsToTable(HMSDDLExecutor.java:212)
          ... 25 more

      Glue Sync exception:

      Exception in thread "main" org.apache.hudi.exception.HoodieException: Could not sync using the meta sync class org.apache.hudi.aws.sync.AwsGlueCatalogSyncTool
      	at org.apache.hudi.sync.common.util.SyncUtilHelpers.runHoodieMetaSync(SyncUtilHelpers.java:58)
      	at org.apache.hudi.utilities.deltastreamer.DeltaSync.runMetaSync(DeltaSync.java:719)
      	at org.apache.hudi.utilities.deltastreamer.DeltaSync.writeToSink(DeltaSync.java:637)
      	at org.apache.hudi.utilities.deltastreamer.DeltaSync.syncOnce(DeltaSync.java:337)
      	at org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.lambda$sync$2(HoodieDeltaStreamer.java:204)
      	at org.apache.hudi.common.util.Option.ifPresent(Option.java:97)
      	at org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.sync(HoodieDeltaStreamer.java:202)
      	at org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.main(HoodieDeltaStreamer.java:571)
      	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
      	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
      	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      	at java.lang.reflect.Method.invoke(Method.java:498)
      	at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
      	at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:955)
      	at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180)
      	at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)
      	at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)
      	at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1043)
      	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1052)
      	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
      Caused by: org.apache.hudi.exception.HoodieException: Got runtime exception when hive syncing test_table
      	at org.apache.hudi.hive.HiveSyncTool.syncHoodieTable(HiveSyncTool.java:145)
      	at org.apache.hudi.sync.common.util.SyncUtilHelpers.runHoodieMetaSync(SyncUtilHelpers.java:56)
      	... 19 more
      Caused by: org.apache.hudi.hive.HoodieHiveSyncException: Failed to sync partitions for table test_table
      	at org.apache.hudi.hive.HiveSyncTool.syncPartitions(HiveSyncTool.java:341)
      	at org.apache.hudi.hive.HiveSyncTool.syncHoodieTable(HiveSyncTool.java:232)
      	at org.apache.hudi.hive.HiveSyncTool.doSync(HiveSyncTool.java:154)
      	at org.apache.hudi.hive.HiveSyncTool.syncHoodieTable(HiveSyncTool.java:142)
      	... 20 more
      Caused by: org.apache.hudi.aws.sync.HoodieGlueSyncException: Fail to add partitions to default.test_table
      	at org.apache.hudi.aws.sync.AWSGlueCatalogSyncClient.addPartitionsToTable(AWSGlueCatalogSyncClient.java:147)
      	at org.apache.hudi.hive.HiveSyncTool.syncPartitions(HiveSyncTool.java:324)
      	... 23 more
      Caused by: org.apache.hudi.com.amazonaws.services.glue.model.InvalidInputException: The number of partition keys do not match the number of partition values (Service: AWSGlue; Status Code: 400; Error Code: InvalidInputException; Request ID: e8d9adf2-13c4-4589-bbec-c578a827749f; Proxy: null)
      	at org.apache.hudi.com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1819)
      	at org.apache.hudi.com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1403)
      	at org.apache.hudi.com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1372)
      	at org.apache.hudi.com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1145)
      	at org.apache.hudi.com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:802)
      	at org.apache.hudi.com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:770)
      	at org.apache.hudi.com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:744)
      	at org.apache.hudi.com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:704)
      	at org.apache.hudi.com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:686)
      	at org.apache.hudi.com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:550)
      	at org.apache.hudi.com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:530)
      	at org.apache.hudi.com.amazonaws.services.glue.AWSGlueClient.doInvoke(AWSGlueClient.java:10640)
      	at org.apache.hudi.com.amazonaws.services.glue.AWSGlueClient.invoke(AWSGlueClient.java:10607)
      	at org.apache.hudi.com.amazonaws.services.glue.AWSGlueClient.invoke(AWSGlueClient.java:10596)
      	at org.apache.hudi.com.amazonaws.services.glue.AWSGlueClient.executeBatchCreatePartition(AWSGlueClient.java:259)
      	at org.apache.hudi.com.amazonaws.services.glue.AWSGlueClient.batchCreatePartition(AWSGlueClient.java:228)
      	at org.apache.hudi.aws.sync.AWSGlueCatalogSyncClient.addPartitionsToTable(AWSGlueCatalogSyncClient.java:139)
      	... 24 more 

      The exception is thrown because the partition path values for meta sync are not properly extracted.  "hoodie.datasource.hive_sync.partition_extractor_class" determines the partition extractor to use and in such a case, the `MultiPartKeysValueExtractor` is inferred to be used.  The root cause is that, this extractor split the parts by slashes.  If user specifies the output dateformat which contains slashes, that is going to fail the extraction.

      The fix is to introduce a new partition extractor so that we treat the partition as a whole when there is only a single partition column, instead of relying on `MultiPartKeysValueExtractor`.

       

       

      Attachments

        Issue Links

          Activity

            People

              guoyihua Ethan Guo
              guoyihua Ethan Guo
              Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: