Use Flake IDs instead of random UUIDs when auto-generating id field

Flake IDs give better lookup performance in Lucene since they share
predictable prefixes (timestamp).

Closes #7531

Closes #6004

Closes #5941
This commit is contained in:
mikemccand 2014-09-02 09:13:51 -04:00
parent 20dcb0e08a
commit 9c1ac95ba8
8 changed files with 448 additions and 49 deletions

View File

@ -616,7 +616,7 @@ public class IndexRequest extends ShardReplicationOperationRequest<IndexRequest>
// generate id if not already provided and id generation is allowed
if (allowIdGeneration) {
if (id == null) {
id(Strings.randomBase64UUID());
id(Strings.base64UUID());
// since we generate the id, change it to CREATE
opType(IndexRequest.OpType.CREATE);
autoGeneratedId = true;

View File

@ -0,0 +1,98 @@
/*
* 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 org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import java.net.NetworkInterface;
import java.net.SocketException;
import java.util.Enumeration;
public class MacAddressProvider {
private static final ESLogger logger = Loggers.getLogger(MacAddressProvider.class);
private static byte[] getMacAddress() throws SocketException {
Enumeration<NetworkInterface> en = NetworkInterface.getNetworkInterfaces();
if (en != null) {
while (en.hasMoreElements()) {
NetworkInterface nint = en.nextElement();
if (!nint.isLoopback()) {
// Pick the first valid non loopback address we find
byte[] address = nint.getHardwareAddress();
if (isValidAddress(address)) {
return address;
}
}
}
}
// Could not find a mac address
return null;
}
private static boolean isValidAddress(byte[] address) {
if (address == null || address.length != 6) {
return false;
}
for (byte b : address) {
if (b != 0x00) {
return true; // If any of the bytes are non zero assume a good address
}
}
return false;
}
public static byte[] getSecureMungedAddress() {
byte[] address = null;
try {
address = getMacAddress();
} catch( SocketException se ) {
logger.warn("Unable to get mac address, will use a dummy address", se);
// address will be set below
}
if (!isValidAddress(address)) {
logger.warn("Unable to get a valid mac address, will use a dummy address");
address = constructDummyMulticastAddress();
}
byte[] mungedBytes = new byte[6];
SecureRandomHolder.INSTANCE.nextBytes(mungedBytes);
for (int i = 0; i < 6; ++i) {
mungedBytes[i] ^= address[i];
}
return mungedBytes;
}
private static byte[] constructDummyMulticastAddress() {
byte[] dummy = new byte[6];
SecureRandomHolder.INSTANCE.nextBytes(dummy);
/*
* Set the broadcast bit to indicate this is not a _real_ mac address
*/
dummy[0] |= (byte) 0x01;
return dummy;
}
}

View File

@ -0,0 +1,68 @@
/*
* 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 org.elasticsearch.ElasticsearchIllegalStateException;
import java.io.IOException;
import java.util.Random;
class RandomBasedUUIDGenerator implements UUIDGenerator {
/**
* Returns a Base64 encoded version of a Version 4.0 compatible UUID
* as defined here: http://www.ietf.org/rfc/rfc4122.txt
*/
@Override
public String getBase64UUID() {
return getBase64UUID(SecureRandomHolder.INSTANCE);
}
/**
* Returns a Base64 encoded version of a Version 4.0 compatible UUID
* randomly initialized by the given {@link java.util.Random} instance
* as defined here: http://www.ietf.org/rfc/rfc4122.txt
*/
public String getBase64UUID(Random random) {
final byte[] randomBytes = new byte[16];
random.nextBytes(randomBytes);
/* Set the version to version 4 (see http://www.ietf.org/rfc/rfc4122.txt)
* The randomly or pseudo-randomly generated version.
* The version number is in the most significant 4 bits of the time
* stamp (bits 4 through 7 of the time_hi_and_version field).*/
randomBytes[6] &= 0x0f; /* clear the 4 most significant bits for the version */
randomBytes[6] |= 0x40; /* set the version to 0100 / 0x40 */
/* Set the variant:
* The high field of th clock sequence multiplexed with the variant.
* We set only the MSB of the variant*/
randomBytes[8] &= 0x3f; /* clear the 2 most significant bits */
randomBytes[8] |= 0x80; /* set the variant (MSB is set)*/
try {
byte[] encoded = Base64.encodeBytesToBytes(randomBytes, 0, randomBytes.length, Base64.URL_SAFE);
// we know the bytes are 16, and not a multi of 3, so remove the 2 padding chars that are added
assert encoded[encoded.length - 1] == '=';
assert encoded[encoded.length - 2] == '=';
return new String(encoded, 0, encoded.length - 2, Base64.PREFERRED_ENCODING);
} catch (IOException e) {
throw new ElasticsearchIllegalStateException("should not be thrown");
}
}
}

