Skip to content
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

ACCUMULO-4708 Limit RFile block size to 2GB #300

Closed
wants to merge 17 commits into from

Conversation

PircDef
Copy link
Member

@PircDef PircDef commented Sep 15, 2017

Put in checks for data sizes and parameter size. Attempts to fail if the key or key/value is too large, but there is a catch in BCFile in case too much data is somehow written.

Put in checks for data sizes and parameter size.
Fixed some incomplete javadoc
*/
public boolean sanityCheckKey() {
// If fails the check
if ((long) ((long) this.row.length + (long) this.colFamily.length + (long) this.colQualifier.length + (long) this.colVisibility.length) > Integer.MAX_VALUE) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There's a few extra casts here. You only need to cast the first int to a long in this sum. Java will cast the rest. If you want to keep those, to be more explicit, that's fine, but the outer cast is redundant, as this is already a summation of longs.

@@ -404,7 +404,7 @@ public void flushIfNeeded() throws IOException {
private SampleLocalityGroupWriter sample;

private SummaryStatistics keyLenStats = new SummaryStatistics();
private double avergageKeySize = 0;
private double averageKeySize = 0;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍

@@ -426,6 +426,10 @@ public void append(Key key, Value value) throws IOException {
throw new IllegalArgumentException("Keys appended out-of-order. New key " + key + ", previous key " + prevKey);
}

if ((long) ((long) key.getSize() + (long) value.getSize()) >= Integer.MAX_VALUE) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same comment about casts.

@@ -528,6 +540,10 @@ public void close() throws IOException {

public Writer(BlockFileWriter bfw, int blockSize) throws IOException {
this(bfw, blockSize, (int) DefaultConfiguration.getInstance().getAsBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX), null, null);
long indexBlockSize = DefaultConfiguration.getInstance().getAsBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX);
if (indexBlockSize > Integer.MAX_VALUE || indexBlockSize < 0) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This appears to only be checking the default value... which is hard-coded. A check in the unit test for Property could ensure we don't make a stupid mistake for the default. I think we may have a validate method in Property.java, also, which might be used to do a quick verification that a value is good when a user sets it or when the process starts up and reads the config for the first time.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Went with a unit test on the default configuration. The changes where I added Preconditions.checkArgument are checking the values when they are first read from the configuration.

@@ -82,7 +82,13 @@ protected FileSKVWriter openWriter(OpenWriterOperation options) throws IOExcepti
AccumuloConfiguration acuconf = options.getTableConfiguration();

long blockSize = acuconf.getAsBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE);
if (blockSize >= Integer.MAX_VALUE || blockSize < 0) {
throw new IllegalArgumentException("table.file.compress.blocksize must be greater than 0 and less than " + Integer.MAX_VALUE);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think there might be a Guava Preconditions.checkArgument(boolean check, String message) which would simplify these IllegalArgumentException checks and throws.

*/
if (size() == Integer.MAX_VALUE) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍 Seems like a reasonable strategy here.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just to be safe, could also do >= in case implementation changes. We know it should never be larger than max int, but if implementation starts returning some larger value (for whatever reason), we don't want to let that through either.

@ctubbsii
Copy link
Member

I see the pull request is against master. I'm thinking we might want to pull this back onto 1.7 and 1.8. We can do that (or you can) after we get this into master, or you can base your changes on the 1.7 branch, and we can merge forward onto 1.8 and master branches after merging it into 1.7. It's up to you.

Fixed redudant casting
Used Preconditions.checkArgument()
Put in a test to check DefaultConfiguration's indexBlockSize
@@ -426,6 +426,10 @@ public void append(Key key, Value value) throws IOException {
throw new IllegalArgumentException("Keys appended out-of-order. New key " + key + ", previous key " + prevKey);
}

if (((long) key.getSize() + (long) value.getSize()) >= Integer.MAX_VALUE) {
throw new IllegalArgumentException("Key/value pair is too large to be appended to RFile.");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Anything we can do to help someone on the receiving end of this message debug it would be good. We could include the size of key+value in the error message. Also, Key.toString() truncates anything that is too long so we could leverage that possibly include the key.

@@ -426,6 +426,10 @@ public void append(Key key, Value value) throws IOException {
throw new IllegalArgumentException("Keys appended out-of-order. New key " + key + ", previous key " + prevKey);
}

if (((long) key.getSize() + (long) value.getSize()) >= Integer.MAX_VALUE) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Serialization will also add overhead. For example some bytes encoding the length of the row are written before the rows data. So this means that what can really be written is less than what is checked for here. Not sure if we want to try to account for that here. We could do something simple like assume each field will need at least 5 bytes for the length (I think 5 is the max because we are using var int).

Even if not accounting for that serialization overhead this is still a really nice check. It will catch most situations at the source leaving a small range of key+val size that would only be caught later when encoding the block.

// If the block reaches or exceeds 2GB, it has no way to determine the amount of data actually written. To prevent
// this, we check to see if adding the key/value will create a problem, and if it will, we force a transition to
// the next block.
} else if (((long) key.getSize() + (long) value.getSize() + blockWriter.getRawSize()) >= Integer.MAX_VALUE) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

may want to consider adding some arbitrary overhead for serialization metadata here... like 128 bytes.

@@ -101,6 +101,9 @@ private final void init(byte r[], int rOff, int rLen, byte cf[], int cfOff, int
colVisibility = copyIfNeeded(cv, cvOff, cvLen, copy);
timestamp = ts;
deleted = del;
if (!sanityCheckKey()) {
throw new IllegalArgumentException("Invalid Key entry. Key exceeds " + Integer.MAX_VALUE + " bytes.");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would be nice to include the size. Also why not have the sanityCheck method throw the exception?

Accounted for overhead with keys when checking sizes
Added exception throwing for sanity checking of keys into the sanitycheck
Added more information to exceptions
Copy link
Contributor

@keith-turner keith-turner left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@PircDef these are really nice changes.

It seems hard to automatically test these changes. Did you have any thoughts on testing? Were you able to do any manual testing?

* Throws an exception if the key is too large.
*
*/
public void sanityCheckKey() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why make this public and add it to the API?

@@ -426,6 +426,12 @@ public void append(Key key, Value value) throws IOException {
throw new IllegalArgumentException("Keys appended out-of-order. New key " + key + ", previous key " + prevKey);
}

// 5 bytes of overhead for fields: value, row, colFamily, colQualifier, colVisibility, and 8 for the long timestamp
if (((long) key.getSize() + (long) value.getSize() + (5L * 5L + 8L)) >= Integer.MAX_VALUE) {
throw new IllegalArgumentException("Key/value pair is too large (" + ((long) key.getSize() + (long) value.getSize())
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would separate the key and val sizes in the message... something like the following with a space between... gives a little more info.

throw new IllegalArgumentException("Key/value pair is too large (" + ((long) key.getSize() +" " + (long) value.getSize())

@@ -426,6 +426,12 @@ public void append(Key key, Value value) throws IOException {
throw new IllegalArgumentException("Keys appended out-of-order. New key " + key + ", previous key " + prevKey);
}

// 5 bytes of overhead for fields: value, row, colFamily, colQualifier, colVisibility, and 8 for the long timestamp
if (((long) key.getSize() + (long) value.getSize() + (5L * 5L + 8L)) >= Integer.MAX_VALUE) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could make (5L * 5L + 8L) a constant in RFile and then Key could also use that constant. The constant could have a comment about its purpose then the code that uses it does not need that comment. I am thinking it would be better to put the constant in RFile instead of Key because Key is in the public API, so the constant would end up in the API if in Key.

sanityCheckKey is now a private
Overhead value is now a constant
Improved an error message
if (((long) this.row.length + (long) this.colFamily.length + (long) this.colQualifier.length + (long) this.colVisibility.length + (4L * 5L + 8L)) >= Integer.MAX_VALUE) {
private void sanityCheckKey() {
//If the key is too large, we throw an exception. We subtract 5L from KEY_VALUE_OVERHEAD to remove the accounting for value overhead
if (((long) this.row.length + (long) this.colFamily.length + (long) this.colQualifier.length + (long) this.colVisibility.length + (RFile.KEY_VALUE_OVERHEAD-5L)) >= Integer.MAX_VALUE) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

could have two constants in RFile : KEY_OVERHEAD and VALUE_OVERHEAD then the subtraction of 5L is not needed here, just use KEY_OVERHEAD

@@ -118,6 +118,9 @@ private RFile() {}
// Buffer sample data so that many sample data blocks are stored contiguously.
private static int sampleBufferSize = 10000000;

// 5 bytes of overhead for fields: value, row, colFamily, colQualifier, colVisibility, and 8 for the timestamp
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

would nice to mention what the overhead is... its worst case serialization overhead

Split KEY_VALUE_OVERHEAD to KEY_OVERHEAD and VALUE_OVERHEAD and updated comment on it.
public void testDefaultBlockSizeIsAllowed() {
long bsi = c.getAsBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX);
assert (bsi > 0);
assert (bsi < Integer.MAX_VALUE);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Probably best to avoid assert keyword statements, because assertions can be disabled in the JVM. Best to use JUnit provided APIs for assertions: Assert.assertTrue( expr )

Also, might be good to add a small one-line comment explaining what we're checking for. As I understand it, this is mostly just a sanity check on our default.

We can check more generally that any value in the config is suitable by putting this check in:

https://github.com/apache/accumulo/blob/master/core/src/main/java/org/apache/accumulo/core/conf/ConfigSanityCheck.java#L46 in the validate() method. This gets executed whenever we read various configurations. We could add a check to ensure that whatever the user has configured, whichever configuration they've configured it in, that it has a reasonable value. This is very similar to how we're already checking that the INSTANCE_ZK_TIMEOUT value is within an allowable range.

Moved the configuation sanity check to ConfigSanityCheck instead of unit tets
Added a >= to the BCFile stream size check
Added unit tests for a key that is too large or a (key, value) pair that is too large. This resulted in a modification to the pom.xml to allow users to configure memory.
Gave pom.xml variable a better name
pom.xml Outdated
@@ -1548,6 +1548,7 @@
<findbugs.skip>true</findbugs.skip>
<skipITs>true</skipITs>
<skipTests>true</skipTests>
<unitTestMemSize>-Xmx1G</unitTestMemSize>
Copy link
Member

@ctubbsii ctubbsii Sep 22, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this was intended to be placed in the main properties section, not the one inside this profile. A simple mistake. :)

@Test
public void testKeyTooLarge() throws Exception {
// If we don't have enough memory (6G) skip these very memory-intensive unit tests
Assume.assumeTrue(Runtime.getRuntime().maxMemory() >= 5726797824L);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's not clear where these numeric literals come from. Perhaps it can be put in a variable, and their computation shown, so it's clear where they come from? If they are arbitrary, then a comment explaining that (eg. "arbitrary size in the range of ... ") would be helpful.

Removed magic numbers
Added comments to help explain the unit tests
Fixed a misleading error message for keys that are too large.
Copy link
Member

@ctubbsii ctubbsii left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Changes look good to me, but I'd like to see another review from @keith-turner to be sure.

@keith-turner
Copy link
Contributor

I took another in depth look at this change and I am not sure the premise of this fix is correct. Looking at the code it seems that in the case where the key+value exceeds 2G but compresses to less than 2G that it may be ok. I need to do more investigation.

@keith-turner
Copy link
Contributor

I ran the following test against master and it was successful after I made the changes mentioned in the comment at the end of the patch. Based on this it seems that entries large than 2G can currently work. The small changes I made were related to the fact that it seems that MAX_INT-2 is the largest array you can allocate in java.

https://gist.github.com/keith-turner/90e318057f55b080dd93aa00277faa4b

I was surprised the test with uncompressible data worked. I suspect it created a compressed block larger than 2G which I was thinking would fail. Need to dig and understand why it worked.

@PircDef
Copy link
Member Author

PircDef commented Oct 18, 2017

If we can, we want to grab compressed data before it is written to disk. This way we insert controls on the file size without speculating about the results of compression.

@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@apache apache deleted a comment from asfbot Oct 18, 2017
@ctubbsii
Copy link
Member

@asfbot went crazy and started replying to its own replies to the notification list, in an endless loop. Infra took a look and I think the problem is resolved now. I've tried to delete the spammy comments from @asfbot ; Hopefully the comment thread is cleaner now. I'll try to clean up the JIRA worklog later, because that got spammed, too. Can't do anything about the notifications list, though.

Added a cipher stream that tracks encrypted data length
Removed they key size limitations added in prior commits
Updated comments about block size
if (count > maxOutputSize) {
throw new IOException("Attempt to write " + count + " bytes was made. A maximum of " + maxOutputSize + " is allowed for an encryption stream.");
}
obuffer = cipher.update(b, off, len);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can you call super.write here?

Updated overrides to count more accurately.
Added a comment suggesting the use of RFileCipherOutputStream with CryptoModule
Copy link
Contributor

@keith-turner keith-turner left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks good @PircDef

@keith-turner
Copy link
Contributor

Squashed and merged this in e875f01

Thanks for the contribution @PircDef. Would you like to added to the Accumulo people page. If so let me know what info you would like added or submit a PR to the accumulo-website repo modifying the people page.

@PircDef PircDef deleted the ACCUMULO-4708 branch May 7, 2021 14:47
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

3 participants