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
Allow user to enable FSDataOutputStream.setDropBehind on majc output #3083
Conversation
This commit introduces a new property, table.compaction.major.output.drop.cache, that defaults to false and if true will call setDropBehind on the major compaction file output stream.
boolean dropCacheBehindMajcOutput = false; | ||
if (!RootTable.ID.equals(this.extent.tableId()) | ||
&& !MetadataTable.ID.equals(this.extent.tableId()) | ||
&& acuTableConf.getBoolean(Property.TABLE_MAJC_OUTPUT_DROP_CACHE)) { |
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.
@keith-turner - I tried looking through new pluggable compaction service code, I'm thinking it may be possible for someone to implement their own CompactionConfigurer such that they can set this new property based on the KeyExtent that is getting compacted. Is that possible? If not, then we may want to add that functionality in a follow-on commit. I'm thinking of the case where you have a date sorted table and you may not want to drop the files from the page cache when compacting recent data, but may want to for older data.
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.
@keith-turner - I tried looking through new pluggable compaction service code, I'm thinking it may be possible for someone to implement their own CompactionConfigurer such that they can set this new property based on the KeyExtent that is getting compacted. Is that possible?
That would be possible and I think its a really good idea. Doing it in 2.1.1 has the same pros and cons as were discussed with adding a new prop in 2.1.1. The SPI definition does not prohibit that though. If there is a compelling enough configurer that could be written with the new method, then maybe that outweighs the possible confusion.
To make the change you suggested I think that could be done by adding key extent here. Adding to a new method there will not break any code written against 2.1.0, it only breaks things when a developer writes code against 2.1.1 and tries to run it against 2.1.0 where it could cause a method not found exception. However its possible no one ever bumps into this problem.
core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java
Outdated
Show resolved
Hide resolved
core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionConfigurer.java
Outdated
Show resolved
Hide resolved
server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java
Outdated
Show resolved
Hide resolved
core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionConfigurer.java
Outdated
Show resolved
Hide resolved
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 think all the API breakages are fixed. I'm okay with adding this to 2.1.1, so long as the presence of the property being set in 2.1.1 (either in the config file or in table properties in ZK) does not prevent downgrade back to 2.1.0 if that becomes necessary for a user to do.
hsync javadoc says, "Note that only the current block is flushed to the disk device. To guarantee durable sync across block boundaries the stream should be created with {@link CreateFlag#SYNC_BLOCK}." Modified code to create an output stream using the SYNC_BLOCK flag in the same way that VolumeManagerImpl.createSyncable does.
@ctubbsii - I tested the properties being set on a table and in the properties file in 2.1.1-SNAPSHOT, then started Accumulo using version 2.1.0. I didn't see any issues, no log messages about it and everything started fine. |
This commit introduces a new property, table.compaction.major.output.drop.cache, that defaults to false and if true will call setDropBehind on the major compaction file output stream.