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-2642] Resurrect the reconfig() methods that were in ZooKeeper.java. #122

Closed
wants to merge 6 commits into from
Closed
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
108 changes: 83 additions & 25 deletions src/java/main/org/apache/zookeeper/ZooKeeper.java
Original file line number Diff line number Diff line change
Expand Up @@ -39,32 +39,10 @@
import org.apache.zookeeper.client.ZKClientConfig;
import org.apache.zookeeper.client.ZooKeeperSaslClient;
import org.apache.zookeeper.common.PathUtils;
import org.apache.zookeeper.common.StringUtils;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.proto.CheckWatchesRequest;
import org.apache.zookeeper.proto.Create2Response;
import org.apache.zookeeper.proto.CreateRequest;
import org.apache.zookeeper.proto.CreateResponse;
import org.apache.zookeeper.proto.CreateTTLRequest;
import org.apache.zookeeper.proto.DeleteRequest;
import org.apache.zookeeper.proto.ExistsRequest;
import org.apache.zookeeper.proto.GetACLRequest;
import org.apache.zookeeper.proto.GetACLResponse;
import org.apache.zookeeper.proto.GetChildren2Request;
import org.apache.zookeeper.proto.GetChildren2Response;
import org.apache.zookeeper.proto.GetChildrenRequest;
import org.apache.zookeeper.proto.GetChildrenResponse;
import org.apache.zookeeper.proto.GetDataRequest;
import org.apache.zookeeper.proto.GetDataResponse;
import org.apache.zookeeper.proto.RemoveWatchesRequest;
import org.apache.zookeeper.proto.ReplyHeader;
import org.apache.zookeeper.proto.RequestHeader;
import org.apache.zookeeper.proto.SetACLRequest;
import org.apache.zookeeper.proto.SetACLResponse;
import org.apache.zookeeper.proto.SetDataRequest;
import org.apache.zookeeper.proto.SetDataResponse;
import org.apache.zookeeper.proto.SyncRequest;
import org.apache.zookeeper.proto.SyncResponse;
import org.apache.zookeeper.proto.*;
Copy link
Contributor

Choose a reason for hiding this comment

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

If I remember well, Patrick Hunt once told me during a code review that we should use explicit imports instead of wildcards. Right, @phunt ?

Copy link
Member Author

Choose a reason for hiding this comment

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

fixed

import org.apache.zookeeper.server.DataTree;
import org.apache.zookeeper.server.EphemeralType;
import org.slf4j.Logger;
Expand Down Expand Up @@ -2175,7 +2153,42 @@ public byte[] getConfig(boolean watch, Stat stat)
public void getConfig(boolean watch, DataCallback cb, Object ctx) {
getConfig(watch ? watchManager.defaultWatcher : null, cb, ctx);
}


/**
* @deprecated instead use the reconfigure() methods instead in {@link org.apache.zookeeper.admin.ZooKeeperAdmin}
*/
@Deprecated
public byte[] reconfig(String joiningServers, String leavingServers, String newMembers, long fromConfig, Stat stat) throws KeeperException, InterruptedException {
return internalReconfig(joiningServers, leavingServers, newMembers, fromConfig, stat);
}

/**
* @deprecated instead use the reconfigure() methods instead in {@link org.apache.zookeeper.admin.ZooKeeperAdmin}
*/
@Deprecated
public byte[] reconfig(List<String> joiningServers, List<String> leavingServers, List<String> newMembers, long fromConfig, Stat stat) throws KeeperException, InterruptedException {
return internalReconfig(joiningServers, leavingServers, newMembers, fromConfig, stat);
}

/**
* @deprecated instead use the reconfigure() methods instead in {@link org.apache.zookeeper.admin.ZooKeeperAdmin}
*/
@Deprecated
public void reconfig(String joiningServers, String leavingServers,
String newMembers, long fromConfig, DataCallback cb, Object ctx) {
internalReconfig(joiningServers, leavingServers, newMembers, fromConfig, cb, ctx);
}

/**
* @deprecated instead use the reconfigure() methods instead in {@link org.apache.zookeeper.admin.ZooKeeperAdmin}
*/
@Deprecated
public void reconfig(List<String> joiningServers,
List<String> leavingServers, List<String> newMembers, long fromConfig,
DataCallback cb, Object ctx) {
internalReconfig(joiningServers, leavingServers, newMembers, fromConfig, cb, ctx);
}

