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

Enhance Auditor #1683

Merged
merged 9 commits into from
Oct 3, 2018
Merged

Enhance Auditor #1683

merged 9 commits into from
Oct 3, 2018

Conversation

reddycharan
Copy link
Contributor

Descriptions of the changes in this PR:

  • store last checkAllLedgers executed time in metadataserver.
  • use that value for determining the initialDelay of checkAllLedgers call
  • add dedicated metric for bookieCheck
  • add more log lines in Auditor

Motivation

In Auditor checkAllLedgers is scheduled to run for every ‘auditorPeriodicCheckInterval’ and with initial delay of ‘auditorPeriodicCheckInterval’. But the problem is if Auditor role keeps transforming to another AR process, it might not get chance to do ‘checkAllLedgers’ task. Considering default value for 'auditorPeriodicCheckInterval' is 1 week, it is not uncommon situation.

Changes

Thats why I'm planning to persist last successful checkAllLedgers execution ctime in ZK, so that next time when a new AR becomes Auditor, it would use that as reference to decide the initial delay value.

@reddycharan
Copy link
Contributor Author

@pasha-kuznetsov fyi

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.

I wonder if we should provide some tool to reset the new item on ZK.
Maybe it is not very useful

Overall this change looks good to me, I left some nits

@@ -184,6 +186,7 @@ private void submitShutdownTask() {
executor.submit(new Runnable() {
public void run() {
synchronized (Auditor.this) {
LOG.error("Shutting down Auditor's Executor");
Copy link
Contributor

Choose a reason for hiding this comment

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

Error?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

changed it to info

} catch (UnavailableException ue) {
LOG.error("Got UnavailableException while trying to get checkAllLedgersCTime", ue);
}
long durationSinceLastExecutionInSecs = (System.currentTimeMillis() - checkAllLedgersLastExecutedCTime)
Copy link
Contributor

Choose a reason for hiding this comment

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

This will be weird if checkAllLedgersLastExecutedCTime is -1

Btw it will work

Copy link
Contributor Author

Choose a reason for hiding this comment

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

it was intentional, but i see your concern there. I'll make adjustments to make it more sensible.

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.

+1 awesome work


@Override
public void setCheckAllLedgersCTime(long checkAllLedgersCTime) throws UnavailableException {
LOG.debug("setCheckAllLedgersCTime");
Copy link
Member

Choose a reason for hiding this comment

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

nit : add if (LOG.isDebugEnabled())

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done


@Override
public long getCheckAllLedgersCTime() throws UnavailableException {
LOG.debug("getCheckAllLedgersCTime");
Copy link
Member

Choose a reason for hiding this comment

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

nit : add if (LOG.isDebugEnabled())

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

LOG.debug("setCheckAllLedgersCTime");
try {
if (zkc.exists(checkAllLedgersCtimeZnode, false) != null) {
zkc.setData(checkAllLedgersCtimeZnode, Long.toString(checkAllLedgersCTime).getBytes(UTF_8), -1);
Copy link
Member

Choose a reason for hiding this comment

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

can you make a util function for Long.toString(checkAllLedgersCTime).getBytes(UTF_8)? I see it is used multiple places.

regardless, shall we consider using a protobuf structure for future extensibility?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

added protobuf

LOG.debug("getCheckAllLedgersCTime");
try {
byte[] data = zkc.getData(checkAllLedgersCtimeZnode, false, null);
return Long.parseLong(new String(data, UTF_8));
Copy link
Member

Choose a reason for hiding this comment

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

Long.parseLong will throw NumberFormatException. Can you catch that and return -1?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

added protobuf

Copy link
Member

@sijie sijie left a comment

Choose a reason for hiding this comment

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

overall looks good. just one general comment, shall we be using protobuf for serializing metadata for future extensibility?

@eolivelli
Copy link
Contributor

@sijie the idea about protobuf is good.
Let's elaborate this idea please

return Long.parseLong(new String(data, UTF_8));
} catch (KeeperException.NoNodeException ne) {
LOG.warn("checkAllLedgersCtimeZnode is not yet available");
return -1;
Copy link
Contributor

@jvrao jvrao Sep 21, 2018

Choose a reason for hiding this comment

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

Why not create the node on auditor startup with -1; So the node exists all the time.

Copy link
Contributor

Choose a reason for hiding this comment

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

users are even able to delete the znode from zk manually, it is better not to have this assumption in this case.

Copy link
Contributor

Choose a reason for hiding this comment

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

we are in a big soup if the user deletes whatever they want. I would rather get an alert/error if that happens.

Copy link
Contributor

Choose a reason for hiding this comment

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

okay, makes sense.
I was just saying that in this case we can be somehow tolerant to this simple case.
in general if the user alters zk contents manually a big mess will happen

so okay to fail

Copy link
Contributor Author

Choose a reason for hiding this comment

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

we don't want this to be part of cluster initialization stuff, this is kind of good to have information but not must have. It brings unnecessary burden for cluster initialization part. For all practical purposes if zNode is not existing / not yet created or if there is an issue in reading this zNode, then lets consider it as '-1' and move forward.

Copy link
Contributor

Choose a reason for hiding this comment

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

Okay @reddycharan you are right

@@ -184,6 +186,7 @@ private void submitShutdownTask() {
executor.submit(new Runnable() {
public void run() {
synchronized (Auditor.this) {
LOG.info("Shutting down Auditor's Executor");
Copy link
Contributor

Choose a reason for hiding this comment

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

Can't we just say - Shutting down the Auditor? 'executor' adds any value?

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 separate Auditor.shutDown method, i don't want to mix them up.

Anyhow I feel some more logging is needed for this method Auditor.submitShutdownTask

} else {
durationSinceLastExecutionInSecs = (System.currentTimeMillis() - checkAllLedgersLastExecutedCTime)
/ 1000;
initialDelay = durationSinceLastExecutionInSecs > interval ? 0
Copy link
Contributor

Choose a reason for hiding this comment

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

Shouldn't it be reversed? i.e (interval > durationSinceLastExecutionInSecs) ?

Copy link
Contributor

Choose a reason for hiding this comment

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

I missed 0 in there.

durationSinceLastExecutionInSecs = -1;
initialDelay = 0;
} else {
durationSinceLastExecutionInSecs = (System.currentTimeMillis() - checkAllLedgersLastExecutedCTime)
Copy link
Contributor

Choose a reason for hiding this comment

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

This may become -ve due to time skews between machines. Check for -ve value. Especially given that you are logging this info below.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

will fix that

checkAllLedgers();
LOG.info("Completed checkAllLedgers");
Copy link
Contributor

Choose a reason for hiding this comment

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

Since you added log info; add more info if possible. At least the time it took and maybe how many ledgers it checked??

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 metric right below for 'checkAllLedgersTime' and if checkAllLedgers succeeded then it should have checked all the ledgers, so not sure if it is the right place to expect / log that info.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

anyhow added log for duration of execution

try {
ledgerUnderreplicationManager.setCheckAllLedgersCTime(System.currentTimeMillis());
} catch (UnavailableException ue) {
LOG.error("Got exception while trying to set checkAllLedgersCTime", ue);
Copy link
Contributor

Choose a reason for hiding this comment

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

Since there is a potential to fail this, why not cache this value. and the auditor reads the zk only if the local/cached value is -1. A minor optimization and also saves us from this kind of errors.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

didn't get you completely. Also not sure how cache would work in all the scenarios here.

Anyhow frequency of checkAllLedgers is quite low (once a week), I don't see much value in this minor optimization and not sure if it is worth complicating logic.

CheckAllLedgersFormat.Builder builder = CheckAllLedgersFormat.newBuilder();
builder.setCheckAllLedgersCTime(checkAllLedgersCTime);
if (zkc.exists(checkAllLedgersCtimeZnode, false) != null) {
zkc.setData(checkAllLedgersCtimeZnode, TextFormat.printToString(builder.build()).getBytes(UTF_8), -1);
Copy link
Contributor

Choose a reason for hiding this comment

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

What about using binary format? This is new stuff, better not to start with forward/backward compatibility problems

Copy link
Contributor Author

Choose a reason for hiding this comment

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

changing to binary format

@@ -95,3 +95,10 @@ message LockDataFormat {
message AuditorVoteFormat {
optional string bookieId = 1;
}

/**
* Debug information for checkAllLedgers
Copy link
Contributor

Choose a reason for hiding this comment

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

This is not debug actually

Copy link
Contributor Author

Choose a reason for hiding this comment

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

changing description

return Long.parseLong(new String(data, UTF_8));
} catch (KeeperException.NoNodeException ne) {
LOG.warn("checkAllLedgersCtimeZnode is not yet available");
return -1;
Copy link
Contributor

Choose a reason for hiding this comment

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

Okay @reddycharan you are right

@sijie
Copy link
Member

sijie commented Sep 25, 2018

@reddycharan : I think @eolivelli has some more comments here.

@reddycharan
Copy link
Contributor Author

@eolivelli @sijie changed it to binary format.

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.

+1 lgtm

Nice work !

@sijie
Copy link
Member

sijie commented Sep 26, 2018

this might require rebasing to master, otherwise it should be ready to go.

- store last checkAllLedgers executed time in metadataserver.
- use that value for determining the initialDelay of checkAllLedgers call
- add dedicated metric for bookieCheck
- add more log lines in Auditor
- fixing spotbugs errors and review comments
- added protobuf message for CheckAllLedgers
- code review comments
- use binary format for CheckAllLedgers protobuf message
@reddycharan
Copy link
Contributor Author

@sijie yes working on rebasing

- rebasing master
@reddycharan
Copy link
Contributor Author

rebased

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.

I have a last comment about zk ACLs.
Overall is ok

if (zkc.exists(checkAllLedgersCtimeZnode, false) != null) {
zkc.setData(checkAllLedgersCtimeZnode, checkAllLedgersFormatByteArray, -1);
} else {
zkc.create(checkAllLedgersCtimeZnode, checkAllLedgersFormatByteArray, Ids.OPEN_ACL_UNSAFE,
Copy link
Contributor

Choose a reason for hiding this comment

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

One last note: we should apply ZK ACLs according to server configuration

- get zkacls from config
@reddycharan
Copy link
Contributor Author

@eolivelli fixed zkAcls

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.

+1 great

@reddycharan
Copy link
Contributor Author

good to go ?

@sijie
Copy link
Member

sijie commented Sep 28, 2018

@reddycharan if @jvrao is good about this, it is ready to go

@sijie
Copy link
Member

sijie commented Sep 28, 2018

run bookkeeper-server bookie tests
run pr validation

@sijie
Copy link
Member

sijie commented Sep 28, 2018

@reddycharan there are some checkstyle violations. can you take a look?

[ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_pullrequest_validation/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/Auditor.java:415: Trailing whitespace [RegexpSingleline]
[ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_pullrequest_validation/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java:33: Import java.nio.charset.Charset appears after other imports that it should precede [ImportOrder]
[ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_pullrequest_validation/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java:73:8: Unused import: org.apache.bookkeeper.zookeeper.ZooKeeperClient. [UnusedImports]
[ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_pullrequest_validation/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java:75:8: Unused import: org.apache.zookeeper.ZooKeeper. [UnusedImports]
[ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_pullrequest_validation/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java:411:39: Should use uppercase 'L'. [UpperEll]
[ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_pullrequest_validation/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java:421:91: Should use uppercase 'L'. [UpperEll]
[ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_pullrequest_validation/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java:423:90: Should use uppercase 'L'. [UpperEll]

- checksum validation fixes
@reddycharan
Copy link
Contributor Author

reddycharan commented Sep 28, 2018

@sijie where did you find those failures? this was the only failure job https://travis-ci.org/apache/bookkeeper/jobs/433747876 and it didn't provide the details.

BTW, this is the correct mvn command to run to validate the fix right? - "mvn clean install -DskipTests apache-rat:check spotbugs:check" iirc, i had successful mvn run for that commit.

Anyhow pushed new commit with the fixes.

@reddycharan
Copy link
Contributor Author

@sijie issue is with CUSTOM_JDK="oraclejdk9" / continuous-integration/travis-ci/pr, not sure if there is something I can do to fix it.

@sijie
Copy link
Member

sijie commented Sep 30, 2018

run integration tests

@sijie
Copy link
Member

sijie commented Oct 1, 2018

I have tried to compile using java9, it seems that I can't reproduce the error we are seen in travis. will try to look into it later again.

@eolivelli
Copy link
Contributor

Seems a pernission error on Travis
home/travis/.m2/repository/org/apache/bookkeeper/tests/bookkeeper-server-compat420/4.7.0-SNAPSHOT/bookkeeper-server-compat420-4.7.0-SNAPSHOT.jar: Permission denied
/home/travis/.m2/repository/com/google/guava/guava/19.0/guava-19.0.jar: Permission denied

@eolivelli
Copy link
Contributor

retest this please

@reddycharan
Copy link
Contributor Author

@sijie is there anything we can do about it?

@eolivelli
Copy link
Contributor

Maybe closing and reopening this pr will trigger a fresh new run of tests

@eolivelli eolivelli closed this Oct 2, 2018
@eolivelli eolivelli reopened this Oct 2, 2018
@sijie
Copy link
Member

sijie commented Oct 2, 2018

run integration tests
rebuild java8

@eolivelli
Copy link
Contributor

I have forced travis with the button in Travis UI

@sijie
Copy link
Member

sijie commented Oct 2, 2018

@eolivelli there is a problem with jdk9 that we need to look into. and you don't need to restart the whole build, you can just restart the jdk9 one.

@sijie
Copy link
Member

sijie commented Oct 2, 2018

run integration tests

@sijie
Copy link
Member

sijie commented Oct 2, 2018

@reddycharan I am not able to figure out why travis is failing with oraclejdk9/linux. I am disabling travis oraclejdk9/linux for this PR, so you can be unblocked.

@sijie
Copy link
Member

sijie commented Oct 2, 2018

run pr validation

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

Successfully merging this pull request may close these issues.

None yet

4 participants