-
Notifications
You must be signed in to change notification settings - Fork 451
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
speeds up fate lock acquisition #5262
base: main
Are you sure you want to change the base?
Changes from 1 commit
20fbbe8
20097d5
a45b3ad
3fbdd6e
3f25ce0
95fb8dc
2309b9b
ac9a6b5
1d4d7f2
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,13 +18,15 @@ | |
*/ | ||
package org.apache.accumulo.core.fate.zookeeper; | ||
|
||
import static java.nio.charset.StandardCharsets.UTF_8; | ||
import static java.util.Objects.requireNonNull; | ||
|
||
import java.util.ArrayList; | ||
import java.util.Collections; | ||
import java.util.List; | ||
import java.util.SortedMap; | ||
import java.util.TreeMap; | ||
import java.util.function.BiPredicate; | ||
|
||
import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.QueueLock; | ||
import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy; | ||
|
@@ -35,6 +37,8 @@ | |
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import com.google.common.base.Preconditions; | ||
|
||
/** | ||
* A persistent lock mechanism in ZooKeeper used for locking tables during FaTE operations. | ||
*/ | ||
|
@@ -68,16 +72,34 @@ public FateLock(ZooReaderWriter zrw, FateLockPath path) { | |
this.path = requireNonNull(path); | ||
} | ||
|
||
public static class FateLockNode { | ||
public final long sequence; | ||
public final String lockData; | ||
|
||
private FateLockNode(String nodeName) { | ||
int len = nodeName.length(); | ||
Preconditions.checkArgument(nodeName.startsWith(PREFIX) && nodeName.charAt(len - 11) == '#', | ||
"Illegal node name %s", nodeName); | ||
sequence = Long.parseLong(nodeName.substring(len - 10)); | ||
lockData = nodeName.substring(PREFIX.length(), len - 11); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. So, while this is probably unlikely to happen, the sequential numbering can overflow causing a negative to occur, which I think would cause a problem with this parsing instead of splitting on There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Added handing and test for negative seq numbers in 2309b9b |
||
} | ||
} | ||
|
||
// TODO change data arg from byte[] to String.. in the rest of the code its always a String. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I will open a follow on issue for this and remove the TODO before merging. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Removed the TODO and #5264. In the TODO I was thinking of using String, but realized using more concrete types would be better. |
||
@Override | ||
public long addEntry(byte[] data) { | ||
|
||
String dataString = new String(data, UTF_8); | ||
Preconditions.checkState(!dataString.contains("#")); | ||
|
||
String newPath; | ||
try { | ||
while (true) { | ||
try { | ||
newPath = zoo.putPersistentSequential(path + "/" + PREFIX, data); | ||
newPath = zoo.putPersistentSequential(path + "/" + PREFIX + dataString + "#", data); | ||
keith-turner marked this conversation as resolved.
Show resolved
Hide resolved
|
||
String[] parts = newPath.split("/"); | ||
String last = parts[parts.length - 1]; | ||
return Long.parseLong(last.substring(PREFIX.length())); | ||
return new FateLockNode(last).sequence; | ||
} catch (NoNodeException nne) { | ||
// the parent does not exist so try to create it | ||
zoo.putPersistentData(path.toString(), new byte[] {}, NodeExistsPolicy.SKIP); | ||
|
@@ -89,7 +111,7 @@ public long addEntry(byte[] data) { | |
} | ||
|
||
@Override | ||
public SortedMap<Long,byte[]> getEarlierEntries(long entry) { | ||
public SortedMap<Long,byte[]> getEntries(BiPredicate<Long,byte[]> predicate) { | ||
SortedMap<Long,byte[]> result = new TreeMap<>(); | ||
try { | ||
List<String> children = Collections.emptyList(); | ||
|
@@ -101,15 +123,10 @@ public SortedMap<Long,byte[]> getEarlierEntries(long entry) { | |
} | ||
|
||
for (String name : children) { | ||
// this try catch must be done inside the loop because some subset of the children may exist | ||
try { | ||
long order = Long.parseLong(name.substring(PREFIX.length())); | ||
if (order <= entry) { | ||
byte[] data = zoo.getData(path + "/" + name); | ||
result.put(order, data); | ||
} | ||
} catch (KeeperException.NoNodeException ex) { | ||
// ignored | ||
var parsed = new FateLockNode(name); | ||
byte[] data = parsed.lockData.getBytes(UTF_8); | ||
if (predicate.test(parsed.sequence, data)) { | ||
result.put(parsed.sequence, data); | ||
} | ||
} | ||
} catch (KeeperException | InterruptedException ex) { | ||
|
@@ -119,9 +136,12 @@ public SortedMap<Long,byte[]> getEarlierEntries(long entry) { | |
} | ||
|
||
@Override | ||
public void removeEntry(long entry) { | ||
public void removeEntry(byte[] data, long entry) { | ||
String dataString = new String(data, UTF_8); | ||
Preconditions.checkState(!dataString.contains("#")); | ||
try { | ||
zoo.recursiveDelete(path + String.format("/%s%010d", PREFIX, entry), NodeMissingPolicy.SKIP); | ||
zoo.recursiveDelete(path + String.format("/%s%s#%010d", PREFIX, dataString, entry), | ||
NodeMissingPolicy.SKIP); | ||
try { | ||
// try to delete the parent if it has no children | ||
zoo.delete(path.toString()); | ||
|
@@ -136,50 +156,25 @@ public void removeEntry(long entry) { | |
/** | ||
* Validate and sort child nodes at this lock path by the lock prefix | ||
*/ | ||
public static List<String> validateAndSort(FateLockPath path, List<String> children) { | ||
public static List<FateLockNode> validateAndWarn(FateLockPath path, List<String> children) { | ||
log.trace("validating and sorting children at path {}", path); | ||
List<String> validChildren = new ArrayList<>(); | ||
|
||
List<FateLockNode> validChildren = new ArrayList<>(); | ||
|
||
if (children == null || children.isEmpty()) { | ||
return validChildren; | ||
} | ||
|
||
children.forEach(c -> { | ||
log.trace("Validating {}", c); | ||
if (c.startsWith(PREFIX)) { | ||
int idx = c.indexOf('#'); | ||
String sequenceNum = c.substring(idx + 1); | ||
if (sequenceNum.length() == 10) { | ||
try { | ||
log.trace("Testing number format of {}", sequenceNum); | ||
Integer.parseInt(sequenceNum); | ||
validChildren.add(c); | ||
} catch (NumberFormatException e) { | ||
log.warn("Fate lock found with invalid sequence number format: {} (not a number)", c); | ||
} | ||
} else { | ||
log.warn("Fate lock found with invalid sequence number format: {} (not 10 characters)", | ||
c); | ||
} | ||
} else { | ||
log.warn("Fate lock found with invalid lock format: {} (does not start with {})", c, | ||
PREFIX); | ||
try { | ||
var fateLockNode = new FateLockNode(c); | ||
validChildren.add(fateLockNode); | ||
} catch (RuntimeException e) { | ||
log.warn("Illegal fate lock node {}", c, e); | ||
} | ||
}); | ||
|
||
if (validChildren.size() > 1) { | ||
validChildren.sort((o1, o2) -> { | ||
// Lock should be of the form: | ||
// lock-sequenceNumber | ||
// Example: | ||
// flock#0000000000 | ||
|
||
// Lock length - sequenceNumber length | ||
// 16 - 10 | ||
int secondHashIdx = 6; | ||
return Integer.valueOf(o1.substring(secondHashIdx)) | ||
.compareTo(Integer.valueOf(o2.substring(secondHashIdx))); | ||
}); | ||
} | ||
log.trace("Children nodes (size: {}): {}", validChildren.size(), validChildren); | ||
return validChildren; | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should the return type be a sorted set instead of a list that you sort later?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Updated to a sorted set in 3f25ce0