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-23381][state] Back-pressure on reaching state change to upload…
… limit
- Loading branch information
1 parent
7bbe5f9
commit f925481
Showing
11 changed files
with
318 additions
and
65 deletions.
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
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
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
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
61 changes: 61 additions & 0 deletions
61
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/UploadThrottle.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,61 @@ | ||
/* | ||
* 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.changelog.fs; | ||
|
||
import javax.annotation.concurrent.NotThreadSafe; | ||
|
||
import static org.apache.flink.util.Preconditions.checkState; | ||
|
||
/** Helper class to throttle upload requests when the in-flight data size limit is exceeded. */ | ||
@NotThreadSafe | ||
class UploadThrottle { | ||
|
||
private final long maxBytesInFlight; | ||
|
||
private long inFlightBytesCounter = 0; | ||
|
||
UploadThrottle(long maxBytesInFlight) { | ||
this.maxBytesInFlight = maxBytesInFlight; | ||
} | ||
|
||
/** | ||
* Seize <b>bytes</b> capacity. It is the caller responsibility to ensure at least some capacity | ||
* {@link #hasCapacity() is available}. <strong>After</strong> this call, the caller is allowed | ||
* to actually use the seized capacity. When the capacity is not needed anymore, the caller is | ||
* required to {@link #releaseCapacity(long) release} it. Called by the Task thread. | ||
* | ||
* @throws IllegalStateException if capacity is unavailable. | ||
*/ | ||
public void seizeCapacity(long bytes) throws IllegalStateException { | ||
checkState(hasCapacity()); | ||
inFlightBytesCounter += bytes; | ||
} | ||
|
||
/** | ||
* Release previously {@link #seizeCapacity(long) seized} capacity. Called by {@link | ||
* BatchingStateChangeUploader} (IO thread). | ||
*/ | ||
public void releaseCapacity(long bytes) { | ||
inFlightBytesCounter -= bytes; | ||
} | ||
|
||
/** Test whether some capacity is available. */ | ||
public boolean hasCapacity() { | ||
return inFlightBytesCounter < maxBytesInFlight; | ||
} | ||
} |
Oops, something went wrong.