-
Notifications
You must be signed in to change notification settings - Fork 2.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[HUDI-8704] Instant time to completion time state upgrade migration for Flink StreamReadMonitoringFunction #12539
Open
voonhous
wants to merge
9
commits into
apache:master
Choose a base branch
from
voonhous:HUDI-8704
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
Show all changes
9 commits
Select commit
Hold shift + click to select a range
450e34a
[HUDI-8704] Added interface to handle state upgrades from 0.x to 1.x
voonhous 3f06f2c
[HUDI-8704] Added state upgrade implementation to StreamReadMonitorin…
voonhous 55c488d
Fix checkstyle
voonhous 4ed759a
Implemented TODOs for upgrade logic
voonhous 23a225e
Shift version details/pinning to concrete StateUpgrader implementation
voonhous dfe893d
Update attribute names to be for easy comprehension
voonhous c54ea81
Added UNKNOWN version and changed exception for testing
voonhous 93be0ed
Added tests for StreamReadMonitoringStateUpgrader
voonhous c8b55ad
Fix checkstyle
voonhous File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
51 changes: 51 additions & 0 deletions
51
...datasource/hudi-flink/src/main/java/org/apache/hudi/state/upgrade/StateUpgradeHelper.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,51 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.hudi.state.upgrade; | ||
|
||
import org.apache.flink.api.common.state.ListState; | ||
|
||
import java.util.List; | ||
import java.util.stream.Collectors; | ||
import java.util.stream.StreamSupport; | ||
|
||
public class StateUpgradeHelper<T> { | ||
|
||
private final ListState<T> state; | ||
private final StateUpgrader<T> upgrader; | ||
private final StateVersion targetVersion; | ||
|
||
public StateUpgradeHelper(ListState<T> state, StateUpgrader<T> upgrader, StateVersion targetVersion) { | ||
this.state = state; | ||
this.upgrader = upgrader; | ||
this.targetVersion = targetVersion; | ||
} | ||
|
||
public void upgradeState() throws Exception { | ||
List<T> currentState = StreamSupport | ||
.stream(state.get().spliterator(), false) | ||
.collect(Collectors.toList()); | ||
|
||
StateVersion detectedVersion = upgrader.detectVersion(currentState); | ||
if (upgrader.canUpgrade(detectedVersion, targetVersion)) { | ||
List<T> upgradedState = upgrader.upgrade(currentState, detectedVersion, targetVersion); | ||
state.clear(); | ||
state.addAll(upgradedState); | ||
} | ||
} | ||
} |
29 changes: 29 additions & 0 deletions
29
...link-datasource/hudi-flink/src/main/java/org/apache/hudi/state/upgrade/StateUpgrader.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,29 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.hudi.state.upgrade; | ||
|
||
import java.util.List; | ||
|
||
public interface StateUpgrader<T> { | ||
List<T> upgrade(List<T> oldState, StateVersion fromVersion, StateVersion toVersion); | ||
|
||
boolean canUpgrade(StateVersion fromVersion, StateVersion toVersion); | ||
|
||
StateVersion detectVersion(List<T> oldState); | ||
} |
35 changes: 35 additions & 0 deletions
35
...flink-datasource/hudi-flink/src/main/java/org/apache/hudi/state/upgrade/StateVersion.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,35 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.hudi.state.upgrade; | ||
|
||
public enum StateVersion { | ||
V0(0), | ||
V1(1), | ||
UNKNOWN(-1); | ||
|
||
private final int version; | ||
|
||
StateVersion(int version) { | ||
this.version = version; | ||
} | ||
|
||
public int getValue() { | ||
return version; | ||
} | ||
} |
111 changes: 111 additions & 0 deletions
111
...src/main/java/org/apache/hudi/state/upgrade/source/StreamReadMonitoringStateUpgrader.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,111 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.hudi.state.upgrade.source; | ||
|
||
import org.apache.hudi.common.table.HoodieTableMetaClient; | ||
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; | ||
import org.apache.hudi.common.table.timeline.HoodieTimeline; | ||
import org.apache.hudi.common.util.StringUtils; | ||
import org.apache.hudi.common.util.ValidationUtils; | ||
import org.apache.hudi.exception.HoodieException; | ||
import org.apache.hudi.exception.HoodieUpgradeDowngradeException; | ||
import org.apache.hudi.state.upgrade.StateUpgrader; | ||
import org.apache.hudi.state.upgrade.StateVersion; | ||
|
||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import java.util.Arrays; | ||
import java.util.List; | ||
|
||
public class StreamReadMonitoringStateUpgrader implements StateUpgrader<String> { | ||
|
||
private static final Logger LOG = LoggerFactory.getLogger(StreamReadMonitoringStateUpgrader.class); | ||
|
||
private final HoodieTableMetaClient metaClient; | ||
private final String issuedInstant; | ||
|
||
public StreamReadMonitoringStateUpgrader(HoodieTableMetaClient metaClient, String issuedInstant) { | ||
this.metaClient = metaClient; | ||
this.issuedInstant = issuedInstant; | ||
} | ||
|
||
@Override | ||
public List<String> upgrade(List<String> oldState, StateVersion fromVersion, StateVersion toVersion) { | ||
switch (fromVersion) { | ||
case V0: | ||
if (toVersion == StateVersion.V1) { | ||
return upgradeV0ToV1(oldState); | ||
} | ||
throw new HoodieUpgradeDowngradeException("Unsupported version upgrade path :" + fromVersion + " -> " + toVersion); | ||
case V1: | ||
// Do nothing | ||
return oldState; | ||
default: | ||
throw new HoodieUpgradeDowngradeException("Unsupported version upgrade path :" + fromVersion + " -> " + toVersion); | ||
} | ||
} | ||
|
||
@Override | ||
public boolean canUpgrade(StateVersion fromVersion, StateVersion toVersion) { | ||
return fromVersion.getValue() < toVersion.getValue(); | ||
} | ||
|
||
@Override | ||
public StateVersion detectVersion(List<String> state) { | ||
switch (state.size()) { | ||
case 1: | ||
return StateVersion.V0; | ||
case 2: | ||
return StateVersion.V1; | ||
default: | ||
throw new IllegalStateException("Unknown state size when detecting version, size: " + state.size()); | ||
} | ||
} | ||
|
||
private List<String> upgradeV0ToV1(List<String> oldState) { | ||
ValidationUtils.checkState(oldState.size() == 1, "Retrieved state must have a size of 1"); | ||
|
||
// this is the case where we have both legacy and new state. | ||
// the two should be mutually exclusive for the operator, thus we throw the exception. | ||
ValidationUtils.checkState(this.issuedInstant != null, | ||
"The " + getClass().getSimpleName() + " has already restored from a previous Flink version."); | ||
|
||
String issuedInstant = oldState.get(0); | ||
HoodieTimeline filteredTimeline; | ||
boolean isReadArchivedTimeline = false; | ||
if (metaClient.getActiveTimeline().isBeforeTimelineStarts(issuedInstant)) { | ||
isReadArchivedTimeline = true; | ||
// if issuedInstant (requestedTime) is in archive timeline, scan archive timeline to find its completion time | ||
LOG.warn("The reader's restored instant is in the archive timeline, will scan the archive timeline for issuedOffset (completionTime) using requestTime: {}", issuedInstant); | ||
HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline(StringUtils.EMPTY_STRING, false); | ||
filteredTimeline = archivedTimeline.findInstantsAfterOrEquals(issuedInstant, 3); | ||
} else { | ||
filteredTimeline = metaClient.getActiveTimeline().findInstantsAfterOrEquals(issuedInstant, 3); | ||
} | ||
|
||
if (filteredTimeline.firstInstant().isEmpty()) { | ||
LOG.error("Unable to find instant {} in [isArchived: {}] timeline: {}", issuedInstant, isReadArchivedTimeline, filteredTimeline.getInstants()); | ||
throw new HoodieException("Unable to find completionTime in timeline for instant: " + issuedInstant); | ||
} | ||
String issuedOffset = filteredTimeline.firstInstant().get().getCompletionTime(); | ||
|
||
return Arrays.asList(issuedInstant, issuedOffset); | ||
} | ||
} |
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I expected this be a simple change, check the
issuedOffset
against the timeline for the first time of read, if theissuedOffset
is an instant time, then just switch to it's completion time or modification time.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Apologies for the misunderstanding, i do not quite understand what you mean, I was under the impression that this would be a backward compatibility improvement when migrating from a Hudi 0.x table to a Hudi 1.0 table.
From what i understand:
Hence, i was under the impression that we needed a way to add a
issuedOffset
when upgrading from0.x
, whereissuedOffset
will be null in the state backend during first read after upgrade.