Skip to content

Conversation

@ThinkerLei
Copy link
Contributor

@ThinkerLei ThinkerLei commented Jan 5, 2023

Change Logs

Fix concurrency conflict when asyncCompaction is enabled

Impact

When asyncCompaction is enabled , CompactTaskMainThread and compactionThread will cause concurrency conflict when send watermark and compactionCommitEvent

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

high

Documentation Update

Contributor's checklist

if (asyncCompaction) {
synchronized (mutex) {
super.processLatencyMarker(latencyMarker);
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we just ignore the watermark and latency marker totally ? The compaction operators belongs to the sink ppeline, there is no need to propagate them to downstream.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

In my opinion, it will be more user-friendly to display through flink ui after delivery, and some specific functions may need to deliver this information. So i do not ignore the watermark and latency marker totally.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

now, i have ignored the watermark and latency marker totally. @danny0405

@ThinkerLei
Copy link
Contributor Author

@hudi-bot re-run the last Azure build

package org.apache.hudi.sink.compact;

import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
import org.apache.hudi.client.HoodieFlinkWriteClient;
Copy link
Contributor

Choose a reason for hiding this comment

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

Fix the import sequence.

@Override
public void processWatermark(Watermark mark) {
public void processWatermark(Watermark mark) throws Exception {
// no need to propagate the watermark
Copy link
Contributor

Choose a reason for hiding this comment

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

Unnecessary change.

@Override
public void processLatencyMarker(LatencyMarker latencyMarker)
throws Exception {
// no need to propagate the latencyMarker
Copy link
Contributor

Choose a reason for hiding this comment

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

No need to throw exception.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

fixed

Copy link
Contributor

Choose a reason for hiding this comment

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

ClusteringOperator should also be fixed.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ClusteringOperator fixed in HUDI-5515

() -> doCompaction(instantTime, compactionOperation, collector, reloadWriteConfig()),
(errMsg, t) -> collector.collect(new CompactionCommitEvent(instantTime, compactionOperation.getFileId(), taskID)),
(errMsg, t) -> {
collector.collect(new CompactionCommitEvent(instantTime,
Copy link
Contributor

Choose a reason for hiding this comment

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

Unnecessary change.

instantTime, maxInstantTime,
writeClient.getHoodieTable().getTaskContextSupplier());
collector.collect(new CompactionCommitEvent(instantTime, compactionOperation.getFileId(), writeStatuses, taskID));
collector.collect(new CompactionCommitEvent(instantTime, compactionOperation.getFileId(),
Copy link
Contributor

Choose a reason for hiding this comment

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

Unnecessary change.

@ThinkerLei
Copy link
Contributor Author

@hudi-bot run azure

@hudi-bot
Copy link
Collaborator

hudi-bot commented Jan 8, 2023

CI report:

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

Copy link
Contributor

@danny0405 danny0405 left a comment

Choose a reason for hiding this comment

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

+1, nice catch ~ We can land this once ClusteringOperator is also fixed.

@danny0405 danny0405 added engine:flink Flink integration area:table-service Table services priority:critical Production degraded; pipelines stalled labels Jan 9, 2023
@danny0405 danny0405 merged commit dc0f880 into apache:master Jan 9, 2023
fengjian428 pushed a commit to fengjian428/hudi that referenced this pull request Jan 31, 2023
nsivabalan pushed a commit to nsivabalan/hudi that referenced this pull request Mar 22, 2023
fengjian428 pushed a commit to fengjian428/hudi that referenced this pull request Apr 5, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

area:table-service Table services engine:flink Flink integration priority:critical Production degraded; pipelines stalled

Projects

Archived in project

Development

Successfully merging this pull request may close these issues.

3 participants