Skip to content

Conversation

@wuzhenhua01
Copy link
Contributor

@wuzhenhua01 wuzhenhua01 commented Nov 30, 2023

Change Logs

fix https://issues.apache.org/jira/browse/HUDI-7163

reference https://issues.apache.org/jira/browse/HUDI-4340

Impact

flink compact

Risk level (write none, low medium or high below)

low medium

Documentation Update

None

Contributor's checklist

  • Read through contributor's guide
  • Change Logs and Impact were stated clearly
  • Adequate tests were added if applicable
  • CI passed

@danny0405
Copy link
Contributor

Can you elaborate a little more about the operation procedure that can reproduce this exception? That would help a lot for understanding the fix.

@bvaradar
Copy link
Contributor

Can you elaborate a little more about the operation procedure that can reproduce this exception? That would help a lot for understanding the fix.

@danny0405 : Looks like this is a porting of #6000 to Flink integration

@danny0405
Copy link
Contributor

Can you elaborate a little more about the operation procedure that can reproduce this exception? That would help a lot for understanding the fix.

@danny0405 : Looks like this is a porting of #6000 to Flink integration

Yeah, It's like a following up of #6000 to fix the timestamp precision of legacy tables.

@wuzhenhua01
Copy link
Contributor Author

wuzhenhua01 commented Dec 4, 2023

Can you elaborate a little more about the operation procedure that can reproduce this exception? That would help a lot for understanding the fix.

Hudi Version 0.14.0

step 1, i use bootstrapping convert parquet table to hudi, like this

spark-submit \
  --class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer \
  /cluster/hudi/hudi-utilities-bundle_2.11-0.12.2.jar \
  --target-base-path /tmp/t1 --table-type MERGE_ON_READ \
  --target-table t1 \
  --run-bootstrap \
  --bootstrap-overwrite \
  --hoodie-conf hoodie.bootstrap.base.path=hdfs://hacluster/user/hive/warehouse/t1 \
  --hoodie-conf hoodie.datasource.write.recordkey.field=id \
  --hoodie-conf hoodie.datasource.write.keygenerator.type=NON_PARTITION \
  --hoodie-conf hoodie.bootstrap.parallelism=2 \

step 2, use flink datastrem writer to this hudi table and turn on compact, like this

    val hudiProps = DFSPropertiesConfiguration.getGlobalProps
    val hudiConf = Configuration.fromMap(hudiProps.asInstanceOf[JMap[String, String]])
    conf.addAll(hudiConf)

    OptionsInference.setupSinkTasks(conf, env.getParallelism)

    val rowType = RowType.of(false, Array(new TimestampType, new BigIntType, new VarCharType, new VarCharType), Array("_origin_op_ts", "id", "content", "date"))
    conf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA, AvroSchemaConverter.convertToSchema(rowType).toString)

    conf.setString(FlinkOptions.PATH, "/tmp/t1")
    conf.setString(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ)
    conf.setString(FlinkOptions.TABLE_NAME, "t1")
    conf.setString(FlinkOptions.PRECOMBINE_FIELD, "_origin_op_ts")
    conf.setString(FlinkOptions.RECORD_KEY_FIELD, "id")

    conf.setBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING, true)
    conf.setString(FlinkOptions.KEYGEN_TYPE, KeyGeneratorType.NON_PARTITION.name)

    conf.setBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED, false)

    val hoodieRecordDataStream = Pipelines.bootstrap(conf, rowType, rowDataDataStream.javaStream)
    val pipeline = Pipelines.hoodieStreamWrite(conf, hoodieRecordDataStream)

    conf.setString(FlinkOptions.COMPACTION_TRIGGER_STRATEGY, FlinkOptions.TIME_ELAPSED)
    conf.setInteger(FlinkOptions.COMPACTION_DELTA_SECONDS, TimeUnit.SECONDS.toSeconds(120).toInt)

    Pipelines.compact(conf, pipeline)
    env.execute

exception:

