-
Notifications
You must be signed in to change notification settings - Fork 226
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[log] Arrow Compression support estimate compression ratio in client
- Loading branch information
1 parent
2658ea0
commit 9ecd4ba
Showing
7 changed files
with
324 additions
and
6 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
106 changes: 106 additions & 0 deletions
106
fluss-common/src/main/java/com/alibaba/fluss/compression/ArrowCompressionRatioEstimator.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,106 @@ | ||
/* | ||
* Copyright (c) 2024 Alibaba Group Holding Ltd. | ||
* | ||
* Licensed 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 com.alibaba.fluss.compression; | ||
|
||
import com.alibaba.fluss.annotation.Internal; | ||
|
||
import javax.annotation.concurrent.ThreadSafe; | ||
|
||
import java.util.Map; | ||
import java.util.concurrent.ConcurrentHashMap; | ||
import java.util.concurrent.locks.Lock; | ||
import java.util.concurrent.locks.ReentrantLock; | ||
|
||
import static com.alibaba.fluss.utils.concurrent.LockUtils.inLock; | ||
|
||
/** | ||
* This class help estimate the compression ratio for each table and each arrow compression type | ||
* combination. | ||
*/ | ||
@Internal | ||
@ThreadSafe | ||
public class ArrowCompressionRatioEstimator { | ||
/** | ||
* The constant speed to increase compression ratio when a batch compresses better than | ||
* expected. | ||
*/ | ||
public static final float COMPRESSION_RATIO_IMPROVING_STEP = 0.005f; | ||
|
||
/** | ||
* The minimum speed to decrease compression ratio when a batch compresses worse than expected. | ||
*/ | ||
public static final float COMPRESSION_RATIO_DETERIORATE_STEP = 0.05f; | ||
|
||
private final Map<Long, Map<String, Float>> compressionRatio; | ||
private final Map<Long, Lock> tableLocks; | ||
|
||
public ArrowCompressionRatioEstimator() { | ||
compressionRatio = new ConcurrentHashMap<>(); | ||
tableLocks = new ConcurrentHashMap<>(); | ||
} | ||
|
||
/** | ||
* Update the compression ratio estimation for a table and related compression info with the | ||
* observed compression ratio. | ||
*/ | ||
public void updateEstimation( | ||
long tableId, ArrowCompressionInfo compressionInfo, float observedRatio) { | ||
Lock lock = tableLocks.computeIfAbsent(tableId, k -> new ReentrantLock()); | ||
inLock( | ||
lock, | ||
() -> { | ||
Map<String, Float> compressionRatioMap = | ||
compressionRatio.computeIfAbsent( | ||
tableId, k -> new ConcurrentHashMap<>()); | ||
String compressionKey = compressionInfo.toString(); | ||
float currentEstimation = | ||
compressionRatioMap.getOrDefault(compressionKey, 1.0f); | ||
if (observedRatio > currentEstimation) { | ||
compressionRatioMap.put( | ||
compressionKey, | ||
Math.max( | ||
currentEstimation + COMPRESSION_RATIO_DETERIORATE_STEP, | ||
observedRatio)); | ||
} else if (observedRatio < currentEstimation) { | ||
compressionRatioMap.put( | ||
compressionKey, | ||
Math.max( | ||
currentEstimation - COMPRESSION_RATIO_IMPROVING_STEP, | ||
observedRatio)); | ||
} | ||
}); | ||
} | ||
|
||
/** Get the compression ratio estimation for a table and related compression info. */ | ||
public float estimation(long tableId, ArrowCompressionInfo compressionInfo) { | ||
Lock lock = tableLocks.computeIfAbsent(tableId, k -> new ReentrantLock()); | ||
return inLock( | ||
lock, | ||
() -> { | ||
Map<String, Float> compressionRatioMap = | ||
compressionRatio.computeIfAbsent( | ||
tableId, k -> new ConcurrentHashMap<>()); | ||
String compressionKey = compressionInfo.toString(); | ||
|
||
if (!compressionRatioMap.containsKey(compressionKey)) { | ||
compressionRatioMap.put(compressionKey, 1.0f); | ||
} | ||
|
||
return compressionRatioMap.get(compressionKey); | ||
}); | ||
} | ||
} |
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
Oops, something went wrong.