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

Conversation

arshadmohammad
Copy link
Contributor

No description provided.

@@ -85,6 +85,14 @@ public String readString(String tag) throws IOException {
}

static public final int maxBuffer = Integer.getInteger("jute.maxbuffer", 0xfffff);
static public int readExtraSize = Integer.getInteger("zookeeper.jute.maxbuffer.extrasize",
Copy link
Contributor

Choose a reason for hiding this comment

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

please make it final

Copy link
Contributor Author

Choose a reason for hiding this comment

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

variable readExtraSize is reinitialized at line number 93. So I think it cannot be marked final.

maxBuffer);
static {
// Earlier hard coded value is 1024, So the value should not be less than that
if (readExtraSize < 1024) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Assign an initial value to a temp variable here, do this small computation, then assign the value to the variable.
It is legal to assign a value in a static block to a final static field, the only constraint is that you do the assignement only once

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ok, changed it

@Before
public void setup() throws Exception {
// Request size for 100 nodes in this test class is 6197 bytes
System.setProperty(ZKConfig.JUTE_MAXBUFFER, Integer.toString(6197));
Copy link
Contributor

Choose a reason for hiding this comment

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

This trick won't work, because the variable will be inizialized only at the first loading of the class.
Once you assign a value you cannot change it

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ZKConfig.JUTE_MAXBUFFER is initialized with only one value that is why it was working, anyway I moved initialization to class level so it is initialized only once

Copy link
Contributor

Choose a reason for hiding this comment

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

@arshadmohammad if you run this test together with the other ones you will see that the system property does not have effect.
You can add an assertion in your test case that the value of readExtraSize is the expected value and it is different from the default one.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks @eolivelli for your suggestions. I had run this test in ant build setup and it was working fine. I think in ant it was forking new jvm for every test class. that is why it might have passed.
But yes , if in one jvm multiple tests are running, value assigned to org.apache.jute.BinaryInputArchive#maxBuffer can not be changed from this test case.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

So what shall we do, shall we commit without test case? any other idea?


@After
public void tearDown() throws Exception {
System.clearProperty(ZKConfig.JUTE_MAXBUFFER);
Copy link
Contributor

Choose a reason for hiding this comment

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

Same here

Copy link
Contributor Author

Choose a reason for hiding this comment

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

moved to class level clean up

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

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

The change is okay.
We have to fix the test IMHO

@Before
public void setup() throws Exception {
// Request size for 100 nodes in this test class is 6197 bytes
System.setProperty(ZKConfig.JUTE_MAXBUFFER, Integer.toString(6197));
Copy link
Contributor

Choose a reason for hiding this comment

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

@arshadmohammad if you run this test together with the other ones you will see that the system property does not have effect.
You can add an assertion in your test case that the value of readExtraSize is the expected value and it is different from the default one.

@eolivelli
Copy link
Contributor

The failure of Maven build is due to a problem on the C client, not related to this patch

client/zookeeper-client-c/generated -Wall -Werror -Wdeclaration-after-statement -fprofile-arcs -ftest-coverage -g -O2 -D_GNU_SOURCE -MT zookeeper.lo -MD -MP -MF .deps/zookeeper.Tpo -c /home/jenkins/jenkins-slave/workspace/PreCommit-ZOOKEEPER-github-pr-build-maven/zookeeper-client/zookeeper-client-c/src/zookeeper.c  -fPIC -DPIC -o .libs/zookeeper.o
     [exec] /home/jenkins/jenkins-slave/workspace/PreCommit-ZOOKEEPER-github-pr-build-maven/zookeeper-client/zookeeper-client-c/src/zookeeper.c: In function 'print_completion_queue':
     [exec] /home/jenkins/jenkins-slave/workspace/PreCommit-ZOOKEEPER-github-pr-build-maven/zookeeper-client/zookeeper-client-c/src/zookeeper.c:2560:5: error: null argument where non-null required (argument 1) [-Werror=nonnull]
     [exec]      fprintf(LOGSTREAM,"Completion queue: ");
     [exec]      ^~~~~~~
     [exec] /home/jenkins/jenkins-slave/workspace/PreCommit-ZOOKEEPER-github-pr-build-maven/zookeeper-client/zookeeper-client-c/src/zookeeper.c:2562:9: error: null argument where non-null required (argument 1) [-Werror=nonnull]
     [exec]          fprintf(LOGSTREAM,"empty\n");
     [exec]          ^~~~~~~
     [exec] /home/jenkins/jenkins-slave/workspace/PreCommit-ZOOKEEPER-github-pr-build-maven/zookeeper-client/zookeeper-client-c/src/zookeeper.c:2568:9: error: null argument where non-null required (argument 1) [-Werror=nonnull]
     [exec]          fprintf(LOGSTREAM,"%d,",cptr->xid);
     [exec]          ^~~~~~~
     [exec] /home/jenkins/jenkins-slave/workspace/PreCommit-ZOOKEEPER-github-pr-build-maven/zookeeper-client/zookeeper-client-c/src/zookeeper.c:2571:5: error: null argument where non-null required (argument 1) [-Werror=nonnull]
     [exec]      fprintf(LOGSTREAM,"end\n");
     [exec]      ^~~~~~~
     [exec] Makefile:1032: recipe for target 'zookeeper.lo' failed
     [exec] cc1: all warnings being treated as errors
     [exec] make: *** [zookeeper.lo] Error 1
[INFO] ----------------------------------------------------

@eolivelli
Copy link
Contributor

@arshadmohammad Do you want this fix to go in 3.5.6 ? I am going to cut the release, I can wait for you

@eolivelli
Copy link
Contributor

Oh I see now that you are targeting branch 3.5.
Please target the PR to "master" then we will cherry-pick/backport to 3.5

@arshadmohammad
Copy link
Contributor Author

@arshadmohammad Do you want this fix to go in 3.5.6 ? I am going to cut the release, I can wait for you

Please go ahead, we can take this issue in the next version

@@ -85,6 +85,18 @@ public String readString(String tag) throws IOException {
}

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

Choose a reason for hiding this comment

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

In order to have better testing we can do as follows:

  • make maxBuffer and readExtraSize two instance variables
  • add two new constants (final static) DEFAULT_MAX_BUFFER and DEFAULT_READEXTRASIZE initialized with the system properties
  • in the constructor initialize the value using DEFAULT_MAX_BUFFER and DEFAULT_READEXTRASIZE
  • add a new constructor that allows to initialize maxBuffer and readExtraSize
  • add only unit tests about BinaryInputArchive, and not end to end tests

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I liked the approach. thanks. Addressed the comments more or less same way. Please have a look.
maxBuffer is used many places and it is not the default value, it is the configured value. So leaving this variable as it is. created other instance variable as suggested but with different name, hope it is ok.

Integer.getInteger("zookeeper.jute.maxbuffer.extrasize", maxBuffer);
if (configuredValue < 1024) {
// Earlier hard coded value is 1024, So the value should not be less than that
readExtraSize = 1024;
Copy link
Contributor

Choose a reason for hiding this comment

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

Do we have a reasonable estimate of an upper bound of the extra field padding which could reasonably cover more practice use cases, so users don't have to configure this value themselves? Obviously it's not tractable to find a single value that cover all cases (unless we use infinity), but it seems reasonable to increase this value - say by 10x while still be safeguarding requests with unreasonable length.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

In this patch we have changed the default value of extra size to same as max buffer value. I do not foresee any scenario where record extra size is more than the actual record size.
So this value is going to be sufficient and possibly there would not be any need to configure jute.maxbuffer.extrasize.

Copy link
Contributor

@hanm hanm left a comment

Choose a reason for hiding this comment

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

I am wondering if we can precisely guarantee we are consistent on both read and write path for the requests. Currently, the extra padding approach is more empirical and it's still possible for any preconfigured value, the check will produce false negative results.

One way of doing this is to check serialized requests on write path so we know precisely the size of the request on write path, but that will be slower comparing to existing implementation.

@arshadmohammad
Copy link
Contributor Author

I am wondering ....

  • Request actual size is known very late in the request processing chain. If very validate request size this time, purpose of sanity check is defeated..
  • If we try to create the request at the beginning of the request processing, i think sufficient data would not be available, also as you pointed out it would slower the performance for not good enough reasons.

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

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

LGTM


static {
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.

Copy link
Contributor

@hanm hanm left a comment

Choose a reason for hiding this comment

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

lgtm (just need doc update).

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 sufficient.
Copy link
Contributor

Choose a reason for hiding this comment

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

replace sufficient with insufficient ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

right, corrected it

Copy link
Contributor

@lvfangmin lvfangmin left a comment

Choose a reason for hiding this comment

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

+1

Still need to tune the extra size when things changed, like a user have a lot more sub txns in a multi-op, but this seems better to tune the jute.maxbuffer which will also allow user to send more data.

asfgit pushed a commit that referenced this pull request Sep 24, 2019
…r service unavailable

Author: Mohammad Arshad <arshad@apache.org>

Reviewers: Enrico Olivelli <eolivelli@gmail.com>, Michael Han <hanm@apache.org>, Allan Lyu <fangmin@apache.org>

Closes #1080 from arshadmohammad/ZOOKEEPER-3496
@hanm
Copy link
Contributor

hanm commented Sep 24, 2019

@arshadmohammad

can you also open a pull request to master branch? this pull request was targeting 3.5 instead of master and has a couple of merge conflicts with master branch.

this pul request is now merged to 3.5: 7d263d0, please close the pull request, thank you.

@arshadmohammad
Copy link
Contributor Author

Thanks @eolivelli @hanm and @lvfangmin for the reviews.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
4 participants