/**
* Set the data for the node of the given path if such a node exists and the
* given version matches the version of the node (if the given version is
Expand Down Expand Up @@ -2910,4 +2923,49 @@ private ClientCnxnSocket getClientCnxnSocket() throws IOException {
throw ioe;
}
}

protected byte[] internalReconfig(String joiningServers, String leavingServers, String newMembers, long fromConfig, Stat stat) throws KeeperException, InterruptedException
{
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: inconsistent brace style for method names from line onwards.

Copy link
Member Author

Choose a reason for hiding this comment

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

fixed

RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.reconfig);
ReconfigRequest request = new ReconfigRequest(joiningServers, leavingServers, newMembers, fromConfig);
GetDataResponse response = new GetDataResponse();
ReplyHeader r = cnxn.submitRequest(h, request, response, null);
if (r.getErr() != 0) {
throw KeeperException.create(KeeperException.Code.get(r.getErr()), "");
}
if (stat != null) {
DataTree.copyStat(response.getStat(), stat);
}
return response.getData();
}

protected byte[] internalReconfig(List<String> joiningServers, List<String> leavingServers, List<String> newMembers, long fromConfig, Stat stat) throws KeeperException, InterruptedException
{
return internalReconfig(StringUtils.joinStrings(joiningServers, ","),
StringUtils.joinStrings(leavingServers, ","),
StringUtils.joinStrings(newMembers, ","),
fromConfig, stat);
}

protected void internalReconfig(String joiningServers, String leavingServers,
String newMembers, long fromConfig, DataCallback cb, Object ctx)
{
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.reconfig);
ReconfigRequest request = new ReconfigRequest(joiningServers, leavingServers, newMembers, fromConfig);
GetDataResponse response = new GetDataResponse();
cnxn.queuePacket(h, new ReplyHeader(), request, response, cb,
ZooDefs.CONFIG_NODE, ZooDefs.CONFIG_NODE, ctx, null);
}

protected void internalReconfig(List<String> joiningServers,
List<String> leavingServers, List<String> newMembers, long fromConfig,
DataCallback cb, Object ctx)
{
internalReconfig(StringUtils.joinStrings(joiningServers, ","),
StringUtils.joinStrings(leavingServers, ","),
StringUtils.joinStrings(newMembers, ","),
fromConfig, cb, ctx);
}
}
63 changes: 17 additions & 46 deletions src/java/main/org/apache/zookeeper/admin/ZooKeeperAdmin.java
Original file line number Diff line number Diff line change
Expand Up @@ -22,18 +22,11 @@
import java.util.List;

import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.AsyncCallback.DataCallback;
import org.apache.zookeeper.client.ZKClientConfig;
import org.apache.zookeeper.common.StringUtils;
import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.proto.GetDataResponse;
import org.apache.zookeeper.proto.ReconfigRequest;
import org.apache.zookeeper.proto.ReplyHeader;
import org.apache.zookeeper.proto.RequestHeader;
import org.apache.zookeeper.server.DataTree;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -172,66 +165,44 @@ public ZooKeeperAdmin(String connectString, int sessionTimeout, Watcher watcher,
* @throws InterruptedException If the server transaction is interrupted.
* @throws KeeperException If the server signals an error with a non-zero error code.
*/
public byte[] reconfig(String joiningServers, String leavingServers,
String newMembers, long fromConfig, Stat stat) throws KeeperException, InterruptedException {
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.reconfig);
ReconfigRequest request = new ReconfigRequest(joiningServers, leavingServers, newMembers, fromConfig);
GetDataResponse response = new GetDataResponse();
ReplyHeader r = cnxn.submitRequest(h, request, response, null);
if (r.getErr() != 0) {
throw KeeperException.create(KeeperException.Code.get(r.getErr()), "");
}
if (stat != null) {
DataTree.copyStat(response.getStat(), stat);
}
return response.getData();
public byte[] reconfigure(String joiningServers, String leavingServers,
String newMembers, long fromConfig, Stat stat) throws KeeperException, InterruptedException {
return internalReconfig(joiningServers, leavingServers, newMembers, fromConfig, stat);
}

/**
* Convenience wrapper around reconfig that takes Lists of strings instead of comma-separated servers.
*
* @see #reconfig
* @see #reconfigure
*
*/
public byte[] reconfig(List<String> joiningServers, List<String> leavingServers,
List<String> newMembers, long fromConfig,
Stat stat) throws KeeperException, InterruptedException {
return reconfig(StringUtils.joinStrings(joiningServers, ","),
StringUtils.joinStrings(leavingServers, ","),
StringUtils.joinStrings(newMembers, ","),
fromConfig, stat);
public byte[] reconfigure(List<String> joiningServers, List<String> leavingServers,
List<String> newMembers, long fromConfig,
Stat stat) throws KeeperException, InterruptedException {
return internalReconfig(joiningServers, leavingServers, newMembers, fromConfig, stat);
}

/**
* The Asynchronous version of reconfig.
*
* @see #reconfig
* @see #reconfigure
*
**/
public void reconfig(String joiningServers, String leavingServers,
String newMembers, long fromConfig, DataCallback cb, Object ctx) {
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.reconfig);
ReconfigRequest request = new ReconfigRequest(joiningServers, leavingServers, newMembers, fromConfig);
GetDataResponse response = new GetDataResponse();
cnxn.queuePacket(h, new ReplyHeader(), request, response, cb,
ZooDefs.CONFIG_NODE, ZooDefs.CONFIG_NODE, ctx, null);
public void reconfigure(String joiningServers, String leavingServers,
String newMembers, long fromConfig, DataCallback cb, Object ctx) {
internalReconfig(joiningServers, leavingServers, newMembers, fromConfig, cb, ctx);
}

/**
* Convenience wrapper around asynchronous reconfig that takes Lists of strings instead of comma-separated servers.
*
* @see #reconfig
* @see #reconfigure
*
*/
public void reconfig(List<String> joiningServers,
List<String> leavingServers, List<String> newMembers, long fromConfig,
DataCallback cb, Object ctx) {
reconfig(StringUtils.joinStrings(joiningServers, ","),
StringUtils.joinStrings(leavingServers, ","),
StringUtils.joinStrings(newMembers, ","),
fromConfig, cb, ctx);
public void reconfigure(List<String> joiningServers,
List<String> leavingServers, List<String> newMembers, long fromConfig,
DataCallback cb, Object ctx) {
internalReconfig(joiningServers, leavingServers, newMembers, fromConfig, cb, ctx);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -154,7 +154,7 @@ public boolean exec() throws CliException {
return false;
}

byte[] curConfig = ((ZooKeeperAdmin)zk).reconfig(joining,
byte[] curConfig = ((ZooKeeperAdmin)zk).reconfigure(joining,
leaving, members, version, stat);
out.println("Committed new configuration:\n" + new String(curConfig));

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -149,7 +149,7 @@ public TestQPMain getTestQPMain() {
// Leader.NEWLEADER
while (true) {
if (qp.isNewLeaderMessage()) {
preReconfigClient.reconfig(serverConfig[joinerId], null, null, -1, null, null);
preReconfigClient.reconfigure(serverConfig[joinerId], null, null, -1, null, null);
break;
} else {
// sleep for 10 millisecond and then again check
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,7 +91,7 @@ public void testIncrementalReconfigInvokedOnHiearchicalQS() throws Exception {
List<String> leavingServers = new ArrayList<String>();
leavingServers.add("3");
try {
zkAdminArr[1].reconfig(null, leavingServers, null, -1, null);
zkAdminArr[1].reconfigure(null, leavingServers, null, -1, null);
Assert.fail("Reconfig should have failed since the current config isn't Majority QS");
} catch (KeeperException.BadArgumentsException e) {
// We expect this to happen.
Expand Down Expand Up @@ -121,7 +121,7 @@ public void testTooFewRemainingPariticipants() throws Exception {
leavingServers.add("2");
leavingServers.add("3");
try {
zkAdminArr[1].reconfig(null, leavingServers, null, -1, null);
zkAdminArr[1].reconfigure(null, leavingServers, null, -1, null);
Assert.fail("Reconfig should have failed since the current config version is not 8");
} catch (KeeperException.BadArgumentsException e) {
// We expect this to happen.
Expand All @@ -147,7 +147,7 @@ public void testReconfigVersionConditionFails() throws Exception {
List<String> leavingServers = new ArrayList<String>();
leavingServers.add("3");
try {
zkAdminArr[1].reconfig(null, leavingServers, null, 8, null);
zkAdminArr[1].reconfigure(null, leavingServers, null, 8, null);
Assert.fail("Reconfig should have failed since the current config version is not 8");
} catch (KeeperException.BadVersionException e) {
// We expect this to happen.
Expand Down Expand Up @@ -182,7 +182,7 @@ public void testLeaderTimesoutOnNewQuorum() throws Exception {
// We try to remove server 3, which requires a quorum of {1,2,3}
// (we have that) and of {1,2}, but 2 is down so we won't get a
// quorum of new config ACKs.
zkAdminArr[1].reconfig(null, leavingServers, null, -1, null);
zkAdminArr[1].reconfigure(null, leavingServers, null, -1, null);
Assert.fail("Reconfig should have failed since we don't have quorum of new config");
} catch (KeeperException.ConnectionLossException e) {
// We expect leader to lose quorum of proposed config and time out
Expand Down Expand Up @@ -255,7 +255,7 @@ public void testObserverToParticipantConversionFails() throws Exception {
}

try {
zkAdmin[1].reconfig("", "", nextQuorumCfgSection, -1, new Stat());
zkAdmin[1].reconfigure("", "", nextQuorumCfgSection, -1, new Stat());
Assert.fail("Reconfig should have failed with NewConfigNoQuorum");
} catch (NewConfigNoQuorum e) {
// This is expected case since server 0 is down and 3 can't vote
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,7 @@ public void startSingleServerTest() throws Exception {
reconfigServers.clear();
reconfigServers.add(Integer.toString(follower2));
try {
zkAdminHandles[follower2].reconfig(null, reconfigServers, null, -1, new Stat());
zkAdminHandles[follower2].reconfigure(null, reconfigServers, null, -1, new Stat());
Assert.fail("reconfig completed successfully even though there is no quorum up in new config!");
} catch (KeeperException.BadArgumentsException e) {
// This is expected.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -214,7 +214,7 @@ private boolean reconfigPort() throws KeeperException, InterruptedException {
+ qu.getPeer(followerId).peer.getQuorumAddress().getPort() /*quorum port*/
+ ":" + qu.getPeer(followerId).peer.getElectionAddress().getPort() /*election port*/
+ ":participant;localhost:" + PortAssignment.unique()/* new client port */);
zkAdmin.reconfig(joiningServers, null, null, -1, new Stat());
zkAdmin.reconfigure(joiningServers, null, null, -1, new Stat());
return true;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,7 @@ private boolean reconfigPort() throws KeeperException, InterruptedException {
+ qu.getPeer(followerId).peer.getQuorumAddress().getPort() /*quorum port*/
+ ":" + qu.getPeer(followerId).peer.getElectionAddress().getPort() /*election port*/
+ ":participant;localhost:" + PortAssignment.unique()/* new client port */);
zkAdmin.reconfig(joiningServers, null, null, -1, new Stat());
zkAdmin.reconfigure(joiningServers, null, null, -1, new Stat());
return true;
}
}
Expand Down
4 changes: 2 additions & 2 deletions src/java/test/org/apache/zookeeper/test/ReconfigTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ public static String reconfig(ZooKeeperAdmin zkAdmin, List<String> joiningServer
byte[] config = null;
for (int j = 0; j < 30; j++) {
try {
config = zkAdmin.reconfig(joiningServers, leavingServers,
config = zkAdmin.reconfigure(joiningServers, leavingServers,
newMembers, fromConfig, new Stat());
break;
} catch (KeeperException.ConnectionLossException e) {
Expand Down Expand Up @@ -481,7 +481,7 @@ public void testRemoveOneAsynchronous() throws Exception {

LinkedList<Integer> results = new LinkedList<Integer>();

zkAdminArr[1].reconfig(null, leavingServers, null, -1, this, results);
zkAdminArr[1].reconfigure(null, leavingServers, null, -1, this, results);

synchronized (results) {
while (results.size() < 1) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -151,7 +151,7 @@ public void testStandaloneReconfigFails() throws Exception {
// generate some transactions that will get logged
try {
zkAdmin.addAuthInfo("digest", "super:test".getBytes());
zkAdmin.reconfig(joiners, null, null, -1, new Stat());
zkAdmin.reconfigure(joiners, null, null, -1, new Stat());
Assert.fail("Reconfiguration in standalone should trigger " +
"UnimplementedException");
} catch (KeeperException.UnimplementedException ex) {
Expand Down