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-4706: Support 64-bit sequential nodes using zxid #2059

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all 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
Original file line number Diff line number Diff line change
Expand Up @@ -215,6 +215,12 @@ counter used to store the next sequence number is a signed int
overflow when incremented beyond 2147483647 (resulting in a
name "<path>-2147483648").

**Added in 3.10.0**: A signed long (8 bytes) sequential version is introduced
to overcome this overflow and the undefined behavior after that point. We
recommend you to start with this in newer ZooKeeper version, but we also do
suggest you to reevaluate your usages in case you are either experiencing or
worrying about above overflow.

<a name="Container+Nodes"></a>

#### Container Nodes
Expand Down
106 changes: 100 additions & 6 deletions zookeeper-server/src/main/java/org/apache/zookeeper/CreateMode.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.zookeeper;

import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -35,17 +36,59 @@ public enum CreateMode {
/**
* The znode will not be automatically deleted upon client's disconnect,
* and its name will be appended with a monotonically increasing number.
*
* @apiNote This number is 32-bit and unique to the parent znode. It is
* formatted with %010d, that is 10 digits with 0 (zero) padding,
* to simplify sorting, i.e. "0000000001". The counter used to
* store the sequence number is a signed int (4bytes) maintained
* by the parent node, it will overflow when incremented beyond
* 2147483647 (resulting in a name "-2147483648").
* @implNote The behavior after overflow is undefined.
*/
PERSISTENT_SEQUENTIAL(2, false, true, false, false),
/**
* The znode will not be automatically deleted upon client's disconnect,
* and its name will be appended with a monotonically increasing number.
*
* @apiNote This number is 64-bit and unique to the parent znode. It is
* formatted with %019d, that is 19 digits with 0 (zero) padding
* so to simplify sorting, i.e. "0000000000000000001".
* @apiNote It is not guaranteed to be consecutive.
* @implNote The number is {@link Stat#getCzxid()} for created node, so
* multiple creations with same prefix path in {@link ZooKeeper#multi(Iterable)}
* will fail with {@link KeeperException.NodeExistsException}.
*/
PERSISTENT_SEQUENTIAL_LONG(7, false, true, false, false, true),
/**
* The znode will be deleted upon the client's disconnect.
*/
EPHEMERAL(1, true, false, false, false),
/**
* The znode will be deleted upon the client's disconnect, and its name
* will be appended with a monotonically increasing number.
*
* @apiNote This number is 32-bit and unique to the parent znode. It is
* formatted with %010d, that is 10 digits with 0 (zero) padding,
* to simplify sorting, i.e. "0000000001". The counter used to
* store the sequence number is a signed int (4bytes) maintained
* by the parent node, it will overflow when incremented beyond
* 2147483647 (resulting in a name "-2147483648").
* @implNote The behavior after overflow is undefined.
*/
EPHEMERAL_SEQUENTIAL(3, true, true, false, false),
/**
* The znode will be deleted upon the client's disconnect, and its name
* will be appended with a monotonically increasing number.
*
* @apiNote This number is 64-bit and unique to the parent znode. It is
* formatted with %019d, that is 19 digits with 0 (zero) padding
* so to simplify sorting, i.e. "0000000000000000001".
* @apiNote It is not guaranteed to be consecutive.
* @implNote The number is {@link Stat#getCzxid()} for created node, so
* multiple creations with same prefix path in {@link ZooKeeper#multi(Iterable)}
* will fail with {@link KeeperException.NodeExistsException}.
*/
EPHEMERAL_SEQUENTIAL_LONG(8, true, true, false, false, true),
/**
* The znode will be a container node. Container
* nodes are special purpose nodes useful for recipes such as leader, lock,
Expand All @@ -67,23 +110,52 @@ public enum CreateMode {
* and its name will be appended with a monotonically increasing number.
* However if the znode has not been modified within the given TTL, it
* will be deleted once it has no children.
*
* @apiNote This number is 32-bit and unique to the parent znode. It is
* formatted with %010d, that is 10 digits with 0 (zero) padding,
* to simplify sorting, i.e. "0000000001". The counter used to
* store the sequence number is a signed int (4bytes) maintained
* by the parent node, it will overflow when incremented beyond
* 2147483647 (resulting in a name "-2147483648").
* @implNote The behavior after overflow is undefined.
*/
PERSISTENT_SEQUENTIAL_WITH_TTL(6, false, true, false, true),
/**
* The znode will not be automatically deleted upon client's disconnect,
* and its name will be appended with a monotonically increasing number.
* However if the znode has not been modified within the given TTL, it
* will be deleted once it has no children.
*
* @apiNote This number is 64-bit and unique to the parent znode. It is
* formatted with %019d, that is 19 digits with 0 (zero) padding
* so to simplify sorting, i.e. "0000000000000000001".
* @apiNote It is not guaranteed to be consecutive.
* @implNote The number is {@link Stat#getCzxid()} for created node, so
* multiple creations with same prefix path in {@link ZooKeeper#multi(Iterable)}
* will fail with {@link KeeperException.NodeExistsException}.
*/
PERSISTENT_SEQUENTIAL_WITH_TTL(6, false, true, false, true);
PERSISTENT_SEQUENTIAL_LONG_WITH_TTL(9, false, true, false, true, true);

private static final Logger LOG = LoggerFactory.getLogger(CreateMode.class);

private boolean ephemeral;
private boolean sequential;
private final boolean ephemeral;
private final boolean sequential;
private final boolean isContainer;
private int flag;
private boolean isTTL;
private final int flag;
private final boolean isTTL;
private final boolean isLong;

CreateMode(int flag, boolean ephemeral, boolean sequential, boolean isContainer, boolean isTTL) {
CreateMode(int flag, boolean ephemeral, boolean sequential, boolean isContainer, boolean isTTL, boolean isLong) {
this.flag = flag;
this.ephemeral = ephemeral;
this.sequential = sequential;
this.isContainer = isContainer;
this.isTTL = isTTL;
this.isLong = isLong;
}

CreateMode(int flag, boolean ephemeral, boolean sequential, boolean isContainer, boolean isTTL) {
this(flag, ephemeral, sequential, isContainer, isTTL, false);
}

public boolean isEphemeral() {
Expand All @@ -94,6 +166,10 @@ public boolean isSequential() {
return sequential;
}

public boolean isLongSequential() {
return sequential && isLong;
}

public boolean isContainer() {
return isContainer;
}
Expand Down Expand Up @@ -132,6 +208,15 @@ public static CreateMode fromFlag(int flag) throws KeeperException {
case 6:
return CreateMode.PERSISTENT_SEQUENTIAL_WITH_TTL;

case 7:
return CreateMode.PERSISTENT_SEQUENTIAL_LONG;

case 8:
return CreateMode.EPHEMERAL_SEQUENTIAL_LONG;

case 9:
return CreateMode.PERSISTENT_SEQUENTIAL_LONG_WITH_TTL;

default:
String errMsg = "Received an invalid flag value: " + flag + " to convert to a CreateMode";
LOG.error(errMsg);
Expand Down Expand Up @@ -165,6 +250,15 @@ public static CreateMode fromFlag(int flag, CreateMode defaultMode) {
case 6:
return CreateMode.PERSISTENT_SEQUENTIAL_WITH_TTL;

case 7:
return CreateMode.PERSISTENT_SEQUENTIAL_LONG;

case 8:
return CreateMode.EPHEMERAL_SEQUENTIAL_LONG;

case 9:
return CreateMode.PERSISTENT_SEQUENTIAL_LONG_WITH_TTL;

default:
return defaultMode;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -665,8 +665,11 @@ private void pRequest2TxnCreate(int type, Request request, Record record) throws
ChangeRecord parentRecord = getRecordForPath(parentPath);

zks.checkACL(request.cnxn, parentRecord.acl, ZooDefs.Perms.CREATE, request.authInfo, path, listACL);
int parentCVersion = parentRecord.stat.getCversion();
if (createMode.isSequential()) {
if (createMode.isLongSequential()) {
long zxid = request.getHdr().getZxid();
path = path + String.format(Locale.ENGLISH, "%019d", zxid);
} else if (createMode.isSequential()) {
int parentCVersion = parentRecord.stat.getCversion();
path = path + String.format(Locale.ENGLISH, "%010d", parentCVersion);
}
validatePath(path, request.sessionId);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,14 +20,16 @@

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNotSame;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.Locale;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
Expand Down Expand Up @@ -549,16 +551,10 @@ public void testSequentialNodeData() throws Exception {
zk.create(queue_handle + "/element", "1".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT_SEQUENTIAL);
List<String> children = zk.getChildren(queue_handle, true);
assertEquals(children.size(), 2);
children.sort(Comparator.naturalOrder());
String child1 = children.get(0);
String child2 = children.get(1);
int compareResult = child1.compareTo(child2);
assertNotSame(compareResult, 0);
if (compareResult < 0) {
} else {
String temp = child1;
child1 = child2;
child2 = temp;
}
assertNotEquals(child1, child2);
String child1data = new String(zk.getData(queue_handle + "/" + child1, false, null));
String child2data = new String(zk.getData(queue_handle + "/" + child2, false, null));
assertEquals(child1data, "0");
Expand All @@ -571,6 +567,35 @@ public void testSequentialNodeData() throws Exception {

}

// Test that long sequential nodes are being created correctly,
// with 0-padding in the node names and monotonically increased in lexicographically order.
@Test
public void testLongSequentialNodes() throws IOException, InterruptedException, KeeperException {
assertEquals(19, String.format("%d", Long.MAX_VALUE).length());
assertEquals("0000000000000000001", String.format(Locale.ENGLISH, "%019d", 1L));

String parent_path = "/SEQUENCE";
String prefix_path = parent_path + "/TEST";

try (ZooKeeper zk = createClient()) {
zk.create(parent_path, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);

Stat stat1 = new Stat();
Stat stat2 = new Stat();
String sequential_path1 = zk.create(prefix_path, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT_SEQUENTIAL_LONG, stat1);
String sequential_path2 = zk.create(prefix_path, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT_SEQUENTIAL_LONG, stat2);

assertTrue(sequential_path1.compareTo(sequential_path2) < 0);

List<String> children = zk.getChildren(parent_path, false);
assertEquals(2, children.size());

// This is the internal detail, it is ok to rewrite if changed in the future.
assertEquals(sequential_path1, String.format("%s%019d", prefix_path, stat1.getCzxid()));
assertEquals(sequential_path2, String.format("%s%019d", prefix_path, stat2.getCzxid()));
}
}

@Test
public void testLargeNodeData() throws Exception {
ZooKeeper zk = null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,31 +37,81 @@ public void testBasicCreateMode() {
assertEquals(cm.toFlag(), 0);
assertFalse(cm.isEphemeral());
assertFalse(cm.isSequential());
assertFalse(cm.isLongSequential());
assertFalse(cm.isContainer());
assertFalse(cm.isTTL());

cm = CreateMode.EPHEMERAL;
assertEquals(cm.toFlag(), 1);
assertTrue(cm.isEphemeral());
assertFalse(cm.isSequential());
assertFalse(cm.isLongSequential());
assertFalse(cm.isContainer());
assertFalse(cm.isTTL());

cm = CreateMode.PERSISTENT_SEQUENTIAL;
assertEquals(cm.toFlag(), 2);
assertFalse(cm.isEphemeral());
assertTrue(cm.isSequential());
assertFalse(cm.isLongSequential());
assertFalse(cm.isContainer());
assertFalse(cm.isTTL());

cm = CreateMode.EPHEMERAL_SEQUENTIAL;
assertEquals(cm.toFlag(), 3);
assertTrue(cm.isEphemeral());
assertTrue(cm.isSequential());
assertFalse(cm.isLongSequential());
assertFalse(cm.isContainer());
assertFalse(cm.isTTL());

cm = CreateMode.CONTAINER;
assertEquals(cm.toFlag(), 4);
assertFalse(cm.isEphemeral());
assertFalse(cm.isSequential());
assertFalse(cm.isLongSequential());
assertTrue(cm.isContainer());
assertFalse(cm.isTTL());

cm = CreateMode.PERSISTENT_WITH_TTL;
assertEquals(cm.toFlag(), 5);
assertFalse(cm.isEphemeral());
assertFalse(cm.isSequential());
assertFalse(cm.isLongSequential());
assertFalse(cm.isContainer());
assertTrue(cm.isTTL());

cm = CreateMode.PERSISTENT_SEQUENTIAL_WITH_TTL;
assertEquals(cm.toFlag(), 6);
assertFalse(cm.isEphemeral());
assertTrue(cm.isSequential());
assertFalse(cm.isLongSequential());
assertFalse(cm.isContainer());
assertTrue(cm.isTTL());

cm = CreateMode.PERSISTENT_SEQUENTIAL_LONG;
assertEquals(cm.toFlag(), 7);
assertFalse(cm.isEphemeral());
assertTrue(cm.isSequential());
assertTrue(cm.isLongSequential());
assertFalse(cm.isContainer());
assertFalse(cm.isTTL());

cm = CreateMode.EPHEMERAL_SEQUENTIAL_LONG;
assertEquals(cm.toFlag(), 8);
assertTrue(cm.isEphemeral());
assertTrue(cm.isSequential());
assertTrue(cm.isLongSequential());
assertFalse(cm.isContainer());
assertFalse(cm.isTTL());

cm = CreateMode.PERSISTENT_SEQUENTIAL_LONG_WITH_TTL;
assertEquals(cm.toFlag(), 9);
assertFalse(cm.isEphemeral());
assertTrue(cm.isSequential());
assertTrue(cm.isLongSequential());
assertFalse(cm.isContainer());
assertTrue(cm.isTTL());
}

@Test
Expand Down Expand Up @@ -89,6 +139,8 @@ public void testInvalidFlagConversion() throws KeeperException {
} catch (KeeperException ke) {
assertEquals(Code.BADARGUMENTS, ke.code());
}

assertEquals(CreateMode.PERSISTENT, CreateMode.fromFlag(-1, CreateMode.PERSISTENT));
}

}