View File

@ -0,0 +1,27 @@
/*
* 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.security.SecureRandom;
class SecureRandomHolder {
// class loading is atomic - this is a lazy & safe singleton to be used by this package
public static final SecureRandom INSTANCE = new SecureRandom();
}

View File

@ -30,8 +30,6 @@ import org.elasticsearch.common.io.FastStringReader;
import org.elasticsearch.common.util.CollectionUtils;
import java.io.BufferedReader;
import java.io.IOException;
import java.security.SecureRandom;
import java.util.*;
/**
@ -49,6 +47,9 @@ public class Strings {
private static final String CURRENT_PATH = ".";
private static final RandomBasedUUIDGenerator RANDOM_UUID_GENERATOR = new RandomBasedUUIDGenerator();
private static final UUIDGenerator TIME_UUID_GENERATOR = new TimeBasedUUIDGenerator();
public static void spaceify(int spaces, String from, StringBuilder to) throws Exception {
try (BufferedReader reader = new BufferedReader(new FastStringReader(from))) {
String line;
@ -1023,51 +1024,6 @@ public class Strings {
return bytes;
}
private static class SecureRandomHolder {
// class loading is atomic - this is a lazy & safe singleton
private static final SecureRandom INSTANCE = new SecureRandom();
}
/**
* Returns a Base64 encoded version of a Version 4.0 compatible UUID
* as defined here: http://www.ietf.org/rfc/rfc4122.txt
*/
public static String randomBase64UUID() {
return randomBase64UUID(SecureRandomHolder.INSTANCE);
}
/**
* Returns a Base64 encoded version of a Version 4.0 compatible UUID
* randomly initialized by the given {@link Random} instance
* as defined here: http://www.ietf.org/rfc/rfc4122.txt
*/
public static String randomBase64UUID(Random random) {
final byte[] randomBytes = new byte[16];
random.nextBytes(randomBytes);
/* Set the version to version 4 (see http://www.ietf.org/rfc/rfc4122.txt)
* The randomly or pseudo-randomly generated version.
* The version number is in the most significant 4 bits of the time
* stamp (bits 4 through 7 of the time_hi_and_version field).*/
randomBytes[6] &= 0x0f; /* clear the 4 most significant bits for the version */
randomBytes[6] |= 0x40; /* set the version to 0100 / 0x40 */
/* Set the variant:
* The high field of th clock sequence multiplexed with the variant.
* We set only the MSB of the variant*/
randomBytes[8] &= 0x3f; /* clear the 2 most significant bits */
randomBytes[8] |= 0x80; /* set the variant (MSB is set)*/
try {
byte[] encoded = Base64.encodeBytesToBytes(randomBytes, 0, randomBytes.length, Base64.URL_SAFE);
// we know the bytes are 16, and not a multi of 3, so remove the 2 padding chars that are added
assert encoded[encoded.length - 1] == '=';
assert encoded[encoded.length - 2] == '=';
// we always have padding of two at the end, encode it differently
return new String(encoded, 0, encoded.length - 2, Base64.PREFERRED_ENCODING);
} catch (IOException e) {
throw new ElasticsearchIllegalStateException("should not be thrown");
}
}
/**
* Return substring(beginIndex, endIndex) that is impervious to string length.
@ -1094,4 +1050,22 @@ public class Strings {
return CollectionUtils.isEmpty(data) ||
data.length == 1 && ("_all".equals(data[0]) || "*".equals(data[0]));
}
}
/** Returns a Base64 encoded version of a Version 4.0 compatible UUID as defined here: http://www.ietf.org/rfc/rfc4122.txt, using a
* private {@code SecureRandom} instance */
public static String randomBase64UUID() {
return RANDOM_UUID_GENERATOR.getBase64UUID();
}
/** Returns a Base64 encoded version of a Version 4.0 compatible UUID as defined here: http://www.ietf.org/rfc/rfc4122.txt, using the
* provided {@code Random} instance */
public static String randomBase64UUID(Random random) {
return RANDOM_UUID_GENERATOR.getBase64UUID(random);
}
/** Generates a time-based UUID (similar to Flake IDs), which is preferred when generating an ID to be indexed into a Lucene index as
* primary key. The id is opaque and the implementation is free to change at any time! */
public static String base64UUID() {
return TIME_UUID_GENERATOR.getBase64UUID();
}
}

