-
Notifications
You must be signed in to change notification settings - Fork 25.3k
Optimize the order of bytes in uuids for better compression. #24615
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
Changes from all commits
8c1e400
f87dfdf
373253e
d8c2ac6
c9e14ab
723ca50
e1584ec
837d88e
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 |
---|---|---|
@@ -0,0 +1,86 @@ | ||
/* | ||
* Licensed to Elasticsearch under one or more contributor | ||
* license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright | ||
* ownership. Elasticsearch licenses this file to you under | ||
* the Apache License, Version 2.0 (the "License"); you may | ||
* not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, | ||
* software distributed under the License is distributed on an | ||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
|
||
package org.elasticsearch.common; | ||
|
||
import java.util.Base64; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
|
||
/** | ||
* These are essentially flake ids (http://boundary.com/blog/2012/01/12/flake-a-decentralized-k-ordered-unique-id-generator-in-erlang) but | ||
* we use 6 (not 8) bytes for timestamp, and use 3 (not 2) bytes for sequence number. | ||
*/ | ||
|
||
class LegacyTimeBasedUUIDGenerator implements UUIDGenerator { | ||
|
||
// We only use bottom 3 bytes for the sequence number. Paranoia: init with random int so that if JVM/OS/machine goes down, clock slips | ||
// backwards, and JVM comes back up, we are less likely to be on the same sequenceNumber at the same time: | ||
private final AtomicInteger sequenceNumber = new AtomicInteger(SecureRandomHolder.INSTANCE.nextInt()); | ||
|
||
// Used to ensure clock moves forward: | ||
private long lastTimestamp; | ||
|
||
private static final byte[] SECURE_MUNGED_ADDRESS = MacAddressProvider.getSecureMungedAddress(); | ||
|
||
static { | ||
assert SECURE_MUNGED_ADDRESS.length == 6; | ||
} | ||
|
||
/** Puts the lower numberOfLongBytes from l into the array, starting index pos. */ | ||
private static void putLong(byte[] array, long l, int pos, int numberOfLongBytes) { | ||
for (int i=0; i<numberOfLongBytes; ++i) { | ||
array[pos+numberOfLongBytes-i-1] = (byte) (l >>> (i*8)); | ||
} | ||
} | ||
|
||
@Override | ||
public String getBase64UUID() { | ||
final int sequenceId = sequenceNumber.incrementAndGet() & 0xffffff; | ||
long timestamp = System.currentTimeMillis(); | ||
|
||
synchronized (this) { | ||
// Don't let timestamp go backwards, at least "on our watch" (while this JVM is running). We are still vulnerable if we are | ||
// shut down, clock goes backwards, and we restart... for this we randomize the sequenceNumber on init to decrease chance of | ||
// collision: | ||
timestamp = Math.max(lastTimestamp, timestamp); | ||
|
||
if (sequenceId == 0) { | ||
// Always force the clock to increment whenever sequence number is 0, in case we have a long time-slip backwards: | ||
timestamp++; | ||
} | ||
|
||
lastTimestamp = timestamp; | ||
} | ||
|
||
final byte[] uuidBytes = new byte[15]; | ||
|
||
// Only use lower 6 bytes of the timestamp (this will suffice beyond the year 10000): | ||
putLong(uuidBytes, timestamp, 0, 6); | ||
|
||
// MAC address adds 6 bytes: | ||
System.arraycopy(SECURE_MUNGED_ADDRESS, 0, uuidBytes, 6, SECURE_MUNGED_ADDRESS.length); | ||
|
||
// Sequence number adds 3 bytes: | ||
putLong(uuidBytes, sequenceId, 12, 3); | ||
|
||
assert 9 + SECURE_MUNGED_ADDRESS.length == uuidBytes.length; | ||
|
||
return Base64.getUrlEncoder().withoutPadding().encodeToString(uuidBytes); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -23,7 +23,8 @@ | |
import java.util.concurrent.atomic.AtomicInteger; | ||
|
||
/** These are essentially flake ids (http://boundary.com/blog/2012/01/12/flake-a-decentralized-k-ordered-unique-id-generator-in-erlang) but | ||
* we use 6 (not 8) bytes for timestamp, and use 3 (not 2) bytes for sequence number. */ | ||
* we use 6 (not 8) bytes for timestamp, and use 3 (not 2) bytes for sequence number. We also reorder bytes in a way that does not make ids | ||
* sort in order anymore, but is more friendly to the way that the Lucene terms dictionary is structured. */ | ||
|
||
class TimeBasedUUIDGenerator implements UUIDGenerator { | ||
|
||
|
@@ -40,17 +41,20 @@ class TimeBasedUUIDGenerator implements UUIDGenerator { | |
assert SECURE_MUNGED_ADDRESS.length == 6; | ||
} | ||
|
||
/** Puts the lower numberOfLongBytes from l into the array, starting index pos. */ | ||
private static void putLong(byte[] array, long l, int pos, int numberOfLongBytes) { | ||
for (int i=0; i<numberOfLongBytes; ++i) { | ||
array[pos+numberOfLongBytes-i-1] = (byte) (l >>> (i*8)); | ||
} | ||
// protected for testing | ||
protected long currentTimeMillis() { | ||
return System.currentTimeMillis(); | ||
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. Is it worth looking at nanotime which doesn't go backwards? 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. That would not work as we need some absolute times. For instance you switch to a different JVM that uses a different origin for nanotime, you might reuse existing ids. 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. What about initializing to System.currentTimeMillis() on startup and using that time + System.nanoTime() on generation? Regardless, if it is worth doing at all it can wait for a followup. 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. yeah, maybe we should focus on the order of bytes here and rethink generation separately? 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.
++ |
||
} | ||
|
||
// protected for testing | ||
protected byte[] macAddress() { | ||
return SECURE_MUNGED_ADDRESS; | ||
} | ||
|
||
@Override | ||
public String getBase64UUID() { | ||
final int sequenceId = sequenceNumber.incrementAndGet() & 0xffffff; | ||
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. Can you skip the masking given the way you are using the sequence id? 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. no, because of the 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 see now. |
||
long timestamp = System.currentTimeMillis(); | ||
long timestamp = currentTimeMillis(); | ||
|
||
synchronized (this) { | ||
// Don't let timestamp go backwards, at least "on our watch" (while this JVM is running). We are still vulnerable if we are | ||
|
@@ -67,17 +71,45 @@ public String getBase64UUID() { | |
} | ||
|
||
final byte[] uuidBytes = new byte[15]; | ||
|
||
// Only use lower 6 bytes of the timestamp (this will suffice beyond the year 10000): | ||
putLong(uuidBytes, timestamp, 0, 6); | ||
|
||
// MAC address adds 6 bytes: | ||
System.arraycopy(SECURE_MUNGED_ADDRESS, 0, uuidBytes, 6, SECURE_MUNGED_ADDRESS.length); | ||
|
||
// Sequence number adds 3 bytes: | ||
putLong(uuidBytes, sequenceId, 12, 3); | ||
|
||
assert 9 + SECURE_MUNGED_ADDRESS.length == uuidBytes.length; | ||
int i = 0; | ||
|
||
// We have auto-generated ids, which are usually used for append-only workloads. | ||
// So we try to optimize the order of bytes for indexing speed (by having quite | ||
// unique bytes close to the beginning of the ids so that sorting is fast) and | ||
// compression (by making sure we share common prefixes between enough ids), | ||
// but not necessarily for lookup speed (by having the leading bytes identify | ||
// segments whenever possible) | ||
|
||
// Blocks in the block tree have between 25 and 48 terms. So all prefixes that | ||
// are shared by ~30 terms should be well compressed. I first tried putting the | ||
// two lower bytes of the sequence id in the beginning of the id, but compression | ||
// is only triggered when you have at least 30*2^16 ~= 2M documents in a segment, | ||
// which is already quite large. So instead, we are putting the 1st and 3rd byte | ||
// of the sequence number so that compression starts to be triggered with smaller | ||
// segment sizes and still gives pretty good indexing speed. We use the sequenceId | ||
// rather than the timestamp because the distribution of the timestamp depends too | ||
// much on the indexing rate, so it is less reliable. | ||
|
||
uuidBytes[i++] = (byte) sequenceId; | ||
// changes every 65k docs, so potentially every second if you have a steady indexing rate | ||
uuidBytes[i++] = (byte) (sequenceId >>> 16); | ||
|
||
// Now we start focusing on compression and put bytes that should not change too often. | ||
uuidBytes[i++] = (byte) (timestamp >>> 16); // changes every ~65 secs | ||
uuidBytes[i++] = (byte) (timestamp >>> 24); // changes every ~4.5h | ||
uuidBytes[i++] = (byte) (timestamp >>> 32); // changes every ~50 days | ||
uuidBytes[i++] = (byte) (timestamp >>> 40); // changes every 35 years | ||
byte[] macAddress = macAddress(); | ||
assert macAddress.length == 6; | ||
System.arraycopy(macAddress, 0, uuidBytes, i, macAddress.length); | ||
i += macAddress.length; | ||
|
||
// Finally we put the remaining bytes, which will likely not be compressed at all. | ||
uuidBytes[i++] = (byte) (timestamp >>> 8); | ||
uuidBytes[i++] = (byte) (sequenceId >>> 8); | ||
uuidBytes[i++] = (byte) timestamp; | ||
|
||
assert i == uuidBytes.length; | ||
|
||
return Base64.getUrlEncoder().withoutPadding().encodeToString(uuidBytes); | ||
} | ||
|
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.
I wonder why we don't make
lastTimestamp
an AtomicLong and usecompareAndSet
since we don't care if the compare and set was successful as long as we are moving forward? something like this:if that doesn't work and we really need to block or have consistent values aren't we vulnerable in terms of collisions on the same timestamp?
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.
I'm working on this for a separate PR if you don't mind.
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.
Thinking about it more, it would ensure that
lastUsedTimestamp
is greater than it was before, but not necessarily greater thantimestamp
. So there is a possibility of collision?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.
but don't we have the sequenc ID for this pupose. there are 2^31 docs to be indexed before you have a collision. I mean if System.currentTimeInMillis() goes backwards we reuse the same timestamp all the time? I am not sure how this would work at all if we can't sustain that?
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.
no, since we force the clock to tick forward whenever the sequence id is 0?
2^24 actually, we only use the 3 lower bytes of the sequence id