|
@@ -19,77 +19,178 @@ package org.apache.hadoop.ozone.om;
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Strings;
|
|
|
import com.google.common.collect.Lists;
|
|
|
-import org.apache.commons.lang3.tuple.ImmutablePair;
|
|
|
-import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
+import org.apache.commons.lang3.StringUtils;
|
|
|
import org.apache.hadoop.hdds.client.BlockID;
|
|
|
-import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
|
|
-import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
|
|
-import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
|
|
|
-import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
|
|
-import org.apache.hadoop.ozone.common.BlockGroup;
|
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
|
+import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
import org.apache.hadoop.ozone.OzoneConsts;
|
|
|
+import org.apache.hadoop.ozone.common.BlockGroup;
|
|
|
import org.apache.hadoop.ozone.om.exceptions.OMException;
|
|
|
import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
|
|
|
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
|
|
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
|
|
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
|
|
|
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketInfo;
|
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
|
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeInfo;
|
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
|
|
|
-
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
-import org.apache.hadoop.utils.BatchOperation;
|
|
|
-import org.apache.hadoop.utils.MetadataKeyFilters;
|
|
|
-import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
|
|
|
-import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter;
|
|
|
-import org.apache.hadoop.utils.MetadataStore;
|
|
|
-import org.apache.hadoop.utils.MetadataStoreBuilder;
|
|
|
+import org.apache.hadoop.utils.db.DBStore;
|
|
|
+import org.apache.hadoop.utils.db.DBStoreBuilder;
|
|
|
+import org.apache.hadoop.utils.db.Table;
|
|
|
+import org.apache.hadoop.utils.db.TableIterator;
|
|
|
+import org.eclipse.jetty.util.StringUtil;
|
|
|
+import org.slf4j.Logger;
|
|
|
+import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
+import java.nio.file.Paths;
|
|
|
+import java.util.ArrayList;
|
|
|
+import java.util.Arrays;
|
|
|
import java.util.Collections;
|
|
|
import java.util.List;
|
|
|
-import java.util.ArrayList;
|
|
|
import java.util.Map;
|
|
|
import java.util.concurrent.locks.Lock;
|
|
|
import java.util.concurrent.locks.ReadWriteLock;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
import java.util.stream.Collectors;
|
|
|
|
|
|
+import static org.apache.hadoop.hdds.server.ServerUtils.getOzoneMetaDirPath;
|
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS;
|
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS_DEFAULT;
|
|
|
-import static org.apache.hadoop.ozone.OzoneConsts.DELETING_KEY_PREFIX;
|
|
|
import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;
|
|
|
-import static org.apache.hadoop.ozone.OzoneConsts.OPEN_KEY_ID_DELIMINATOR;
|
|
|
-import static org.apache.hadoop.ozone.OzoneConsts.OPEN_KEY_PREFIX;
|
|
|
-import static org.apache.hadoop.ozone.om.OMConfigKeys
|
|
|
- .OZONE_OM_DB_CACHE_SIZE_DEFAULT;
|
|
|
-import static org.apache.hadoop.ozone.om.OMConfigKeys
|
|
|
- .OZONE_OM_DB_CACHE_SIZE_MB;
|
|
|
-import static org.apache.hadoop.hdds.server.ServerUtils.getOzoneMetaDirPath;
|
|
|
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
|
|
|
|
|
|
/**
|
|
|
* Ozone metadata manager interface.
|
|
|
*/
|
|
|
public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
+ private static final Logger LOG =
|
|
|
+ LoggerFactory.getLogger(OmMetadataManagerImpl.class);
|
|
|
+
|
|
|
+ /**
|
|
|
+ * OM RocksDB Structure .
|
|
|
+ * <p>
|
|
|
+ * OM DB stores metadata as KV pairs in different column families.
|
|
|
+ * <p>
|
|
|
+ * OM DB Schema:
|
|
|
+ * |-------------------------------------------------------------------|
|
|
|
+ * | Column Family | VALUE |
|
|
|
+ * |-------------------------------------------------------------------|
|
|
|
+ * | userTable | user->VolumeList |
|
|
|
+ * |-------------------------------------------------------------------|
|
|
|
+ * | volumeTable | /volume->VolumeInfo |
|
|
|
+ * |-------------------------------------------------------------------|
|
|
|
+ * | bucketTable | /volume/bucket-> BucketInfo |
|
|
|
+ * |-------------------------------------------------------------------|
|
|
|
+ * | keyTable | /volumeName/bucketName/keyName->KeyInfo |
|
|
|
+ * |-------------------------------------------------------------------|
|
|
|
+ * | deletedTable | /volumeName/bucketName/keyName->KeyInfo |
|
|
|
+ * |-------------------------------------------------------------------|
|
|
|
+ * | openKey | /volumeName/bucketName/keyName/id->KeyInfo |
|
|
|
+ * |-------------------------------------------------------------------|
|
|
|
+ */
|
|
|
+
|
|
|
+ private static final String USER_TABLE = "userTable";
|
|
|
+ private static final String VOLUME_TABLE = "volumeTable";
|
|
|
+ private static final String BUCKET_TABLE = "bucketTable";
|
|
|
+ private static final String KEY_TABLE = "keyTable";
|
|
|
+ private static final String DELETED_TABLE = "deletedTable";
|
|
|
+ private static final String OPEN_KEY_TABLE = "openKeyTable";
|
|
|
|
|
|
- private final MetadataStore store;
|
|
|
+ private final DBStore store;
|
|
|
+
|
|
|
+ // TODO: Make this lock move into Table instead of *ONE* lock for the whole
|
|
|
+ // DB.
|
|
|
private final ReadWriteLock lock;
|
|
|
private final long openKeyExpireThresholdMS;
|
|
|
|
|
|
+ private final Table userTable;
|
|
|
+ private final Table volumeTable;
|
|
|
+ private final Table bucketTable;
|
|
|
+ private final Table keyTable;
|
|
|
+ private final Table deletedTable;
|
|
|
+ private final Table openKeyTable;
|
|
|
+
|
|
|
public OmMetadataManagerImpl(OzoneConfiguration conf) throws IOException {
|
|
|
File metaDir = getOzoneMetaDirPath(conf);
|
|
|
- final int cacheSize = conf.getInt(OZONE_OM_DB_CACHE_SIZE_MB,
|
|
|
- OZONE_OM_DB_CACHE_SIZE_DEFAULT);
|
|
|
- File omDBFile = new File(metaDir.getPath(), OM_DB_NAME);
|
|
|
- this.store = MetadataStoreBuilder.newBuilder()
|
|
|
- .setConf(conf)
|
|
|
- .setDbFile(omDBFile)
|
|
|
- .setCacheSize(cacheSize * OzoneConsts.MB)
|
|
|
- .build();
|
|
|
this.lock = new ReentrantReadWriteLock();
|
|
|
this.openKeyExpireThresholdMS = 1000 * conf.getInt(
|
|
|
OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS,
|
|
|
OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS_DEFAULT);
|
|
|
+
|
|
|
+ this.store = DBStoreBuilder.newBuilder(conf)
|
|
|
+ .setName(OM_DB_NAME)
|
|
|
+ .setPath(Paths.get(metaDir.getPath()))
|
|
|
+ .addTable(USER_TABLE)
|
|
|
+ .addTable(VOLUME_TABLE)
|
|
|
+ .addTable(BUCKET_TABLE)
|
|
|
+ .addTable(KEY_TABLE)
|
|
|
+ .addTable(DELETED_TABLE)
|
|
|
+ .addTable(OPEN_KEY_TABLE)
|
|
|
+ .build();
|
|
|
+
|
|
|
+ userTable = this.store.getTable(USER_TABLE);
|
|
|
+ checkTableStatus(userTable, USER_TABLE);
|
|
|
+
|
|
|
+ volumeTable = this.store.getTable(VOLUME_TABLE);
|
|
|
+ checkTableStatus(volumeTable, VOLUME_TABLE);
|
|
|
+
|
|
|
+ bucketTable = this.store.getTable(BUCKET_TABLE);
|
|
|
+ checkTableStatus(bucketTable, BUCKET_TABLE);
|
|
|
+
|
|
|
+ keyTable = this.store.getTable(KEY_TABLE);
|
|
|
+ checkTableStatus(keyTable, KEY_TABLE);
|
|
|
+
|
|
|
+ deletedTable = this.store.getTable(DELETED_TABLE);
|
|
|
+ checkTableStatus(deletedTable, DELETED_TABLE);
|
|
|
+
|
|
|
+ openKeyTable = this.store.getTable(OPEN_KEY_TABLE);
|
|
|
+ checkTableStatus(openKeyTable, OPEN_KEY_TABLE);
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public Table getUserTable() {
|
|
|
+ return userTable;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public Table getVolumeTable() {
|
|
|
+ return volumeTable;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public Table getBucketTable() {
|
|
|
+ return bucketTable;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public Table getKeyTable() {
|
|
|
+ return keyTable;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public Table getDeletedTable() {
|
|
|
+ return deletedTable;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public Table getOpenKeyTable() {
|
|
|
+ return openKeyTable;
|
|
|
+ }
|
|
|
+
|
|
|
+ private void checkTableStatus(Table table, String name) throws IOException {
|
|
|
+ String logMessage = "Unable to get a reference to %s table. Cannot " +
|
|
|
+ "continue.";
|
|
|
+ String errMsg = "Inconsistent DB state, Table - %s. Please check the logs" +
|
|
|
+ "for more info.";
|
|
|
+ if (table == null) {
|
|
|
+ LOG.error(String.format(logMessage, name));
|
|
|
+ throw new IOException(String.format(errMsg, name));
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -104,7 +205,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
* Stop metadata manager.
|
|
|
*/
|
|
|
@Override
|
|
|
- public void stop() throws IOException {
|
|
|
+ public void stop() throws Exception {
|
|
|
if (store != null) {
|
|
|
store.close();
|
|
|
}
|
|
@@ -112,86 +213,75 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
|
|
|
/**
|
|
|
* Get metadata store.
|
|
|
+ *
|
|
|
* @return store - metadata store.
|
|
|
*/
|
|
|
@VisibleForTesting
|
|
|
@Override
|
|
|
- public MetadataStore getStore() {
|
|
|
+ public DBStore getStore() {
|
|
|
return store;
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Given a volume return the corresponding DB key.
|
|
|
+ *
|
|
|
* @param volume - Volume name
|
|
|
*/
|
|
|
+ @Override
|
|
|
public byte[] getVolumeKey(String volume) {
|
|
|
- String dbVolumeName = OzoneConsts.OM_VOLUME_PREFIX + volume;
|
|
|
- return DFSUtil.string2Bytes(dbVolumeName);
|
|
|
+ return DFSUtil.string2Bytes(OzoneConsts.OM_KEY_PREFIX + volume);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Given a user return the corresponding DB key.
|
|
|
+ *
|
|
|
* @param user - User name
|
|
|
*/
|
|
|
+ @Override
|
|
|
public byte[] getUserKey(String user) {
|
|
|
- String dbUserName = OzoneConsts.OM_USER_PREFIX + user;
|
|
|
- return DFSUtil.string2Bytes(dbUserName);
|
|
|
+ return DFSUtil.string2Bytes(user);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Given a volume and bucket, return the corresponding DB key.
|
|
|
+ *
|
|
|
* @param volume - User name
|
|
|
* @param bucket - Bucket name
|
|
|
*/
|
|
|
+ @Override
|
|
|
public byte[] getBucketKey(String volume, String bucket) {
|
|
|
- String bucketKeyString = OzoneConsts.OM_VOLUME_PREFIX + volume
|
|
|
- + OzoneConsts.OM_BUCKET_PREFIX + bucket;
|
|
|
- return DFSUtil.string2Bytes(bucketKeyString);
|
|
|
- }
|
|
|
+ StringBuilder builder =
|
|
|
+ new StringBuilder().append(OM_KEY_PREFIX).append(volume);
|
|
|
|
|
|
- /**
|
|
|
- * @param volume
|
|
|
- * @param bucket
|
|
|
- * @return
|
|
|
- */
|
|
|
- private String getBucketWithDBPrefix(String volume, String bucket) {
|
|
|
- StringBuffer sb = new StringBuffer();
|
|
|
- sb.append(OzoneConsts.OM_VOLUME_PREFIX)
|
|
|
- .append(volume)
|
|
|
- .append(OzoneConsts.OM_BUCKET_PREFIX);
|
|
|
- if (!Strings.isNullOrEmpty(bucket)) {
|
|
|
- sb.append(bucket);
|
|
|
+ if (StringUtils.isNotBlank(bucket)) {
|
|
|
+ builder.append(OM_KEY_PREFIX).append(bucket);
|
|
|
}
|
|
|
- return sb.toString();
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public String getKeyWithDBPrefix(String volume, String bucket, String key) {
|
|
|
- String keyVB = OzoneConsts.OM_KEY_PREFIX + volume
|
|
|
- + OzoneConsts.OM_KEY_PREFIX + bucket
|
|
|
- + OzoneConsts.OM_KEY_PREFIX;
|
|
|
- return Strings.isNullOrEmpty(key) ? keyVB : keyVB + key;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public byte[] getDBKeyBytes(String volume, String bucket, String key) {
|
|
|
- return DFSUtil.string2Bytes(getKeyWithDBPrefix(volume, bucket, key));
|
|
|
+ return DFSUtil.string2Bytes(builder.toString());
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public byte[] getDeletedKeyName(byte[] keyName) {
|
|
|
- return DFSUtil.string2Bytes(
|
|
|
- DELETING_KEY_PREFIX + DFSUtil.bytes2String(keyName));
|
|
|
+ public byte[] getOzoneKeyBytes(String volume, String bucket, String key) {
|
|
|
+ StringBuilder builder = new StringBuilder()
|
|
|
+ .append(OM_KEY_PREFIX).append(volume);
|
|
|
+ // TODO : Throw if the Bucket is null?
|
|
|
+ builder.append(OM_KEY_PREFIX).append(bucket);
|
|
|
+ if (StringUtil.isNotBlank(key)) {
|
|
|
+ builder.append(OM_KEY_PREFIX).append(key);
|
|
|
+ }
|
|
|
+ return DFSUtil.string2Bytes(builder.toString());
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public byte[] getOpenKeyNameBytes(String keyName, int id) {
|
|
|
- return DFSUtil.string2Bytes(OPEN_KEY_PREFIX + id +
|
|
|
- OPEN_KEY_ID_DELIMINATOR + keyName);
|
|
|
+ public byte[] getOpenKeyBytes(String volume, String bucket,
|
|
|
+ String key, long id) {
|
|
|
+ String openKey = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket +
|
|
|
+ OM_KEY_PREFIX + key + OM_KEY_PREFIX + id;
|
|
|
+ return DFSUtil.string2Bytes(openKey);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Returns the read lock used on Metadata DB.
|
|
|
+ *
|
|
|
* @return readLock
|
|
|
*/
|
|
|
@Override
|
|
@@ -201,6 +291,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
|
|
|
/**
|
|
|
* Returns the write lock used on Metadata DB.
|
|
|
+ *
|
|
|
* @return writeLock
|
|
|
*/
|
|
|
@Override
|
|
@@ -209,71 +300,79 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Returns the value associated with this key.
|
|
|
- * @param key - key
|
|
|
- * @return value
|
|
|
+ * Returns true if the firstArray startsWith the bytes of secondArray.
|
|
|
+ *
|
|
|
+ * @param firstArray - Byte array
|
|
|
+ * @param secondArray - Byte array
|
|
|
+ * @return true if the first array bytes match the bytes in the second array.
|
|
|
*/
|
|
|
- @Override
|
|
|
- public byte[] get(byte[] key) throws IOException {
|
|
|
- return store.get(key);
|
|
|
- }
|
|
|
+ private boolean startsWith(byte[] firstArray, byte[] secondArray) {
|
|
|
|
|
|
- /**
|
|
|
- * Puts a Key into Metadata DB.
|
|
|
- * @param key - key
|
|
|
- * @param value - value
|
|
|
- */
|
|
|
- @Override
|
|
|
- public void put(byte[] key, byte[] value) throws IOException {
|
|
|
- store.put(key, value);
|
|
|
- }
|
|
|
+ if (firstArray == null) {
|
|
|
+ // if both are null, then the arrays match, else if first is null and
|
|
|
+ // second is not, then this function returns false.
|
|
|
+ return secondArray == null;
|
|
|
+ }
|
|
|
|
|
|
- /**
|
|
|
- * Deletes a Key from Metadata DB.
|
|
|
- * @param key - key
|
|
|
- */
|
|
|
- public void delete(byte[] key) throws IOException {
|
|
|
- store.delete(key);
|
|
|
- }
|
|
|
|
|
|
- @Override
|
|
|
- public void writeBatch(BatchOperation batch) throws IOException {
|
|
|
- this.store.writeBatch(batch);
|
|
|
+ if (secondArray != null) {
|
|
|
+ // If the second array is longer then first array cannot be starting with
|
|
|
+ // the bytes of second array.
|
|
|
+ if (secondArray.length > firstArray.length) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ for (int ndx = 0; ndx < secondArray.length; ndx++) {
|
|
|
+ if (firstArray[ndx] != secondArray[ndx]) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return true; //match, return true.
|
|
|
+ }
|
|
|
+ return false; // if first is not null and second is null, we define that
|
|
|
+ // array does not start with same chars.
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Given a volume, check if it is empty, i.e there are no buckets inside it.
|
|
|
+ * We iterate in the bucket table and see if there is any key that starts with
|
|
|
+ * the volume prefix. We actually look for /volume/, since if we don't have
|
|
|
+ * the trailing slash it is possible that we might match some other volume.
|
|
|
+ * <p>
|
|
|
+ * For example, vol1 and vol122 might match, to avoid that we look for /vol1/
|
|
|
+ *
|
|
|
* @param volume - Volume name
|
|
|
* @return true if the volume is empty
|
|
|
*/
|
|
|
+ @Override
|
|
|
public boolean isVolumeEmpty(String volume) throws IOException {
|
|
|
- String dbVolumeRootName = OzoneConsts.OM_VOLUME_PREFIX + volume
|
|
|
- + OzoneConsts.OM_BUCKET_PREFIX;
|
|
|
- byte[] dbVolumeRootKey = DFSUtil.string2Bytes(dbVolumeRootName);
|
|
|
- ImmutablePair<byte[], byte[]> volumeRoot =
|
|
|
- store.peekAround(0, dbVolumeRootKey);
|
|
|
- if (volumeRoot != null) {
|
|
|
- return !DFSUtil.bytes2String(volumeRoot.getKey())
|
|
|
- .startsWith(dbVolumeRootName);
|
|
|
+ byte[] volumePrefix = getVolumeKey(volume + OM_KEY_PREFIX);
|
|
|
+ try (TableIterator<Table.KeyValue> bucketIter = bucketTable.iterator()) {
|
|
|
+ Table.KeyValue kv = bucketIter.seek(volumePrefix);
|
|
|
+ if (kv != null && startsWith(kv.getKey(), volumePrefix)) {
|
|
|
+ return false; // we found at least one bucket with this volume prefix.
|
|
|
+ }
|
|
|
}
|
|
|
return true;
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Given a volume/bucket, check if it is empty,
|
|
|
- * i.e there are no keys inside it.
|
|
|
+ * Given a volume/bucket, check if it is empty, i.e there are no keys inside
|
|
|
+ * it. Prefix is /volume/bucket/, and we lookup the keyTable.
|
|
|
+ *
|
|
|
* @param volume - Volume name
|
|
|
* @param bucket - Bucket name
|
|
|
* @return true if the bucket is empty
|
|
|
*/
|
|
|
+ @Override
|
|
|
public boolean isBucketEmpty(String volume, String bucket)
|
|
|
throws IOException {
|
|
|
- String keyRootName = getKeyWithDBPrefix(volume, bucket, null);
|
|
|
- byte[] keyRoot = DFSUtil.string2Bytes(keyRootName);
|
|
|
- ImmutablePair<byte[], byte[]> firstKey = store.peekAround(0, keyRoot);
|
|
|
- if (firstKey != null) {
|
|
|
- return !DFSUtil.bytes2String(firstKey.getKey())
|
|
|
- .startsWith(keyRootName);
|
|
|
+ byte[] keyPrefix = getBucketKey(volume, bucket + OM_KEY_PREFIX);
|
|
|
+ try (TableIterator<Table.KeyValue> keyIter = keyTable.iterator()) {
|
|
|
+ Table.KeyValue kv = keyIter.seek(keyPrefix);
|
|
|
+ if (kv != null && startsWith(kv.getKey(), keyPrefix)) {
|
|
|
+ return false; // we found at least one key with this vol/bucket prefix.
|
|
|
+ }
|
|
|
}
|
|
|
return true;
|
|
|
}
|
|
@@ -283,8 +382,8 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
*/
|
|
|
@Override
|
|
|
public List<OmBucketInfo> listBuckets(final String volumeName,
|
|
|
- final String startBucket, final String bucketPrefix,
|
|
|
- final int maxNumOfBuckets) throws IOException {
|
|
|
+ final String startBucket, final String bucketPrefix,
|
|
|
+ final int maxNumOfBuckets) throws IOException {
|
|
|
List<OmBucketInfo> result = new ArrayList<>();
|
|
|
if (Strings.isNullOrEmpty(volumeName)) {
|
|
|
throw new OMException("Volume name is required.",
|
|
@@ -292,49 +391,61 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
}
|
|
|
|
|
|
byte[] volumeNameBytes = getVolumeKey(volumeName);
|
|
|
- if (store.get(volumeNameBytes) == null) {
|
|
|
+ if (volumeTable.get(volumeNameBytes) == null) {
|
|
|
throw new OMException("Volume " + volumeName + " not found.",
|
|
|
ResultCodes.FAILED_VOLUME_NOT_FOUND);
|
|
|
}
|
|
|
|
|
|
|
|
|
- // A bucket starts with /#volume/#bucket_prefix
|
|
|
- MetadataKeyFilter filter = (preKey, currentKey, nextKey) -> {
|
|
|
- if (currentKey != null) {
|
|
|
- String bucketNamePrefix =
|
|
|
- getBucketWithDBPrefix(volumeName, bucketPrefix);
|
|
|
- String bucket = DFSUtil.bytes2String(currentKey);
|
|
|
- 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,
|
|
|
- // the maxNumOfBuckets is incremented.
|
|
|
- rangeResult = store.getSequentialRangeKVs(
|
|
|
- getBucketKey(volumeName, startBucket),
|
|
|
- maxNumOfBuckets + 1, filter);
|
|
|
- if (!rangeResult.isEmpty()) {
|
|
|
- //Remove start key from result.
|
|
|
- rangeResult.remove(0);
|
|
|
- }
|
|
|
+ byte[] startKey;
|
|
|
+ boolean skipStartKey = false;
|
|
|
+ if (StringUtil.isNotBlank(startBucket)) {
|
|
|
+ // if the user has specified a start key, we need to seek to that key
|
|
|
+ // and avoid that key in the response set.
|
|
|
+ startKey = getBucketKey(volumeName, startBucket);
|
|
|
+ skipStartKey = true;
|
|
|
} else {
|
|
|
- rangeResult = store.getSequentialRangeKVs(null, maxNumOfBuckets, filter);
|
|
|
+ // If the user has specified a prefix key, we need to get to the first
|
|
|
+ // of the keys with the prefix match. We can leverage RocksDB to do that.
|
|
|
+ // However, if the user has specified only a prefix, we cannot skip
|
|
|
+ // the first prefix key we see, the boolean skipStartKey allows us to
|
|
|
+ // skip the startkey or not depending on what patterns are specified.
|
|
|
+ startKey = getBucketKey(volumeName, bucketPrefix);
|
|
|
}
|
|
|
|
|
|
- for (Map.Entry<byte[], byte[]> entry : rangeResult) {
|
|
|
- OmBucketInfo info = OmBucketInfo.getFromProtobuf(
|
|
|
- BucketInfo.parseFrom(entry.getValue()));
|
|
|
- result.add(info);
|
|
|
+ byte[] seekPrefix;
|
|
|
+ if (StringUtil.isNotBlank(bucketPrefix)) {
|
|
|
+ seekPrefix = getBucketKey(volumeName, bucketPrefix);
|
|
|
+ } else {
|
|
|
+ seekPrefix = getVolumeKey(volumeName + OM_KEY_PREFIX);
|
|
|
+ }
|
|
|
+ int currentCount = 0;
|
|
|
+ try (TableIterator<Table.KeyValue> bucketIter = bucketTable.iterator()) {
|
|
|
+ Table.KeyValue kv = bucketIter.seek(startKey);
|
|
|
+ while (currentCount < maxNumOfBuckets && bucketIter.hasNext()) {
|
|
|
+ kv = bucketIter.next();
|
|
|
+ // Skip the Start Bucket if needed.
|
|
|
+ if (kv != null && skipStartKey &&
|
|
|
+ Arrays.equals(kv.getKey(), startKey)) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ if (kv != null && startsWith(kv.getKey(), seekPrefix)) {
|
|
|
+ result.add(OmBucketInfo.getFromProtobuf(
|
|
|
+ BucketInfo.parseFrom(kv.getValue())));
|
|
|
+ currentCount++;
|
|
|
+ } else {
|
|
|
+ // The SeekPrefix does not match any more, we can break out of the
|
|
|
+ // loop.
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
return result;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public List<OmKeyInfo> listKeys(String volumeName, String bucketName,
|
|
|
- String startKey, String keyPrefix, int maxKeys) throws IOException {
|
|
|
+ String startKey, String keyPrefix, int maxKeys) throws IOException {
|
|
|
List<OmKeyInfo> result = new ArrayList<>();
|
|
|
if (Strings.isNullOrEmpty(volumeName)) {
|
|
|
throw new OMException("Volume name is required.",
|
|
@@ -347,47 +458,61 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
}
|
|
|
|
|
|
byte[] bucketNameBytes = getBucketKey(volumeName, bucketName);
|
|
|
- if (store.get(bucketNameBytes) == null) {
|
|
|
+ if (getBucketTable().get(bucketNameBytes) == null) {
|
|
|
throw new OMException("Bucket " + bucketName + " not found.",
|
|
|
ResultCodes.FAILED_BUCKET_NOT_FOUND);
|
|
|
}
|
|
|
|
|
|
- MetadataKeyFilter filter = new KeyPrefixFilter()
|
|
|
- .addFilter(getKeyWithDBPrefix(volumeName, bucketName, keyPrefix));
|
|
|
-
|
|
|
- List<Map.Entry<byte[], byte[]>> rangeResult;
|
|
|
- if (!Strings.isNullOrEmpty(startKey)) {
|
|
|
- //Since we are excluding start key from the result,
|
|
|
- // the maxNumOfBuckets is incremented.
|
|
|
- rangeResult = store.getSequentialRangeKVs(
|
|
|
- getDBKeyBytes(volumeName, bucketName, startKey),
|
|
|
- maxKeys + 1, filter);
|
|
|
- if (!rangeResult.isEmpty()) {
|
|
|
- //Remove start key from result.
|
|
|
- rangeResult.remove(0);
|
|
|
- }
|
|
|
+ byte[] seekKey;
|
|
|
+ boolean skipStartKey = false;
|
|
|
+ if (StringUtil.isNotBlank(startKey)) {
|
|
|
+ // Seek to the specified key.
|
|
|
+ seekKey = getOzoneKeyBytes(volumeName, bucketName, startKey);
|
|
|
+ skipStartKey = true;
|
|
|
} else {
|
|
|
- rangeResult = store.getSequentialRangeKVs(null, maxKeys, filter);
|
|
|
+ // This allows us to seek directly to the first key with the right prefix.
|
|
|
+ seekKey = getOzoneKeyBytes(volumeName, bucketName, keyPrefix);
|
|
|
}
|
|
|
|
|
|
- for (Map.Entry<byte[], byte[]> entry : rangeResult) {
|
|
|
- OmKeyInfo info = OmKeyInfo.getFromProtobuf(
|
|
|
- KeyInfo.parseFrom(entry.getValue()));
|
|
|
- result.add(info);
|
|
|
+ byte[] seekPrefix;
|
|
|
+ if (StringUtil.isNotBlank(keyPrefix)) {
|
|
|
+ seekPrefix = getOzoneKeyBytes(volumeName, bucketName, keyPrefix);
|
|
|
+ } else {
|
|
|
+ seekPrefix = getBucketKey(volumeName, bucketName + OM_KEY_PREFIX);
|
|
|
+ }
|
|
|
+ int currentCount = 0;
|
|
|
+ try (TableIterator<Table.KeyValue> keyIter = getKeyTable().iterator()) {
|
|
|
+ Table.KeyValue kv = keyIter.seek(seekKey);
|
|
|
+ while (currentCount < maxKeys && keyIter.hasNext()) {
|
|
|
+ kv = keyIter.next();
|
|
|
+ // Skip the Start key if needed.
|
|
|
+ if (kv != null && skipStartKey && Arrays.equals(kv.getKey(), seekKey)) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ if (kv != null && startsWith(kv.getKey(), seekPrefix)) {
|
|
|
+ result.add(OmKeyInfo.getFromProtobuf(
|
|
|
+ KeyInfo.parseFrom(kv.getValue())));
|
|
|
+ currentCount++;
|
|
|
+ } else {
|
|
|
+ // The SeekPrefix does not match any more, we can break out of the
|
|
|
+ // loop.
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
return result;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public List<OmVolumeArgs> listVolumes(String userName,
|
|
|
- String prefix, String startKey, int maxKeys) throws IOException {
|
|
|
+ String prefix, String startKey, int maxKeys) throws IOException {
|
|
|
List<OmVolumeArgs> result = Lists.newArrayList();
|
|
|
VolumeList volumes;
|
|
|
- if (Strings.isNullOrEmpty(userName)) {
|
|
|
- volumes = getAllVolumes();
|
|
|
- } else {
|
|
|
- volumes = getVolumesByUser(userName);
|
|
|
+ if (StringUtil.isBlank(userName)) {
|
|
|
+ throw new OMException("User name is required to list Volumes.",
|
|
|
+ ResultCodes.FAILED_USER_NOT_FOUND);
|
|
|
}
|
|
|
+ volumes = getVolumesByUser(userName);
|
|
|
|
|
|
if (volumes == null || volumes.getVolumeNamesCount() == 0) {
|
|
|
return result;
|
|
@@ -406,7 +531,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
continue;
|
|
|
}
|
|
|
if (startKeyFound && result.size() < maxKeys) {
|
|
|
- byte[] volumeInfo = store.get(this.getVolumeKey(volumeName));
|
|
|
+ byte[] volumeInfo = getVolumeTable().get(this.getVolumeKey(volumeName));
|
|
|
if (volumeInfo == null) {
|
|
|
// Could not get volume info by given volume name,
|
|
|
// since the volume name is loaded from db,
|
|
@@ -433,7 +558,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
throws OMException {
|
|
|
VolumeList volumes = null;
|
|
|
try {
|
|
|
- byte[] volumesInBytes = store.get(userNameKey);
|
|
|
+ byte[] volumesInBytes = getUserTable().get(userNameKey);
|
|
|
if (volumesInBytes == null) {
|
|
|
// No volume found for this user, return an empty list
|
|
|
return VolumeList.newBuilder().build();
|
|
@@ -447,32 +572,12 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
return volumes;
|
|
|
}
|
|
|
|
|
|
- private VolumeList getAllVolumes() throws IOException {
|
|
|
- // Scan all users in database
|
|
|
- KeyPrefixFilter filter =
|
|
|
- new KeyPrefixFilter().addFilter(OzoneConsts.OM_USER_PREFIX);
|
|
|
- // We are not expecting a huge number of users per cluster,
|
|
|
- // it should be fine to scan all users in db and return us a
|
|
|
- // list of volume names in string per user.
|
|
|
- List<Map.Entry<byte[], byte[]>> rangeKVs = store
|
|
|
- .getSequentialRangeKVs(null, Integer.MAX_VALUE, filter);
|
|
|
-
|
|
|
- VolumeList.Builder builder = VolumeList.newBuilder();
|
|
|
- for (Map.Entry<byte[], byte[]> entry : rangeKVs) {
|
|
|
- VolumeList volumes = this.getVolumesByUser(entry.getKey());
|
|
|
- builder.addAllVolumeNames(volumes.getVolumeNamesList());
|
|
|
- }
|
|
|
-
|
|
|
- return builder.build();
|
|
|
- }
|
|
|
-
|
|
|
@Override
|
|
|
public List<BlockGroup> getPendingDeletionKeys(final int count)
|
|
|
throws IOException {
|
|
|
List<BlockGroup> keyBlocksList = Lists.newArrayList();
|
|
|
- List<Map.Entry<byte[], byte[]>> rangeResult =
|
|
|
- store.getRangeKVs(null, count,
|
|
|
- MetadataKeyFilters.getDeletingKeyFilter());
|
|
|
+ // TODO: Fix this later, Not part of this patch.
|
|
|
+ List<Map.Entry<byte[], byte[]>> rangeResult = Collections.emptyList();
|
|
|
for (Map.Entry<byte[], byte[]> entry : rangeResult) {
|
|
|
OmKeyInfo info =
|
|
|
OmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(entry.getValue()));
|
|
@@ -482,7 +587,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
return Collections.emptyList();
|
|
|
}
|
|
|
List<BlockID> item = latest.getLocationList().stream()
|
|
|
- .map(b->new BlockID(b.getContainerID(), b.getLocalID()))
|
|
|
+ .map(b -> new BlockID(b.getContainerID(), b.getLocalID()))
|
|
|
.collect(Collectors.toList());
|
|
|
BlockGroup keyBlocks = BlockGroup.newBuilder()
|
|
|
.setKeyName(DFSUtil.bytes2String(entry.getKey()))
|
|
@@ -497,11 +602,9 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
public List<BlockGroup> getExpiredOpenKeys() throws IOException {
|
|
|
List<BlockGroup> keyBlocksList = Lists.newArrayList();
|
|
|
long now = Time.now();
|
|
|
- final MetadataKeyFilter openKeyFilter =
|
|
|
- new KeyPrefixFilter().addFilter(OPEN_KEY_PREFIX);
|
|
|
- List<Map.Entry<byte[], byte[]>> rangeResult =
|
|
|
- store.getSequentialRangeKVs(null, Integer.MAX_VALUE,
|
|
|
- openKeyFilter);
|
|
|
+ // TODO: Fix the getExpiredOpenKeys, Not part of this patch.
|
|
|
+ List<Map.Entry<byte[], byte[]>> rangeResult = Collections.emptyList();
|
|
|
+
|
|
|
for (Map.Entry<byte[], byte[]> entry : rangeResult) {
|
|
|
OmKeyInfo info =
|
|
|
OmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(entry.getValue()));
|
|
@@ -513,7 +616,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
|
|
|
// Get block keys as a list.
|
|
|
List<BlockID> item = info.getLatestVersionLocations()
|
|
|
.getBlocksLatestVersionOnly().stream()
|
|
|
- .map(b->new BlockID(b.getContainerID(), b.getLocalID()))
|
|
|
+ .map(b -> new BlockID(b.getContainerID(), b.getLocalID()))
|
|
|
.collect(Collectors.toList());
|
|
|
BlockGroup keyBlocks = BlockGroup.newBuilder()
|
|
|
.setKeyName(DFSUtil.bytes2String(entry.getKey()))
|