forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-25575][streaming] Add data structure to collect CommittableMes…
…sages. Using operators will be able to bulk commit committables and retry failed. Co-authored-by: Arvid Heise <[email protected]>
- Loading branch information
Showing
15 changed files
with
1,545 additions
and
1 deletion.
There are no files selected for viewing
36 changes: 36 additions & 0 deletions
36
...che/flink/streaming/runtime/operators/sink/committables/CheckpointCommittableManager.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,36 @@ | ||
/* | ||
* 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.flink.streaming.runtime.operators.sink.committables; | ||
|
||
import org.apache.flink.annotation.Internal; | ||
|
||
/** | ||
* This interface adds checkpoint meta information to the committable. | ||
* | ||
* @param <CommT> type of the committable | ||
*/ | ||
@Internal | ||
public interface CheckpointCommittableManager<CommT> extends CommittableManager<CommT> { | ||
/** | ||
* Returns the checkpoint id in which the committable was created. | ||
* | ||
* @return checkpoint id | ||
*/ | ||
long getCheckpointId(); | ||
} |
158 changes: 158 additions & 0 deletions
158
...flink/streaming/runtime/operators/sink/committables/CheckpointCommittableManagerImpl.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,158 @@ | ||
/* | ||
* 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.flink.streaming.runtime.operators.sink.committables; | ||
|
||
import org.apache.flink.api.connector.sink2.Committer; | ||
import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; | ||
import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; | ||
|
||
import java.io.IOException; | ||
import java.util.ArrayList; | ||
import java.util.Collection; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.stream.Collectors; | ||
|
||
import static org.apache.flink.util.Preconditions.checkArgument; | ||
import static org.apache.flink.util.Preconditions.checkNotNull; | ||
|
||
class CheckpointCommittableManagerImpl<CommT> implements CheckpointCommittableManager<CommT> { | ||
/** Mapping of subtask id to {@link SubtaskCommittableManager}. */ | ||
private final Map<Integer, SubtaskCommittableManager<CommT>> subtasksCommittableManagers; | ||
|
||
private final long checkpointId; | ||
private final int subtaskId; | ||
private final int numberOfSubtasks; | ||
|
||
CheckpointCommittableManagerImpl(int subtaskId, int numberOfSubtasks, long checkpointId) { | ||
this.subtaskId = subtaskId; | ||
this.numberOfSubtasks = numberOfSubtasks; | ||
this.checkpointId = checkpointId; | ||
this.subtasksCommittableManagers = new HashMap<>(); | ||
} | ||
|
||
CheckpointCommittableManagerImpl( | ||
Map<Integer, SubtaskCommittableManager<CommT>> subtasksCommittableManagers, | ||
long checkpointId) { | ||
this.subtasksCommittableManagers = checkNotNull(subtasksCommittableManagers); | ||
this.subtaskId = 0; | ||
this.numberOfSubtasks = 1; | ||
this.checkpointId = checkpointId; | ||
} | ||
|
||
@Override | ||
public long getCheckpointId() { | ||
return checkpointId; | ||
} | ||
|
||
Collection<SubtaskCommittableManager<CommT>> getSubtaskCommittableManagers() { | ||
return subtasksCommittableManagers.values(); | ||
} | ||
|
||
void upsertSummary(CommittableSummary<CommT> summary) { | ||
SubtaskCommittableManager<CommT> existing = | ||
subtasksCommittableManagers.putIfAbsent( | ||
summary.getSubtaskId(), | ||
new SubtaskCommittableManager<>( | ||
summary.getNumberOfCommittables(), | ||
subtaskId, | ||
summary.getCheckpointId().isPresent() | ||
? summary.getCheckpointId().getAsLong() | ||
: null)); | ||
if (existing != null) { | ||
throw new UnsupportedOperationException( | ||
"Currently it is not supported to update the CommittableSummary for a checkpoint coming from the same subtask. Please check the status of FLINK-25920"); | ||
} | ||
} | ||
|
||
void addCommittable(CommittableWithLineage<CommT> committable) { | ||
getSubtaskCommittableManager(committable.getSubtaskId()).add(committable); | ||
} | ||
|
||
SubtaskCommittableManager<CommT> getSubtaskCommittableManager(int subtaskId) { | ||
SubtaskCommittableManager<CommT> committables = | ||
this.subtasksCommittableManagers.get(subtaskId); | ||
return checkNotNull(committables, "Unknown subtask for %s", subtaskId); | ||
} | ||
|
||
@Override | ||
public CommittableSummary<CommT> getSummary() { | ||
return new CommittableSummary<>( | ||
subtaskId, | ||
numberOfSubtasks, | ||
checkpointId, | ||
subtasksCommittableManagers.values().stream() | ||
.mapToInt(SubtaskCommittableManager::getNumCommittables) | ||
.sum(), | ||
subtasksCommittableManagers.values().stream() | ||
.mapToInt(SubtaskCommittableManager::getNumPending) | ||
.sum(), | ||
subtasksCommittableManagers.values().stream() | ||
.mapToInt(SubtaskCommittableManager::getNumFailed) | ||
.sum()); | ||
} | ||
|
||
boolean isFinished() { | ||
return subtasksCommittableManagers.values().stream() | ||
.allMatch(SubtaskCommittableManager::isFinished); | ||
} | ||
|
||
@Override | ||
public Collection<CommittableWithLineage<CommT>> commit( | ||
boolean fullyReceived, Committer<CommT> committer) | ||
throws IOException, InterruptedException { | ||
Collection<CommitRequestImpl<CommT>> requests = getPendingRequests(fullyReceived); | ||
requests.forEach(CommitRequestImpl::setSelected); | ||
committer.commit(new ArrayList<>(requests)); | ||
requests.forEach(CommitRequestImpl::setCommittedIfNoError); | ||
return drainFinished(); | ||
} | ||
|
||
Collection<CommitRequestImpl<CommT>> getPendingRequests(boolean fullyReceived) { | ||
return subtasksCommittableManagers.values().stream() | ||
.filter(subtask -> !fullyReceived || subtask.hasReceivedAll()) | ||
.flatMap(SubtaskCommittableManager::getPendingRequests) | ||
.collect(Collectors.toList()); | ||
} | ||
|
||
Collection<CommittableWithLineage<CommT>> drainFinished() { | ||
return subtasksCommittableManagers.values().stream() | ||
.flatMap(subtask -> subtask.drainCommitted().stream()) | ||
.collect(Collectors.toList()); | ||
} | ||
|
||
CheckpointCommittableManagerImpl<CommT> merge(CheckpointCommittableManagerImpl<CommT> other) { | ||
checkArgument(other.checkpointId == checkpointId); | ||
for (Map.Entry<Integer, SubtaskCommittableManager<CommT>> subtaskEntry : | ||
other.subtasksCommittableManagers.entrySet()) { | ||
subtasksCommittableManagers.merge( | ||
subtaskEntry.getKey(), | ||
subtaskEntry.getValue(), | ||
SubtaskCommittableManager::merge); | ||
} | ||
return this; | ||
} | ||
|
||
CheckpointCommittableManagerImpl<CommT> copy() { | ||
return new CheckpointCommittableManagerImpl<CommT>( | ||
subtasksCommittableManagers.entrySet().stream() | ||
.collect(Collectors.toMap(Map.Entry::getKey, (e) -> e.getValue().copy())), | ||
checkpointId); | ||
} | ||
} |
108 changes: 108 additions & 0 deletions
108
...ava/org/apache/flink/streaming/runtime/operators/sink/committables/CommitRequestImpl.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,108 @@ | ||
/* | ||
* 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.flink.streaming.runtime.operators.sink.committables; | ||
|
||
import org.apache.flink.annotation.Internal; | ||
import org.apache.flink.api.connector.sink2.Committer; | ||
|
||
/** | ||
* Internal implementation to commit a specific committable and handle the response. | ||
* | ||
* @param <CommT> type of committable | ||
*/ | ||
@Internal | ||
public class CommitRequestImpl<CommT> implements Committer.CommitRequest<CommT> { | ||
|
||
private CommT committable; | ||
private int numRetries; | ||
private CommitRequestState state; | ||
|
||
protected CommitRequestImpl(CommT committable) { | ||
this.committable = committable; | ||
state = CommitRequestState.RECEIVED; | ||
} | ||
|
||
protected CommitRequestImpl(CommT committable, int numRetries, CommitRequestState state) { | ||
this.committable = committable; | ||
this.numRetries = numRetries; | ||
this.state = state; | ||
} | ||
|
||
boolean isFinished() { | ||
return state.isFinalState(); | ||
} | ||
|
||
CommitRequestState getState() { | ||
return state; | ||
} | ||
|
||
@Override | ||
public CommT getCommittable() { | ||
return committable; | ||
} | ||
|
||
@Override | ||
public int getNumberOfRetries() { | ||
return numRetries; | ||
} | ||
|
||
@Override | ||
public void signalFailedWithKnownReason(Throwable t) { | ||
state = CommitRequestState.FAILED; | ||
// TODO: FLINK-25857 add metric later | ||
// let the user configure a strategy for failing and apply it here | ||
} | ||
|
||
@Override | ||
public void signalFailedWithUnknownReason(Throwable t) { | ||
state = CommitRequestState.FAILED; | ||
// TODO: FLINK-25857 add metric later | ||
// let the user configure a strategy for failing and apply it here | ||
throw new IllegalStateException("Failed to commit " + committable, t); | ||
} | ||
|
||
@Override | ||
public void retryLater() { | ||
state = CommitRequestState.RETRY; | ||
numRetries++; | ||
// TODO: FLINK-25857 add metric later | ||
} | ||
|
||
@Override | ||
public void updateAndRetryLater(CommT committable) { | ||
this.committable = committable; | ||
retryLater(); | ||
} | ||
|
||
@Override | ||
public void signalAlreadyCommitted() { | ||
// TODO: FLINK-25857 add metric later | ||
state = CommitRequestState.COMMITTED; | ||
} | ||
|
||
void setSelected() { | ||
state = CommitRequestState.RECEIVED; | ||
} | ||
|
||
void setCommittedIfNoError() { | ||
if (state == CommitRequestState.RECEIVED) { | ||
state = CommitRequestState.COMMITTED; | ||
} | ||
} | ||
} |
40 changes: 40 additions & 0 deletions
40
...va/org/apache/flink/streaming/runtime/operators/sink/committables/CommitRequestState.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,40 @@ | ||
/* | ||
* 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.flink.streaming.runtime.operators.sink.committables; | ||
|
||
import org.apache.flink.annotation.Internal; | ||
|
||
/** Internal state of a committable. */ | ||
@Internal | ||
public enum CommitRequestState { | ||
RECEIVED(false), | ||
RETRY(false), | ||
FAILED(true), | ||
COMMITTED(true); | ||
|
||
final boolean finalState; | ||
|
||
CommitRequestState(boolean finalState) { | ||
this.finalState = finalState; | ||
} | ||
|
||
public boolean isFinalState() { | ||
return finalState; | ||
} | ||
} |
Oops, something went wrong.