View File

@ -0,0 +1,97 @@
/*
* 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 org.elasticsearch.ElasticsearchIllegalStateException;
import java.io.IOException;
import java.util.Random;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
/** 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 TimeBasedUUIDGenerator 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[] secureMungedAddress = MacAddressProvider.getSecureMungedAddress();
static {
assert secureMungedAddress.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(secureMungedAddress, 0, uuidBytes, 6, secureMungedAddress.length);
// Sequence number adds 3 bytes:
putLong(uuidBytes, sequenceId, 12, 3);
assert 9 + secureMungedAddress.length == uuidBytes.length;
byte[] encoded;
try {
encoded = Base64.encodeBytesToBytes(uuidBytes, 0, uuidBytes.length, Base64.URL_SAFE);
} catch (IOException e) {
throw new ElasticsearchIllegalStateException("should not be thrown", e);
}
// We are a multiple of 3 bytes so we should not see any padding:
assert encoded[encoded.length - 1] != '=';
return new String(encoded, 0, encoded.length, Base64.PREFERRED_ENCODING);
}
}

View File

@ -0,0 +1,27 @@
/*
* 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;
/**
* Generates opaque unique strings.
*/
interface UUIDGenerator {
public String getBase64UUID();
}

View File

@ -0,0 +1,108 @@
/*
* 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 org.elasticsearch.test.ElasticsearchTestCase;
import org.junit.Test;
import java.io.IOException;
import java.util.HashSet;
public class UUIDTests extends ElasticsearchTestCase {
static UUIDGenerator timeUUIDGen = new TimeBasedUUIDGenerator();
static UUIDGenerator randomUUIDGen = new RandomBasedUUIDGenerator();
@Test
public void testRandomUUID() {
verifyUUIDSet(100000, randomUUIDGen);
}
@Test
public void testTimeUUID() {
verifyUUIDSet(100000, timeUUIDGen);
}
@Test
public void testThreadedTimeUUID() {
testUUIDThreaded(timeUUIDGen);
}
@Test
public void testThreadedRandomUUID() {
testUUIDThreaded(randomUUIDGen);
}
HashSet verifyUUIDSet(int count, UUIDGenerator uuidSource) {
HashSet<String> uuidSet = new HashSet<>();
for (int i = 0; i < count; ++i) {
uuidSet.add(uuidSource.getBase64UUID());
}
assertEquals(count, uuidSet.size());
return uuidSet;
}
class UUIDGenRunner implements Runnable {
int count;
public HashSet<String> uuidSet = null;
UUIDGenerator uuidSource;
public UUIDGenRunner(int count, UUIDGenerator uuidSource) {
this.count = count;
this.uuidSource = uuidSource;
}
@Override
public void run() {
uuidSet = verifyUUIDSet(count, uuidSource);
}
}
public void testUUIDThreaded(UUIDGenerator uuidSource) {
HashSet<UUIDGenRunner> runners = new HashSet<>();
HashSet<Thread> threads = new HashSet<>();
int count = 100;
int uuids = 10000;
for (int i = 0; i < count; ++i) {
UUIDGenRunner runner = new UUIDGenRunner(uuids, uuidSource);
Thread t = new Thread(runner);
threads.add(t);
runners.add(runner);
}
for (Thread t : threads) {
t.start();
}
boolean retry = false;
do {
for (Thread t : threads) {
try {
t.join();
} catch (InterruptedException ie) {
retry = true;
}
}
} while (retry);
HashSet<String> globalSet = new HashSet<>();
for (UUIDGenRunner runner : runners) {
globalSet.addAll(runner.uuidSet);
}
assertEquals(count*uuids, globalSet.size());
}
}