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

flink pending Compaction error

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 0.10.0
    • None
    • compaction, flink, table-service
    • None

    Description

      this is my create table sql

      create table hudi.ods_user_behavior_logic(
      uuid_did string,
      content string,
      client_ip string,
      userid bigint,
      visit_time_ts bigint,
      `event_time` TIMESTAMP(3) COMMENT 'ETL创建时间',
      `etl_update_time` TIMESTAMP(3) COMMENT 'ETL更新时间',
      `visit_date` string
      )PARTITIONED BY (`visit_date`)
      with (
      'connector' = 'hudi'
      ,'is_generic' = 'true'
      ,'path' = 'hdfs:///tmp/flink_hudi/ods_user_behavior'
      ,'hoodie.datasource.write.recordkey.field' = 'uuid_did'
      ,'hoodie.datasource.write.partitionpath.field' = 'visit_date'
      ,'write.precombine.field' = 'etl_update_time'
      ,'write.tasks' = '10'
      ,'table.type' = 'MERGE_ON_READ'
      ,'compaction.delta_commits' = '5'
      ,'compaction.async.enabled' = 'true'
      ,'compaction.trigger.strategy' = 'num_commits'
      ,'compaction.delta_seconds' = '90'
      ,'compaction.tasks' = '10'
      ,'hive_sync.enable' = 'true'
      ,'hive_sync.db' = 'hudi'
      ,'hive_sync.table' = 'ods_user_behavior'
      ,'hive_sync.username' = 'data'
      ,'hive_sync.file_format' = 'PARQUET'
      ,'hive_sync.support_timestamp' = 'true'
      ,'hive_sync.use_jdbc' = 'true'
      ,'hive_sync.jdbc_url' = 'jdbc:hive2://xxx:7001'
      ,'hive_sync.metastore.uris' = 'thrift://xxx:7001,thrift://10.165.37.57:7004'
      ,'hoodie.datasource.hive_sync.partition_extractor_class' = 'hudi.DatePartitionExtractor'
      ,'hoodie.datasource.hive_style_partition' = 'true'
      ,'hive_sync.partition_fields' = 'visit_date'
      ,'hive_sync.auto_create_database' = 'true'
      ,'hive_sync.skip_ro_suffix' = 'false'
      ,'hive_sync.support_timestamp' = 'false'
      ,'read.tasks' = '10'
      ,'read.streaming.enabled' = 'true'
      ,'hoodie.datasource.query.type' = 'snapshot'
      ,'read.streaming.start-commit' = '20210817112200'
      ,'read.streaming.check-interval' = '30'
      ,'hoodie.datasource.merge.type' = 'payload_combine'
      ,'read.utc-timezone' = 'false'
      );

      when i use flink consum kafka and insert into table ods_user_behavior_logic

      i got error 

      Got runtime exception servicing request basepath=obs%3A%2F%2Fmogubigdata%2Fuser%2Fhaifeng%2Fflink_hudi%2Fods_user_behavior&lastinstantts=20211014181917&timelinehash=30e179400bc141747be43e8aa48d25111a3d099e83757631bd77e24cdbf87332
      java.lang.IllegalArgumentException: Last known instant from client was 20211014181917 but server has the following timeline [[20211014174158__clean__COMPLETED] ... [20211014181917__deltacommit__COMPLETED]]
      at org.apache.hudi.common.util.ValidationUtils.checkArgument(ValidationUtils.java:40) ~[flink-batch-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
      at org.apache.hudi.timeline.service.RequestHandler$ViewHandler.handle(RequestHandler.java:510) ~[flink-batch-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
      at io.javalin.security.SecurityUtil.noopAccessManager(SecurityUtil.kt:22) ~[flink-batch-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
      at io.javalin.Javalin.lambda$addHandler$0(Javalin.java:606) ~[flink-batch-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
      at io.javalin.core.JavalinServlet$service$2$1.invoke(JavalinServlet.kt:46) [flink-batch-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
      at io.javalin.core.JavalinServlet$service$2$1.invoke(JavalinServlet.kt:17) [flink-batch-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
      at io.javalin.core.JavalinServlet$service$1.invoke(JavalinServlet.kt:143) [flink-batch-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
      at io.javalin.core.JavalinServlet$service$2.invoke(JavalinServlet.kt:41) [flink-batch-1.0-SNAPSHOT-jar-with-dependencies.jar:?]
      at io.javalin.core.JavalinServlet.service(JavalinServlet.kt:107) [flink-batch-1.0-SNAPSHOT-jar-with-dependencies.jar:?]

       

      How can I solve this error,thanks

      Attachments

        Activity

          People

            Unassigned Unassigned
            caibaojiang mo.wu
            Votes:
            0 Vote for this issue
            Watchers:
            1 Start watching this issue

            Dates

              Created:
              Updated: