Author: mahadev
Date: Fri Jan 30 23:01:33 2009
New Revision: 739446
URL: http://svn.apache.org/viewvc?rev=739446&view=rev
Log:
ZOOKEEPER-16. Need to do path validation. (pat, mahadev)
Modified:
hadoop/zookeeper/trunk/CHANGES.txt
hadoop/zookeeper/trunk/src/c/src/zookeeper.c
hadoop/zookeeper/trunk/src/c/tests/TestClient.cc
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java
Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=739446&r1=739445&r2=739446&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Fri Jan 30 23:01:33 2009
@@ -78,7 +78,9 @@
ZOOKEEPER-275. Bug in FastLeaderElection. (flavio via mahadev)
ZOOKEEPER-272. getchildren can fail for large number of children. (mahadev)
-
+
+ ZOOKEEPER-16. Need to do path validation. (pat, mahadev)
+
IMPROVEMENTS:
ZOOKEEPER-64. Log system env information when initializing server and
Modified: hadoop/zookeeper/trunk/src/c/src/zookeeper.c
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/src/zookeeper.c?rev=739446&r1=739445&r2=739446&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/c/src/zookeeper.c (original)
+++ hadoop/zookeeper/trunk/src/c/src/zookeeper.c Fri Jan 30 23:01:33 2009
@@ -1863,6 +1863,45 @@
return rc;
}
+static int isValidPath(const char* path) {
+ if (path == 0)
+ return 0;
+ const int len = strlen(path);
+ if (len == 0)
+ return 0;
+ if (path[0] != '/')
+ return 0;
+ if (len == 1) // done checking - it's the root
+ return 1;
+ if (path[len - 1] == '/')
+ return 0;
+
+ char lastc = '/';
+ char c;
+ int i = 1;
+ for (; i < len; lastc = path[i], i++) {
+ c = path[i];
+
+ if (c == 0) {
+ return 0;
+ } else if (c == '/' && lastc == '/') {
+ return 0;
+ } else if (c == '.' && lastc == '.') {
+ if (path[i-2] == '/' && ((i + 1 == len) || path[i+1] == '/')) {
+ return 0;
+ }
+ } else if (c == '.') {
+ if ((path[i-1] == '/') && ((i + 1 == len) || path[i+1] == '/')) {
+ return 0;
+ }
+ } else if (c > 0x00 && c < 0x1f) {
+ return 0;
+ }
+ }
+
+ return 1;
+}
+
int zoo_aget(zhandle_t *zh, const char *path, int watch, data_completion_t dc,
const void *data)
{
@@ -1878,7 +1917,7 @@
struct GetDataRequest req = { (char*)path, watcher!=0 };
int rc;
- if (zh==0 || path==0)
+ if (zh==0 || !isValidPath(path))
return ZBADARGUMENTS;
if (is_unrecoverable(zh))
return ZINVALIDSTATE;
@@ -1910,7 +1949,7 @@
struct SetDataRequest req;
int rc;
- if (zh==0 || path==0)
+ if (zh==0 || !isValidPath(path))
return ZBADARGUMENTS;
if (is_unrecoverable(zh))
return ZINVALIDSTATE;
@@ -1945,7 +1984,7 @@
struct CreateRequest req;
int rc;
- if (zh==0 || path==0)
+ if (zh==0 || !isValidPath(path))
return ZBADARGUMENTS;
if (is_unrecoverable(zh))
return ZINVALIDSTATE;
@@ -1985,7 +2024,7 @@
struct DeleteRequest req;
int rc;
- if (zh==0 || path==0)
+ if (zh==0 || !isValidPath(path))
return ZBADARGUMENTS;
if (is_unrecoverable(zh))
return ZINVALIDSTATE;
@@ -2024,7 +2063,7 @@
struct ExistsRequest req = {(char*)path, watcher!=0 };
int rc;
- if (zh==0 || path==0)
+ if (zh==0 || !isValidPath(path))
return ZBADARGUMENTS;
if (is_unrecoverable(zh))
return ZINVALIDSTATE;
@@ -2063,7 +2102,7 @@
struct GetChildrenRequest req={(char*)path, watcher!=0 };
int rc;
- if (zh==0 || path==0)
+ if (zh==0 || !isValidPath(path))
return ZBADARGUMENTS;
if (is_unrecoverable(zh))
return ZINVALIDSTATE;
@@ -2094,7 +2133,7 @@
struct SyncRequest req;
int rc;
- if (zh==0 || path==0)
+ if (zh==0 || !isValidPath(path))
return ZBADARGUMENTS;
if (is_unrecoverable(zh))
return ZINVALIDSTATE;
@@ -2126,7 +2165,7 @@
struct GetACLRequest req;
int rc;
- if (zh==0 || path==0)
+ if (zh==0 || !isValidPath(path))
return ZBADARGUMENTS;
if (is_unrecoverable(zh))
return ZINVALIDSTATE;
@@ -2157,7 +2196,7 @@
struct SetACLRequest req;
int rc;
- if (zh==0 || path==0)
+ if (zh==0 || !isValidPath(path))
return ZBADARGUMENTS;
if (is_unrecoverable(zh))
return ZINVALIDSTATE;
Modified: hadoop/zookeeper/trunk/src/c/tests/TestClient.cc
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/c/tests/TestClient.cc?rev=739446&r1=739445&r2=739446&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/c/tests/TestClient.cc (original)
+++ hadoop/zookeeper/trunk/src/c/tests/TestClient.cc Fri Jan 30 23:01:33 2009
@@ -157,6 +157,7 @@
CPPUNIT_TEST_SUITE(Zookeeper_simpleSystem);
CPPUNIT_TEST(testAsyncWatcherAutoReset);
#ifdef THREADED
+ CPPUNIT_TEST(testPathValidation);
CPPUNIT_TEST(testPing);
CPPUNIT_TEST(testWatcherAutoResetWithGlobal);
CPPUNIT_TEST(testWatcherAutoResetWithLocal);
@@ -285,6 +286,57 @@
}
}
+ static void verifyCreateFails(const char *path, zhandle_t *zk) {
+ CPPUNIT_ASSERT_EQUAL((int)ZBADARGUMENTS, zoo_create(zk,
+ path, "", 0, &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0));
+ }
+
+ static void verifyCreateOk(const char *path, zhandle_t *zk) {
+ CPPUNIT_ASSERT_EQUAL((int)ZOK, zoo_create(zk,
+ path, "", 0, &ZOO_OPEN_ACL_UNSAFE, 0, 0, 0));
+ }
+
+ void testPathValidation() {
+ watchctx_t ctx;
+ zhandle_t *zk = createClient(&ctx);
+ CPPUNIT_ASSERT(zk);
+
+ verifyCreateFails(0, zk);
+ verifyCreateFails("", zk);
+ verifyCreateFails("//", zk);
+ verifyCreateFails("///", zk);
+ verifyCreateFails("////", zk);
+ verifyCreateFails("/.", zk);
+ verifyCreateFails("/..", zk);
+ verifyCreateFails("/./", zk);
+ verifyCreateFails("/../", zk);
+ verifyCreateFails("/foo/./", zk);
+ verifyCreateFails("/foo/../", zk);
+ verifyCreateFails("/foo/.", zk);
+ verifyCreateFails("/foo/..", zk);
+ verifyCreateFails("/./.", zk);
+ verifyCreateFails("/../..", zk);
+ verifyCreateFails("/foo/bar/", zk);
+ verifyCreateFails("/foo//bar", zk);
+ verifyCreateFails("/foo/bar//", zk);
+
+ verifyCreateFails("foo", zk);
+ verifyCreateFails("a", zk);
+
+ verifyCreateOk("/.foo", zk);
+ verifyCreateOk("/.f.", zk);
+ verifyCreateOk("/..f", zk);
+ verifyCreateOk("/..f..", zk);
+ verifyCreateOk("/f.c", zk);
+ verifyCreateOk("/f", zk);
+ verifyCreateOk("/f/.f", zk);
+ verifyCreateOk("/f/f.", zk);
+ verifyCreateOk("/f/..f", zk);
+ verifyCreateOk("/f/f..", zk);
+ verifyCreateOk("/f/.f/f", zk);
+ verifyCreateOk("/f/f./f", zk);
+ }
+
void testAsyncWatcherAutoReset()
{
watchctx_t ctx;
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java?rev=739446&r1=739445&r2=739446&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ZooKeeper.java Fri Jan 30 23:01:33
2009
@@ -97,10 +97,10 @@
*/
public class ZooKeeper {
private static final Logger LOG;
-
+
static {
LOG = Logger.getLogger(ZooKeeper.class);
-
+
Environment.logEnv("Client environment:", LOG);
}
@@ -134,7 +134,7 @@
new HashMap<String, Set<Watcher>>();
private final Map<String, Set<Watcher>> childWatches =
new HashMap<String, Set<Watcher>>();
-
+
private volatile Watcher defaultWatcher;
final private void addTo(Set<Watcher> from, Set<Watcher> to) {
@@ -142,7 +142,7 @@
to.addAll(from);
}
}
-
+
/* (non-Javadoc)
* @see org.apache.zookeeper.ClientWatchManager#materialize(Event.KeeperState, Event.EventType,
java.lang.String)
*/
@@ -219,7 +219,7 @@
return result;
}
}
-
+
/**
* Register a watcher for a particular path.
*/
@@ -233,7 +233,7 @@
}
abstract protected Map<String, Set<Watcher>> getWatches(int rc);
-
+
/**
* Register the watcher with the set of watches on path.
* @param rc the result code of the operation that attempted to
@@ -270,12 +270,12 @@
public ExistsWatchRegistration(Watcher watcher, String path) {
super(watcher, path);
}
-
+
@Override
protected Map<String, Set<Watcher>> getWatches(int rc) {
return rc == 0 ? watchManager.dataWatches : watchManager.existWatches;
}
-
+
@Override
protected boolean shouldAddWatch(int rc) {
return rc == 0 || rc == KeeperException.Code.NONODE.intValue();
@@ -292,7 +292,7 @@
return watchManager.dataWatches;
}
}
-
+
class ChildWatchRegistration extends WatchRegistration {
public ChildWatchRegistration(Watcher watcher, String path) {
super(watcher, path);
@@ -493,12 +493,15 @@
* @return the actual path of the created node
* @throws KeeperException if the server returns a non-zero error code
* @throws org.apache.zookeeper.KeeperException.InvalidACLException if the ACL is invalid
- * @throws InterruptedException if the transaction is interrrupted
+ * @throws InterruptedException if the transaction is interrupted
+ * @throws IllegalArgumentException if an invalid path is specified
*/
public String create(String path, byte data[], List<ACL> acl,
CreateMode createMode)
throws KeeperException, InterruptedException
{
+ validatePath(path);
+
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.create);
CreateRequest request = new CreateRequest();
@@ -519,6 +522,72 @@
}
/**
+ * Validate the provided znode path string
+ * @param path znode path string
+ * @throws IllegalArgumentException if the path is invalid
+ */
+ public static void validatePath(String path) throws IllegalArgumentException {
+ if (path == null) {
+ throw new IllegalArgumentException("Path cannot be null");
+ }
+ if (path.length() == 0) {
+ throw new IllegalArgumentException("Path length must be > 0");
+ }
+ if (path.charAt(0) != '/') {
+ throw new IllegalArgumentException(
+ "Path must start with / character");
+ }
+ if (path.length() == 1) { // done checking - it's the root
+ return;
+ }
+ if (path.charAt(path.length() - 1) == '/') {
+ throw new IllegalArgumentException(
+ "Path must not end with / character");
+ }
+
+ String reason = null;
+ char lastc = '/';
+ char chars[] = path.toCharArray();
+ char c;
+ for (int i = 1; i < chars.length; lastc = chars[i], i++) {
+ c = chars[i];
+
+ if (c == 0) {
+ reason = "null character not allowed @" + i;
+ break;
+ } else if (c == '/' && lastc == '/') {
+ reason = "empty node name specified @" + i;
+ break;
+ } else if (c == '.' && lastc == '.') {
+ if (chars[i-2] == '/' &&
+ ((i + 1 == chars.length)
+ || chars[i+1] == '/')) {
+ reason = "relative paths not allowed @" + i;
+ break;
+ }
+ } else if (c == '.') {
+ if (chars[i-1] == '/' &&
+ ((i + 1 == chars.length)
+ || chars[i+1] == '/')) {
+ reason = "relative paths not allowed @" + i;
+ break;
+ }
+ } else if (c > '\u0000' && c < '\u001f'
+ || c > '\u007f' && c < '\u009F'
+ || c > '\ud800' && c < '\uf8ff'
+ || c > '\ufff0' && c < '\uffff') {
+ reason = "invalid charater @" + i;
+ break;
+ }
+ }
+
+ if (reason != null) {
+ throw new IllegalArgumentException(
+ "Invalid path string \"" + path + "\" caused by " + reason);
+ }
+ }
+
+ /**
* The Asynchronous version of create. The request doesn't actually until
* the asynchronous callback is called.
*
@@ -528,6 +597,8 @@
public void create(String path, byte data[], List<ACL> acl,
CreateMode createMode, StringCallback cb, Object ctx)
{
+ validatePath(path);
+
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.create);
CreateRequest request = new CreateRequest();
@@ -564,9 +635,12 @@
* the expected node version.
* @throws InterruptedException IF the server transaction is interrupted
* @throws KeeperException If the server signals an error with a non-zero return code.
+ * @throws IllegalArgumentException if an invalid path is specified
*/
public void delete(String path, int version) throws
InterruptedException, KeeperException {
+ validatePath(path);
+
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.delete);
DeleteRequest request = new DeleteRequest();
@@ -585,6 +659,8 @@
* @see #delete(String, int)
*/
public void delete(String path, int version, VoidCallback cb, Object ctx) {
+ validatePath(path);
+
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.delete);
DeleteRequest request = new DeleteRequest();
@@ -608,10 +684,13 @@
* node exists.
* @throws KeeperException If the server signals an error
* @throws InterruptedException If the server transaction is interrupted.
+ * @throws IllegalArgumentException if an invalid path is specified
*/
public Stat exists(String path, Watcher watcher) throws KeeperException,
InterruptedException
{
+ validatePath(path);
+
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.exists);
ExistsRequest request = new ExistsRequest();
@@ -666,6 +745,8 @@
public void exists(String path, Watcher watcher, StatCallback cb,
Object ctx)
{
+ validatePath(path);
+
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.exists);
ExistsRequest request = new ExistsRequest();
@@ -707,9 +788,12 @@
* @return the data of the node
* @throws KeeperException If the server signals an error with a non-zero error code
* @throws InterruptedException If the server transaction is interrupted.
+ * @throws IllegalArgumentException if an invalid path is specified
*/
public byte[] getData(String path, Watcher watcher, Stat stat)
throws KeeperException, InterruptedException {
+ validatePath(path);
+
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.getData);
GetDataRequest request = new GetDataRequest();
@@ -760,6 +844,8 @@
* @see #getData(String, Watcher, Stat)
*/
public void getData(String path, Watcher watcher, DataCallback cb, Object ctx) {
+ validatePath(path);
+
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.getData);
GetDataRequest request = new GetDataRequest();
@@ -810,9 +896,12 @@
* @return the state of the node
* @throws InterruptedException If the server transaction is interrupted.
* @throws KeeperException If the server signals an error with a non-zero error code.
+ * @throws IllegalArgumentException if an invalid path is specified
*/
public Stat setData(String path, byte data[], int version)
throws KeeperException, InterruptedException {
+ validatePath(path);
+
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.setData);
SetDataRequest request = new SetDataRequest();
@@ -835,6 +924,8 @@
*/
public void setData(String path, byte data[], int version, StatCallback cb,
Object ctx) {
+ validatePath(path);
+
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.setData);
SetDataRequest request = new SetDataRequest();
@@ -860,9 +951,12 @@
* @return the ACL array of the given node.
* @throws InterruptedException If the server transaction is interrupted.
* @throws KeeperException If the server signals an error with a non-zero error code.
+ * @throws IllegalArgumentException if an invalid path is specified
*/
public List<ACL> getACL(String path, Stat stat)
throws KeeperException, InterruptedException {
+ validatePath(path);
+
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.getACL);
GetACLRequest request = new GetACLRequest();
@@ -883,6 +977,8 @@
* @see #getACL(String, Stat)
*/
public void getACL(String path, Stat stat, ACLCallback cb, Object ctx) {
+ validatePath(path);
+
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.getACL);
GetACLRequest request = new GetACLRequest();
@@ -911,9 +1007,12 @@
* @throws InterruptedException If the server transaction is interrupted.
* @throws KeeperException If the server signals an error with a non-zero error code.
* @throws org.apache.zookeeper.KeeperException.InvalidACLException If the acl is invalide.
+ * @throws IllegalArgumentException if an invalid path is specified
*/
public Stat setACL(String path, List<ACL> acl, int version)
throws KeeperException, InterruptedException {
+ validatePath(path);
+
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.setACL);
SetACLRequest request = new SetACLRequest();
@@ -939,6 +1038,8 @@
*/
public void setACL(String path, List<ACL> acl, int version,
StatCallback cb, Object ctx) {
+ validatePath(path);
+
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.setACL);
SetACLRequest request = new SetACLRequest();
@@ -970,9 +1071,12 @@
* @return an unordered array of children of the node with the given path
* @throws InterruptedException If the server transaction is interrupted.
* @throws KeeperException If the server signals an error with a non-zero error code.
+ * @throws IllegalArgumentException if an invalid path is specified
*/
public List<String> getChildren(String path, Watcher watcher)
throws KeeperException, InterruptedException {
+ validatePath(path);
+
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.getChildren);
GetChildrenRequest request = new GetChildrenRequest();
@@ -1023,6 +1127,8 @@
*/
public void getChildren(String path, Watcher watcher, ChildrenCallback cb,
Object ctx) {
+ validatePath(path);
+
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.getChildren);
GetChildrenRequest request = new GetChildrenRequest();
@@ -1050,8 +1156,14 @@
/**
* Asynchronous sync. Flushes channel between process and leader.
+ * @param path
+ * @param cb a handler for the callback
+ * @param ctx context to be provided to the callback
+ * @throws IllegalArgumentException if an invalid path is specified
*/
public void sync(String path, VoidCallback cb, Object ctx){
+ validatePath(path);
+
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.sync);
SyncRequest request = new SyncRequest();
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java?rev=739446&r1=739445&r2=739446&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
(original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
Fri Jan 30 23:01:33 2009
@@ -30,6 +30,7 @@
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.ZooDefs.OpCode;
import org.apache.zookeeper.data.ACL;
@@ -67,7 +68,13 @@
LOG.info("zookeeper.skipACL==\"yes\", ACL checks will be skipped");
}
}
-
+
+ /**
+ * this is only for testing purposes.
+ * should never be useed otherwise
+ */
+ public static boolean failCreate = false;
+
LinkedBlockingQueue<Request> submittedRequests = new LinkedBlockingQueue<Request>();
RequestProcessor nextProcessor;
@@ -189,7 +196,16 @@
createRequest);
String path = createRequest.getPath();
int lastSlash = path.lastIndexOf('/');
- if (lastSlash == -1 || path.indexOf('\0') != -1) {
+ if (lastSlash == -1 || path.indexOf('\0') != -1 || failCreate) {
+ LOG.warn("Invalid path " + path + " with session " +
+ Long.toHexString(request.sessionId));
+ throw new KeeperException.BadArgumentsException();
+ }
+ try {
+ ZooKeeper.validatePath(path);
+ } catch(IllegalArgumentException ie) {
+ LOG.warn("Invalid path " + path + " with session " +
+ Long.toHexString(request.sessionId));
throw new KeeperException.BadArgumentsException();
}
if (!fixupACL(request.authInfo, createRequest.getAcl())) {
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java?rev=739446&r1=739445&r2=739446&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java Fri Jan
30 23:01:33 2009
@@ -40,6 +40,7 @@
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Id;
import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.PrepRequestProcessor;
import org.junit.Test;
public class ClientTest extends ClientBase {
@@ -387,13 +388,7 @@
assertEquals(EventType.NodeDeleted, event.getType());
assertEquals(KeeperState.SyncConnected, event.getState());
}
- zk.create("/good\u0001path", "".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
- //try {
- // zk.create("/bad\u0000path", "".getBytes(), null, CreateMode.PERSISTENT);
- // fail("created an invalid path");
- //} catch(KeeperException e) {
- // assertEquals(KeeperException.Code.BadArguments, e.code());
- //}
+ zk.create("/good\u0040path", "".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
zk.create("/duplicate", "".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
try {
@@ -455,6 +450,67 @@
}
}
+ private void verifyCreateFails(String path, ZooKeeper zk) throws Exception {
+ try {
+ zk.create(path, null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ } catch (IllegalArgumentException e) {
+ // this is good
+ return;
+ }
+ fail("bad path \"" + path + "\" not caught");
+ }
+
+ // Test that the path string is validated
+ @Test
+ public void testPathValidation() throws Exception {
+ ZooKeeper zk = createClient();
+
+ verifyCreateFails(null, zk);
+ verifyCreateFails("", zk);
+ verifyCreateFails("//", zk);
+ verifyCreateFails("///", zk);
+ verifyCreateFails("////", zk);
+ verifyCreateFails("/.", zk);
+ verifyCreateFails("/..", zk);
+ verifyCreateFails("/./", zk);
+ verifyCreateFails("/../", zk);
+ verifyCreateFails("/foo/./", zk);
+ verifyCreateFails("/foo/../", zk);
+ verifyCreateFails("/foo/.", zk);
+ verifyCreateFails("/foo/..", zk);
+ verifyCreateFails("/./.", zk);
+ verifyCreateFails("/../..", zk);
+ verifyCreateFails("/\u0001foo", zk);
+ verifyCreateFails("/foo/bar/", zk);
+ verifyCreateFails("/foo//bar", zk);
+ verifyCreateFails("/foo/bar//", zk);
+
+ verifyCreateFails("foo", zk);
+ verifyCreateFails("a", zk);
+ //check for the code path that throws at server
+ PrepRequestProcessor.failCreate = true;
+ try {
+ zk.create("/m", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ assertTrue(false);
+ } catch(KeeperException.BadArgumentsException be) {
+ // catch this.
+ }
+ PrepRequestProcessor.failCreate = false;
+ zk.create("/.foo", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ zk.create("/.f.", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ zk.create("/..f", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ zk.create("/..f..", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ zk.create("/f.c", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ zk.create("/f\u0040f", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ zk.create("/f", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ zk.create("/f/.f", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ zk.create("/f/f.", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ zk.create("/f/..f", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ zk.create("/f/f..", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ zk.create("/f/.f/f", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ zk.create("/f/f./f", null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ }
+
// private void notestConnections()
// throws IOException, InterruptedException, KeeperException
// {
|