HDFS-12506. Ozone: ListBucket is too slow. Contributed by Weiwei Yang.

This commit is contained in:
Weiwei Yang 2017-09-25 16:50:58 +08:00 committed by Owen O'Malley
parent bf6f0cd831
commit fd1564b87e
10 changed files with 218 additions and 87 deletions

View File

@ -98,10 +98,29 @@ public enum Versioning {NOT_DEFINED, ENABLED, DISABLED}
/**
* KSM LevelDB prefixes.
*
* KSM DB stores metadata as KV pairs with certain prefixes,
* prefix is used to improve the performance to get related
* metadata.
*
* KSM DB Schema:
* ----------------------------------------------------------
* | KEY | VALUE |
* ----------------------------------------------------------
* | $userName | VolumeList |
* ----------------------------------------------------------
* | /#volumeName | VolumeInfo |
* ----------------------------------------------------------
* | /#volumeName/#bucketName | BucketInfo |
* ----------------------------------------------------------
* | /volumeName/bucketName/keyName | KeyInfo |
* ----------------------------------------------------------
* | #deleting#/volumeName/bucketName/keyName | KeyInfo |
* ----------------------------------------------------------
*/
public static final String KSM_VOLUME_PREFIX = "/";
public static final String KSM_BUCKET_PREFIX = KSM_VOLUME_PREFIX;
public static final String KSM_KEY_PREFIX = KSM_VOLUME_PREFIX;
public static final String KSM_VOLUME_PREFIX = "/#";
public static final String KSM_BUCKET_PREFIX = "/#";
public static final String KSM_KEY_PREFIX = "/";
public static final String KSM_USER_PREFIX = "$";
/**

View File

@ -21,7 +21,10 @@
import com.google.common.collect.Lists;
import org.apache.commons.lang3.tuple.ImmutablePair;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.ozone.ksm.helpers.*;
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
import org.apache.hadoop.ozone.common.BlockGroup;
import org.apache.hadoop.ozone.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
@ -146,16 +149,16 @@ private String getBucketKeyPrefix(String volume, String bucket) {
}
private String getKeyKeyPrefix(String volume, String bucket, String key) {
String keyStr = getBucketKeyPrefix(volume, bucket);
keyStr = Strings.isNullOrEmpty(key) ? keyStr + OzoneConsts.KSM_KEY_PREFIX
: keyStr + OzoneConsts.KSM_KEY_PREFIX + key;
return keyStr;
String keyVB = OzoneConsts.KSM_KEY_PREFIX + volume
+ OzoneConsts.KSM_KEY_PREFIX + bucket
+ OzoneConsts.KSM_KEY_PREFIX;
return Strings.isNullOrEmpty(key) ? keyVB : keyVB + key;
}
@Override
public byte[] getDBKeyForKey(String volume, String bucket, String key) {
String keyKeyString = OzoneConsts.KSM_VOLUME_PREFIX + volume
+ OzoneConsts.KSM_BUCKET_PREFIX + bucket + OzoneConsts.KSM_KEY_PREFIX
String keyKeyString = OzoneConsts.KSM_KEY_PREFIX + volume
+ OzoneConsts.KSM_KEY_PREFIX + bucket + OzoneConsts.KSM_KEY_PREFIX
+ key;
return DFSUtil.string2Bytes(keyKeyString);
}
@ -223,15 +226,14 @@ public void writeBatch(BatchOperation batch) throws IOException {
* @return true if the volume is empty
*/
public boolean isVolumeEmpty(String volume) throws IOException {
String dbVolumeRootName = OzoneConsts.KSM_VOLUME_PREFIX + volume;
String dbVolumeRootName = OzoneConsts.KSM_VOLUME_PREFIX + volume
+ OzoneConsts.KSM_BUCKET_PREFIX;
byte[] dbVolumeRootKey = DFSUtil.string2Bytes(dbVolumeRootName);
// Seek to the root of the volume and look for the next key
ImmutablePair<byte[], byte[]> volumeRoot =
store.peekAround(1, dbVolumeRootKey);
store.peekAround(0, dbVolumeRootKey);
if (volumeRoot != null) {
String firstBucketKey = DFSUtil.bytes2String(volumeRoot.getKey());
return !firstBucketKey.startsWith(dbVolumeRootName
+ OzoneConsts.KSM_BUCKET_PREFIX);
return !DFSUtil.bytes2String(volumeRoot.getKey())
.startsWith(dbVolumeRootName);
}
return true;
}
@ -245,13 +247,13 @@ public boolean isVolumeEmpty(String volume) throws IOException {
*/
public boolean isBucketEmpty(String volume, String bucket)
throws IOException {
String keyRootName = OzoneConsts.KSM_VOLUME_PREFIX + volume
+ OzoneConsts.KSM_BUCKET_PREFIX + bucket;
String keyRootName = OzoneConsts.KSM_KEY_PREFIX + volume
+ OzoneConsts.KSM_KEY_PREFIX + bucket + OzoneConsts.KSM_KEY_PREFIX;
byte[] keyRoot = DFSUtil.string2Bytes(keyRootName);
ImmutablePair<byte[], byte[]> firstKey = store.peekAround(1, keyRoot);
ImmutablePair<byte[], byte[]> firstKey = store.peekAround(0, keyRoot);
if (firstKey != null) {
return !DFSUtil.bytes2String(firstKey.getKey())
.startsWith(keyRootName + OzoneConsts.KSM_KEY_PREFIX);
.startsWith(keyRootName);
}
return true;
}
@ -276,30 +278,27 @@ public List<KsmBucketInfo> listBuckets(final String volumeName,
}
// A bucket must start with /volume/bucket_prefix
// and exclude keys /volume/bucket_xxx/key_xxx
// A bucket starts with /#volume/#bucket_prefix
MetadataKeyFilter filter = (preKey, currentKey, nextKey) -> {
if (currentKey != null) {
String bucketNamePrefix = getBucketKeyPrefix(volumeName, bucketPrefix);
String bucket = DFSUtil.bytes2String(currentKey);
return bucket.startsWith(bucketNamePrefix) &&
!bucket.replaceFirst(bucketNamePrefix, "")
.contains(OzoneConsts.KSM_KEY_PREFIX);
return bucket.startsWith(bucketNamePrefix);
}
return false;
};
List<Map.Entry<byte[], byte[]>> rangeResult;
if (!Strings.isNullOrEmpty(startBucket)) {
//Since we are excluding start key from the result,
// Since we are excluding start key from the result,
// the maxNumOfBuckets is incremented.
rangeResult = store.getRangeKVs(
rangeResult = store.getSequentialRangeKVs(
getBucketKey(volumeName, startBucket),
maxNumOfBuckets + 1, filter);
//Remove start key from result.
rangeResult.remove(0);
} else {
rangeResult = store.getRangeKVs(null, maxNumOfBuckets, filter);
rangeResult = store.getSequentialRangeKVs(null, maxNumOfBuckets, filter);
}
for (Map.Entry<byte[], byte[]> entry : rangeResult) {

View File

@ -63,6 +63,7 @@
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB;
import static org.apache.hadoop.ozone.OzoneConsts.KSM_DB_NAME;
import static org.apache.hadoop.ozone.OzoneConsts.KSM_USER_PREFIX;
import static org.apache.hadoop.ozone.OzoneConsts.KSM_BUCKET_PREFIX;
import static org.apache.hadoop.ozone.OzoneConsts.KSM_VOLUME_PREFIX;
import static org.apache.hadoop.ozone.OzoneConsts.NODEPOOL_DB;
import static org.apache.hadoop.ozone.OzoneConsts.OPEN_CONTAINERS_DB;
@ -385,7 +386,7 @@ private void convertKSMDB(Path dbPath, Path outPath) throws Exception {
try {
insertKSMDB(conn, type, keyString, value);
} catch (IOException | SQLException ex) {
LOG.error("Exception inserting key {}", keyString, ex);
LOG.error("Exception inserting key {} type {}", keyString, type, ex);
}
return true;
});
@ -445,18 +446,11 @@ private void insertKSMDB(Connection conn, KeyType type, String keyName,
private KeyType getKeyType(String key) {
if (key.startsWith(KSM_USER_PREFIX)) {
return KeyType.USER;
} else {
int count = key.length() - key.replace(KSM_VOLUME_PREFIX, "").length();
// NOTE : when delimiter gets changed, will need to change this part
if (count == 1) {
return KeyType.VOLUME;
} else if (count == 2) {
return KeyType.BUCKET;
} else if (count >= 3) {
} else if (key.startsWith(KSM_VOLUME_PREFIX)) {
return key.replaceFirst(KSM_VOLUME_PREFIX, "")
.contains(KSM_BUCKET_PREFIX) ? KeyType.BUCKET : KeyType.VOLUME;
}else {
return KeyType.KEY;
} else {
return KeyType.UNKNOWN;
}
}
}

View File

@ -36,6 +36,7 @@
import java.util.List;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Map;
import java.util.Map.Entry;
/**
@ -179,7 +180,7 @@ public ImmutablePair<byte[], byte[]> peekAround(int offset,
it.seek(from);
}
if (!it.hasNext()) {
throw new IOException("Key not found");
return null;
}
switch (offset) {
case 0:
@ -260,6 +261,20 @@ public void writeBatch(BatchOperation operation) throws IOException {
}
}
@Override
public List<Map.Entry<byte[], byte[]>> getRangeKVs(byte[] startKey,
int count, MetadataKeyFilters.MetadataKeyFilter... filters)
throws IOException, IllegalArgumentException {
return getRangeKVs(startKey, count, false, filters);
}
@Override
public List<Map.Entry<byte[], byte[]>> getSequentialRangeKVs(byte[] startKey,
int count, MetadataKeyFilters.MetadataKeyFilter... filters)
throws IOException, IllegalArgumentException {
return getRangeKVs(startKey, count, true, filters);
}
/**
* Returns a certain range of key value pairs as a list based on a
* startKey or count. Further a {@link MetadataKeyFilter} can be added to
@ -287,9 +302,9 @@ public void writeBatch(BatchOperation operation) throws IOException {
* @throws IOException if an invalid startKey is given or other I/O errors.
* @throws IllegalArgumentException if count is less than 0.
*/
@Override
public List<Entry<byte[], byte[]>> getRangeKVs(byte[] startKey,
int count, MetadataKeyFilter... filters) throws IOException {
private List<Entry<byte[], byte[]>> getRangeKVs(byte[] startKey,
int count, boolean sequential, MetadataKeyFilter... filters)
throws IOException {
List<Entry<byte[], byte[]>> result = new ArrayList<>();
long start = System.currentTimeMillis();
if (count < 0) {
@ -314,10 +329,21 @@ public List<Entry<byte[], byte[]>> getRangeKVs(byte[] startKey,
byte[] preKey = dbIter.hasPrev() ? dbIter.peekPrev().getKey() : null;
byte[] nextKey = dbIter.hasNext() ? dbIter.peekNext().getKey() : null;
Entry<byte[], byte[]> current = dbIter.next();
if (filters == null || Arrays.asList(filters).stream()
.allMatch(entry -> entry.filterKey(preKey,
current.getKey(), nextKey))) {
if (filters == null) {
result.add(current);
} else {
if (Arrays.asList(filters).stream().allMatch(
entry -> entry.filterKey(preKey, current.getKey(), nextKey))) {
result.add(current);
} else {
if (result.size() > 0 && sequential) {
// if the caller asks for a sequential range of results,
// and we met a dis-match, abort iteration from here.
// if result is empty, we continue to look for the first match.
break;
}
}
}
}
} finally {

View File

@ -98,6 +98,27 @@ List<Map.Entry<byte[], byte[]>> getRangeKVs(byte[] startKey,
int count, MetadataKeyFilter... filters)
throws IOException, IllegalArgumentException;
/**
* This method is very similar with
* {@link #getRangeKVs(byte[], int, MetadataKeyFilter...)}, the only
* different is this method is supposed to return a sequential range
* of elements based on the filters. While iterating the elements,
* if it met any entry that cannot pass the filter, the iterator will stop
* from this point without looking for next match. If no filter is given,
* this method behaves just like
* {@link #getRangeKVs(byte[], int, MetadataKeyFilter...)}.
*
* @param startKey a start key.
* @param count max number of entries to return.
* @param filters customized one or more {@link MetadataKeyFilter}.
* @return a list of entries found in the database.
* @throws IOException
* @throws IllegalArgumentException
*/
List<Map.Entry<byte[], byte[]>> getSequentialRangeKVs(byte[] startKey,
int count, MetadataKeyFilter... filters)
throws IOException, IllegalArgumentException;
/**
* A batch of PUT, DELETE operations handled as a single atomic write.
*

View File

@ -133,6 +133,20 @@ public void delete(byte[] key) throws IOException {
public List<Map.Entry<byte[], byte[]>> getRangeKVs(byte[] startKey,
int count, MetadataKeyFilters.MetadataKeyFilter... filters)
throws IOException, IllegalArgumentException {
return getRangeKVs(startKey, count, false, filters);
}
@Override
public List<Map.Entry<byte[], byte[]>> getSequentialRangeKVs(byte[] startKey,
int count, MetadataKeyFilters.MetadataKeyFilter... filters)
throws IOException, IllegalArgumentException {
return getRangeKVs(startKey, count, true, filters);
}
private List<Map.Entry<byte[], byte[]>> getRangeKVs(byte[] startKey,
int count, boolean sequential,
MetadataKeyFilters.MetadataKeyFilter... filters)
throws IOException, IllegalArgumentException {
List<Map.Entry<byte[], byte[]>> result = new ArrayList<>();
long start = System.currentTimeMillis();
if (count < 0) {
@ -161,11 +175,23 @@ public List<Map.Entry<byte[], byte[]>> getRangeKVs(byte[] startKey,
it.next();
final byte[] nextKey = it.isValid() ? it.key() : null;
if (filters == null || Arrays.asList(filters).stream()
if (filters == null) {
result.add(new AbstractMap.SimpleImmutableEntry<>(currentKey,
currentValue));
} else {
if (Arrays.asList(filters).stream()
.allMatch(entry -> entry.filterKey(prevKey,
currentKey, nextKey))) {
result.add(new AbstractMap.SimpleImmutableEntry<>(currentKey,
currentValue));
} else {
if (result.size() > 0 && sequential) {
// if the caller asks for a sequential range of results,
// and we met a dis-match, abort iteration from here.
// if result is empty, we continue to look for the first match.
break;
}
}
}
}
} finally {
@ -261,7 +287,7 @@ public ImmutablePair<byte[], byte[]> peekAround(int offset,
it.seek(from);
}
if (!it.isValid()) {
throw new IOException("Key not found");
return null;
}
switch (offset) {

View File

@ -21,6 +21,7 @@
import org.apache.commons.io.FileUtils;
import org.apache.commons.lang3.tuple.ImmutablePair;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.utils.BatchOperation;
import org.apache.hadoop.utils.MetadataStore;
@ -293,6 +294,25 @@ public void testGetRangeKVs() throws IOException {
Assert.assertEquals("a0", getString(result.get(0).getKey()));
}
@Test
public void testGetSequentialRangeKVs() throws IOException {
MetadataKeyFilter suffixFilter = (preKey, currentKey, nextKey)
-> DFSUtil.bytes2String(currentKey).endsWith("2");
// Suppose to return a2 and b2
List<Map.Entry<byte[], byte[]>> result =
store.getRangeKVs(null, MAX_GETRANGE_LENGTH, suffixFilter);
Assert.assertEquals(2, result.size());
Assert.assertEquals("a2", DFSUtil.bytes2String(result.get(0).getKey()));
Assert.assertEquals("b2", DFSUtil.bytes2String(result.get(1).getKey()));
// Suppose to return just a2, because when it iterates to a3,
// the filter no long matches and it should stop from there.
result = store.getSequentialRangeKVs(null,
MAX_GETRANGE_LENGTH, suffixFilter);
Assert.assertEquals(1, result.size());
Assert.assertEquals("a2", DFSUtil.bytes2String(result.get(0).getKey()));
}
@Test
public void testGetRangeLength() throws IOException {
List<Map.Entry<byte[], byte[]>> result = null;

View File

@ -80,9 +80,9 @@ private KSMMetadataManager getMetadataManagerMock(String... volumesToCreate)
@Override
public Boolean answer(InvocationOnMock invocation)
throws Throwable {
String keyRootName = OzoneConsts.KSM_VOLUME_PREFIX
String keyRootName = OzoneConsts.KSM_KEY_PREFIX
+ invocation.getArguments()[0]
+ OzoneConsts.KSM_BUCKET_PREFIX
+ OzoneConsts.KSM_KEY_PREFIX
+ invocation.getArguments()[1]
+ OzoneConsts.KSM_KEY_PREFIX;
Iterator<String> keyIterator = metadataDB.keySet().iterator();

View File

@ -124,6 +124,19 @@ static void runTestCreateBucket(OzoneRestClient client)
assertEquals(vol.getOwnerName(), "bilbo");
assertEquals(vol.getQuota().getUnit(), OzoneQuota.Units.TB);
assertEquals(vol.getQuota().getSize(), 100);
// Test create a bucket with invalid bucket name,
// not use Rule here because the test method is static.
try {
String invalidBucketName = "#" + OzoneUtils.getRequestID().toLowerCase();
vol.createBucket(invalidBucketName, acls, StorageType.DEFAULT);
fail("Except the bucket creation to be failed because the"
+ " bucket name starts with an invalid char #");
} catch (Exception e) {
assertTrue(e instanceof OzoneRestClientException);
assertTrue(e.getMessage().contains("Bucket or Volume name"
+ " has an unsupported character : #"));
}
}
@Test

View File

@ -37,7 +37,6 @@
import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.junit.*;
import org.mockito.Mockito;
import java.io.File;
@ -47,6 +46,11 @@
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import org.junit.BeforeClass;
import org.junit.AfterClass;
import org.junit.Test;
import org.junit.Ignore;
import static org.junit.Assert.fail;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
@ -125,6 +129,20 @@ static void runTestCreateVolume(OzoneRestClient client)
// verify the key creation time
assertTrue((OzoneUtils.formatDate(vol.getCreatedOn())
/ 1000) >= (currentTime / 1000));
// Test create a volume with invalid volume name,
// not use Rule here because the test method is static.
try {
String invalidVolumeName = "#" + OzoneUtils.getRequestID().toLowerCase();
client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
mockClient.createVolume(invalidVolumeName, "bilbo", "100TB");
fail("Except the volume creation be failed because the"
+ " volume name starts with an invalid char #");
} catch (Exception e) {
assertTrue(e instanceof OzoneRestClientException);
assertTrue(e.getMessage().contains("Bucket or Volume name"
+ " has an unsupported character : #"));
}
}
@Test
@ -239,7 +257,7 @@ static void runTestListVolumePagination(OzoneRestClient client)
prevKey = ovols.get(ovols.size() - 1);
pagecount++;
}
Assert.assertEquals(volCount / step, pagecount);
assertEquals(volCount / step, pagecount);
}
// TODO: remove @Ignore below once the problem has been resolved.
@ -275,7 +293,7 @@ static void runTestListAllVolumes(OzoneRestClient client)
}
// becasue we are querying an existing ozone store, there will
// be volumes created by other tests too. So we should get more page counts.
Assert.assertEquals(volCount / step, pagecount);
assertEquals(volCount / step, pagecount);
}
@Test
@ -382,12 +400,10 @@ private static List<CloseableHttpClient> mockHttpClients(
private static void verifyHttpConnectionClosed(
List<CloseableHttpClient> mockedHttpClients) {
final AtomicInteger totalCalled = new AtomicInteger();
Assert.assertTrue(mockedHttpClients.stream().allMatch(
closeableHttpClient -> {
assertTrue(mockedHttpClients.stream().allMatch(closeableHttpClient -> {
boolean clientUsed = false;
try {
verify(closeableHttpClient, times(1))
.execute(Mockito.any());
verify(closeableHttpClient, times(1)).execute(Mockito.any());
totalCalled.incrementAndGet();
clientUsed = true;
} catch (Throwable e) {
@ -399,8 +415,7 @@ private static void verifyHttpConnectionClosed(
if (clientUsed) {
try {
// If a client is used, ensure the close function is called.
verify(closeableHttpClient,
times(1)).close();
verify(closeableHttpClient, times(1)).close();
return true;
} catch (IOException e) {
return false;
@ -409,10 +424,8 @@ private static void verifyHttpConnectionClosed(
return true;
}
}));
System.out.println("Successful connections "
+ totalCalled.get());
Assert.assertTrue(
"The mocked http client should be called at least once.",
System.out.println("Successful connections " + totalCalled.get());
assertTrue("The mocked http client should be called at least once.",
totalCalled.get() > 0);
}
}