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
[CARBONDATA-3692] Support NoneCompression during loading data. #3611
base: master
Are you sure you want to change the base?
Conversation
Why is this PR needed? In some cases, the data need to be uncompressed after loading into Carbondata file. In the current version, the project do not support loading data without compression. What changes were proposed in this PR? Provide a new Compressor as NoneCompressor implement the AbstractCompressor. This compressor can be set by calling CarbonProperties.getInstance().addProperty(CarbonCommonConstants.COMPRESSOR,"none"); Does this PR introduce any user interface change? Yes Is any new testcase added? Yes
Can one of the admins verify this patch? |
Can you please explain the scenario where no-compression would be beneficial? |
This NoneCompress Compressor will improve the speed of loading data from Flink to OBS File by trade-off space and IO in some cases. For example: when loading data from Flink to OBS, data needs to be compressed by Flink to temporary files and then decompressed by OBS. |
@Pickupolddriver : Agree that it can improve the loading speed. But data will be 3x bigger. So, storage cost on OBS will be 3x more! |
CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_OFFHEAP_SORT, "true") | ||
CarbonProperties.getInstance().addProperty(CarbonCommonConstants.COMPRESSOR, "none") | ||
loadData() | ||
checkAnswer(sql(s"SELECT count(*) FROM $tableName"), Seq(Row(16))) |
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.
Can you change select count(*) to select * so that actual data is validated
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.
So you want to change all the test cases in this class from select count(*) to *?
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.
No, just change the newly added test cases to select *
Data would be processed after loaded to OBS. So if we could provide a NonCompressor, it could avoid the data being compressed and then uncompressed. And the uncompressed data would be deleted after processed in OBS. |
@@ -35,6 +35,7 @@ | |||
private final Map<String, Compressor> allSupportedCompressors = new HashMap<>(); | |||
|
|||
public enum NativeSupportedCompressor { | |||
NONE("none",NoneCompressor.class), |
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.
add space after ,
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.
suggest change to NONE("nocompress", NoneCompressor.class)
nocompress
will be appended to the data file name
@@ -0,0 +1,51 @@ | |||
package org.apache.carbondata.core.datastore.compression; |
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.
please add license header as other source file
|
||
import java.io.IOException; | ||
|
||
public class NoneCompressor extends AbstractCompressor { |
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.
You can add description to this class that it does not perform any compression
add to whitelist |
please rebase |
Why is this PR needed?
In some cases, the data need to be uncompressed after loading into Carbondata file.
In the current version, the project do not support loading data without compression.
What changes were proposed in this PR?
Provide a new Compressor as NoneCompressor implement the AbstractCompressor.
This compressor can be set by calling
CarbonProperties.getInstance().addProperty(CarbonCommonConstants.COMPRESSOR,"none");
Does this PR introduce any user interface change?
Yes
Is any new testcase added?
Yes