2023-12-14 00:19:42
org.apache.flink.util.FlinkException: Global failure triggered by OperatorCoordinator for 'stream_write: int_hudi_crm_cus_inst_offer_inst_fee_info_realtime' (operator 1426344399aaa371a412d3586035d87e).
	at org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder$LazyInitializedCoordinatorContext.failJob(OperatorCoordinatorHolder.java:545)
	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.lambda$start$0(StreamWriteOperatorCoordinator.java:196)
	at org.apache.hudi.sink.utils.NonThrownExecutor.handleException(NonThrownExecutor.java:142)
	at org.apache.hudi.sink.utils.NonThrownExecutor.lambda$wrapAction$0(NonThrownExecutor.java:133)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.hudi.exception.HoodieException: Executor executes action [commits the instant 20231214001926316] error
	... 6 more
Caused by: org.apache.hudi.exception.HoodieException: Get median instant time with interval [00000000000001, 20231214001926316] error
	at org.apache.hudi.util.StreamerUtil.medianInstantTime(StreamerUtil.java:341)
	at org.apache.hudi.util.CompactionUtil.getCompactionInstantTime(CompactionUtil.java:87)
	at org.apache.hudi.util.CompactionUtil.scheduleCompaction(CompactionUtil.java:68)
	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.scheduleTableServices(StreamWriteOperatorCoordinator.java:460)
	at org.apache.hudi.sink.StreamWriteOperatorCoordinator.lambda$notifyCheckpointComplete$2(StreamWriteOperatorCoordinator.java:260)
	at org.apache.hudi.sink.utils.NonThrownExecutor.lambda$wrapAction$0(NonThrownExecutor.java:130)
	... 3 more
Caused by: java.text.ParseException: Text '00000000000001999' could not be parsed: Invalid value for YearOfEra (valid values 1 - 999999999/1000000000): 0
	at org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator.parseDateFromInstantTime(HoodieInstantTimeGenerator.java:99)
	at org.apache.hudi.common.table.timeline.HoodieActiveTimeline.parseDateFromInstantTime(HoodieActiveTimeline.java:98)
	at org.apache.hudi.util.StreamerUtil.medianInstantTime(StreamerUtil.java:330)
	... 8 more

@wuzhenhua01
Copy link
Contributor Author

After the first step, it will generate the first deltacommit 00000000000001.deltacommit, and when generate next compact, it will throw exception.

image

throw new HoodieException("Get median instant time with interval [" + lowVal + ", " + highVal + "] error", e);
long high = HoodieActiveTimeline.parseDateFromInstantTimeSafely(highVal)
.orElseThrow(() -> new HoodieException("Get instant time diff with interval [" + highVal + "] error")).getTime();
long low = HoodieActiveTimeline.parseDateFromInstantTimeSafely(lowVal)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does the exception only happens for table upgrade?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it is mainly fordefensive programming

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you encounter exceptions in production?

Copy link
Contributor Author

@wuzhenhua01 wuzhenhua01 Dec 14, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

i mean the method orElseThrow is fordefensive programming. sure, we can repeat the provided operation procedure to reproduce this exception

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Okay, if we are not enocunting problems, would de-prioritize the PR a little bit.

@github-actions github-actions bot added the size:S PR with lines of changes in (10, 100] label Feb 26, 2024
@yihua
Copy link
Contributor

yihua commented Mar 7, 2024

@danny0405 do you think we should land this or the changes are no longer needed?

@danny0405
Copy link
Contributor

We can land it.

@hudi-bot
Copy link
Collaborator

hudi-bot commented Mar 9, 2024

CI report:

Bot commands @hudi-bot supports the following commands:
  • @hudi-bot run azure re-run the last Azure build

@yihua yihua changed the title [HUDI-7163] fix not parsable text DateTimeParseException when compact [HUDI-7163] Fix not parsable text DateTimeParseException when compact Mar 9, 2024
@yihua yihua merged commit e666415 into apache:master Mar 9, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

release-0.14.1 size:S PR with lines of changes in (10, 100]

Projects

Status: ✅ Done

Development

Successfully merging this pull request may close these issues.

6 participants