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

ZOOKEEPER-3496: Transaction larger than jute.maxbuffer makes ZooKeeper service unavailable #1080

Closed
wants to merge 4 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
11 changes: 11 additions & 0 deletions zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md
Expand Up @@ -1077,6 +1077,17 @@ the variable does.
problems will arise. This is really a sanity check. ZooKeeper is
designed to store data on the order of kilobytes in size.

* *jute.maxbuffer.extrasize*:
(Java system property: **zookeeper.jute.maxbuffer.extrasize**)
**New in 3.5.7:**
While processing client requests ZooKeeper server adds some additional information into
the requests before persisting it as a transaction. Earlier this additional information size
was fixed to 1024 bytes. For many scenarios, specially scenarios where jute.maxbuffer value
is more than 1 MB and request type is multi, this fixed size was insufficient.
To handle all the scenarios additional information size is increased from 1024 byte
to same as jute.maxbuffer size and also it is made configurable through jute.maxbuffer.extrasize.
Generally this property is not required to be configured as default value is the most optimal value.

* *skipACL* :
(Java system property: **zookeeper.skipACL**)
Skips ACL checks. This results in a boost in throughput,
Expand Down
Expand Up @@ -27,8 +27,23 @@
*
*/
public class BinaryInputArchive implements InputArchive {
static public final String UNREASONBLE_LENGTH= "Unreasonable length = ";
public static final String UNREASONBLE_LENGTH= "Unreasonable length = ";
public static final int maxBuffer = Integer.getInteger("jute.maxbuffer", 0xfffff);
private static final int extraMaxBuffer;

static {
final Integer configuredExtraMaxBuffer =
Integer.getInteger("zookeeper.jute.maxbuffer.extrasize", maxBuffer);
Copy link
Contributor

Choose a reason for hiding this comment

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

one suggestion: existing maxbuffer has a name of "jute.maxbuffer" (note, without zookeeper prefix), so it might be better
that we name this extra size property as "jute.maxbuffer.extrasize" to be consistent with existing style.

also, we need documentation on this new property.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

There is big problem with jute.maxbuffer. It can not be configured in zoo.cfg. We are forced to configure it though java system property like -Djute.maxbuffer=someValue.

When we are giving property name zookeeper.jute.maxbuffer.extrasize. it can be configured in zoo.cfg as jute.maxbuffer.extrasize=someValue

This is the reason I do not want to be consistent with jute.maxbuffer. is it OK?

Copy link
Contributor

Choose a reason for hiding this comment

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

i see - sounds reasonable. I am ok with current name.

side note - I never really remember why we have a few configuration options that's only available through java system properties and what's the criteria of choosing one versus another.

if (configuredExtraMaxBuffer < 1024) {
// Earlier hard coded value was 1024, So the value should not be less than that value
extraMaxBuffer = 1024;
} else {
extraMaxBuffer = configuredExtraMaxBuffer;
}
}
private DataInput in;
private int maxBufferSize;
private int extraMaxBufferSize;

static public BinaryInputArchive getArchive(InputStream strm) {
return new BinaryInputArchive(new DataInputStream(strm));
Expand All @@ -48,7 +63,13 @@ public void incr() {
}
/** Creates a new instance of BinaryInputArchive */
public BinaryInputArchive(DataInput in) {
this(in, maxBuffer, extraMaxBuffer);
}

public BinaryInputArchive(DataInput in, int maxBufferSize, int extraMaxBufferSize) {
this.in = in;
this.maxBufferSize = maxBufferSize;
this.extraMaxBufferSize = extraMaxBufferSize;
}

public byte readByte(String tag) throws IOException {
Expand Down Expand Up @@ -83,8 +104,6 @@ public String readString(String tag) throws IOException {
in.readFully(b);
return new String(b, "UTF8");
}

static public final int maxBuffer = Integer.getInteger("jute.maxbuffer", 0xfffff);

public byte[] readBuffer(String tag) throws IOException {
int len = readInt(tag);
Expand Down Expand Up @@ -123,7 +142,7 @@ public void endMap(String tag) throws IOException {}
// make up for extra fields, etc. (otherwise e.g. clients may be able to
// write buffers larger than we can read from disk!)
private void checkLength(int len) throws IOException {
if (len < 0 || len > maxBuffer + 1024) {
if (len < 0 || len > maxBufferSize + extraMaxBufferSize) {
throw new IOException(UNREASONBLE_LENGTH + len);
}
}
Expand Down
Expand Up @@ -21,8 +21,10 @@
import org.junit.Test;

import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.io.IOException;

import java.nio.ByteBuffer;
import java.security.SecureRandom;

// TODO: introduce JuteTestCase as in ZKTestCase
public class BinaryInputArchiveTest {
Expand All @@ -41,4 +43,59 @@ public void testReadStringCheckLength() {
e.getMessage().startsWith(BinaryInputArchive.UNREASONBLE_LENGTH));
}
}

/**
* Record length is more than the maxbuffer + extrasize length
*/
@Test
public void testReadStringForRecordsHavingLengthMoreThanMaxAllowedSize() {
int maxBufferSize = 2000;
int extraMaxBufferSize = 1025;
//this record size is more than the max allowed size
int recordSize = maxBufferSize + extraMaxBufferSize + 100;
BinaryInputArchive ia =
getBinaryInputArchive(recordSize, maxBufferSize, extraMaxBufferSize);
try {
ia.readString("");
Assert.fail("Should have thrown an IOException");
} catch (IOException e) {
Assert.assertTrue("Not 'Unreasonable length' exception: " + e,
e.getMessage().startsWith(BinaryInputArchive.UNREASONBLE_LENGTH));
}
}

/**
* Record length is less than then maxbuffer + extrasize length
*/
@Test
public void testReadStringForRecordsHavingLengthLessThanMaxAllowedSize()
throws IOException {
int maxBufferSize = 2000;
int extraMaxBufferSize = 1025;
int recordSize = maxBufferSize + extraMaxBufferSize - 100;
//Exception is not expected as record size is less than the allowed size
BinaryInputArchive ia =
getBinaryInputArchive(recordSize, maxBufferSize, extraMaxBufferSize);
String s = ia.readString("");
Assert.assertNotNull(s);
Assert.assertEquals(recordSize, s.getBytes().length);
}

private BinaryInputArchive getBinaryInputArchive(int recordSize, int maxBufferSize,
int extraMaxBufferSize) {
byte[] data = getData(recordSize);
DataInputStream dis = new DataInputStream(new ByteArrayInputStream(data));
return new BinaryInputArchive(dis, maxBufferSize, extraMaxBufferSize);
}

private byte[] getData(int recordSize) {
ByteBuffer buf = ByteBuffer.allocate(recordSize + 4);
buf.putInt(recordSize);
byte[] bytes = new byte[recordSize];
for (int i = 0; i < recordSize; i++) {
bytes[i] = (byte) 'a';
}
buf.put(bytes);
return buf.array();
}
}