|
@@ -17,13 +17,18 @@
|
|
*/
|
|
*/
|
|
package org.apache.hadoop.ozone.web.localstorage;
|
|
package org.apache.hadoop.ozone.web.localstorage;
|
|
|
|
|
|
-import org.apache.commons.logging.Log;
|
|
|
|
-import org.apache.commons.logging.LogFactory;
|
|
|
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
|
|
|
|
+import org.apache.hadoop.ozone.web.handlers.ListArgs;
|
|
|
|
+import org.apache.hadoop.ozone.web.response.KeyInfo;
|
|
|
|
+import org.apache.hadoop.ozone.web.response.ListKeys;
|
|
|
|
+import org.slf4j.LoggerFactory;
|
|
|
|
+import org.slf4j.Logger;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
|
|
import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
|
|
import org.apache.hadoop.ozone.web.exceptions.OzoneException;
|
|
import org.apache.hadoop.ozone.web.exceptions.OzoneException;
|
|
import org.apache.hadoop.ozone.web.handlers.BucketArgs;
|
|
import org.apache.hadoop.ozone.web.handlers.BucketArgs;
|
|
|
|
+import org.apache.hadoop.ozone.web.handlers.KeyArgs;
|
|
import org.apache.hadoop.ozone.web.handlers.UserArgs;
|
|
import org.apache.hadoop.ozone.web.handlers.UserArgs;
|
|
import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
|
|
import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
|
|
import org.apache.hadoop.ozone.web.request.OzoneAcl;
|
|
import org.apache.hadoop.ozone.web.request.OzoneAcl;
|
|
@@ -34,9 +39,13 @@ import org.apache.hadoop.ozone.web.response.VolumeInfo;
|
|
import org.apache.hadoop.ozone.web.response.VolumeOwner;
|
|
import org.apache.hadoop.ozone.web.response.VolumeOwner;
|
|
import org.apache.hadoop.ozone.web.utils.OzoneConsts;
|
|
import org.apache.hadoop.ozone.web.utils.OzoneConsts;
|
|
import org.iq80.leveldb.DBException;
|
|
import org.iq80.leveldb.DBException;
|
|
|
|
+import org.apache.commons.codec.digest.DigestUtils;
|
|
|
|
|
|
import java.io.File;
|
|
import java.io.File;
|
|
|
|
+import java.io.FileInputStream;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
|
|
+import java.io.OutputStream;
|
|
|
|
+import java.io.FileOutputStream;
|
|
import java.nio.charset.Charset;
|
|
import java.nio.charset.Charset;
|
|
import java.text.SimpleDateFormat;
|
|
import java.text.SimpleDateFormat;
|
|
import java.util.Date;
|
|
import java.util.Date;
|
|
@@ -44,6 +53,7 @@ import java.util.List;
|
|
import java.util.ListIterator;
|
|
import java.util.ListIterator;
|
|
import java.util.Locale;
|
|
import java.util.Locale;
|
|
import java.util.TimeZone;
|
|
import java.util.TimeZone;
|
|
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|
import java.util.concurrent.locks.ReadWriteLock;
|
|
import java.util.concurrent.locks.ReadWriteLock;
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
|
|
|
@@ -111,7 +121,7 @@ public final class OzoneMetadataManager {
|
|
// // stand-alone tests for the protocol and client code.
|
|
// // stand-alone tests for the protocol and client code.
|
|
|
|
|
|
*/
|
|
*/
|
|
- static final Log LOG = LogFactory.getLog(OzoneMetadataManager.class);
|
|
|
|
|
|
+ static final Logger LOG = LoggerFactory.getLogger(OzoneMetadataManager.class);
|
|
private static final String USER_DB = "/user.db";
|
|
private static final String USER_DB = "/user.db";
|
|
private static final String META_DB = "/metadata.db";
|
|
private static final String META_DB = "/metadata.db";
|
|
private static OzoneMetadataManager bm = null;
|
|
private static OzoneMetadataManager bm = null;
|
|
@@ -119,28 +129,37 @@ public final class OzoneMetadataManager {
|
|
private OzoneLevelDBStore metadataDB;
|
|
private OzoneLevelDBStore metadataDB;
|
|
private ReadWriteLock lock;
|
|
private ReadWriteLock lock;
|
|
private Charset encoding = Charset.forName("UTF-8");
|
|
private Charset encoding = Charset.forName("UTF-8");
|
|
|
|
+ private String storageRoot;
|
|
|
|
+ private static final String OBJECT_DIR = "/_objects/";
|
|
|
|
+
|
|
|
|
+ // This table keeps a pointer to objects whose operations
|
|
|
|
+ // are in progress but not yet committed to persistent store
|
|
|
|
+ private ConcurrentHashMap<OutputStream, String> inProgressObjects;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Constructs OzoneMetadataManager.
|
|
* Constructs OzoneMetadataManager.
|
|
*/
|
|
*/
|
|
- private OzoneMetadataManager(Configuration conf) {
|
|
|
|
|
|
+ private OzoneMetadataManager(Configuration conf) throws IOException {
|
|
|
|
|
|
lock = new ReentrantReadWriteLock();
|
|
lock = new ReentrantReadWriteLock();
|
|
- String storageRoot =
|
|
|
|
|
|
+ storageRoot =
|
|
conf.getTrimmed(OzoneConfigKeys.DFS_STORAGE_LOCAL_ROOT,
|
|
conf.getTrimmed(OzoneConfigKeys.DFS_STORAGE_LOCAL_ROOT,
|
|
OzoneConfigKeys.DFS_STORAGE_LOCAL_ROOT_DEFAULT);
|
|
OzoneConfigKeys.DFS_STORAGE_LOCAL_ROOT_DEFAULT);
|
|
|
|
|
|
- File file = new File(storageRoot);
|
|
|
|
|
|
+ File file = new File(storageRoot + OBJECT_DIR);
|
|
|
|
|
|
if (!file.exists() && !file.mkdirs()) {
|
|
if (!file.exists() && !file.mkdirs()) {
|
|
- LOG.fatal("Creation of Ozone root failed. " + file.toString());
|
|
|
|
|
|
+ LOG.error("Creation of Ozone root failed. " + file.toString());
|
|
|
|
+ throw new IOException("Creation of Ozone root failed.");
|
|
}
|
|
}
|
|
|
|
|
|
try {
|
|
try {
|
|
userDB = new OzoneLevelDBStore(new File(storageRoot + USER_DB), true);
|
|
userDB = new OzoneLevelDBStore(new File(storageRoot + USER_DB), true);
|
|
metadataDB = new OzoneLevelDBStore(new File(storageRoot + META_DB), true);
|
|
metadataDB = new OzoneLevelDBStore(new File(storageRoot + META_DB), true);
|
|
|
|
+ inProgressObjects = new ConcurrentHashMap<>();
|
|
} catch (IOException ex) {
|
|
} catch (IOException ex) {
|
|
- LOG.fatal("Cannot open db :" + ex.getMessage());
|
|
|
|
|
|
+ LOG.error("Cannot open db :" + ex.getMessage());
|
|
|
|
+ throw ex;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -150,7 +169,7 @@ public final class OzoneMetadataManager {
|
|
* @return OzoneMetadataManager
|
|
* @return OzoneMetadataManager
|
|
*/
|
|
*/
|
|
public static synchronized OzoneMetadataManager
|
|
public static synchronized OzoneMetadataManager
|
|
- getOzoneMetadataManager(Configuration conf) {
|
|
|
|
|
|
+ getOzoneMetadataManager(Configuration conf) throws IOException {
|
|
if (bm == null) {
|
|
if (bm == null) {
|
|
bm = new OzoneMetadataManager(conf);
|
|
bm = new OzoneMetadataManager(conf);
|
|
}
|
|
}
|
|
@@ -440,8 +459,8 @@ public final class OzoneMetadataManager {
|
|
|
|
|
|
if (args.getRemoveAcls() != null) {
|
|
if (args.getRemoveAcls() != null) {
|
|
OzoneException ex = ErrorTable.newError(ErrorTable.MALFORMED_ACL, args);
|
|
OzoneException ex = ErrorTable.newError(ErrorTable.MALFORMED_ACL, args);
|
|
- ex.setMessage("Remove ACLs specified in bucket create. Please remove " +
|
|
|
|
- "them and retry.");
|
|
|
|
|
|
+ ex.setMessage("Remove ACLs specified in bucket create. Please remove "
|
|
|
|
+ + "them and retry.");
|
|
throw ex;
|
|
throw ex;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -680,6 +699,249 @@ public final class OzoneMetadataManager {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Creates a key and returns a stream to which this key can be written to.
|
|
|
|
+ * @param args KeyArgs
|
|
|
|
+ * @return - A stream into which key can be written to.
|
|
|
|
+ * @throws OzoneException
|
|
|
|
+ */
|
|
|
|
+ public OutputStream createKey(KeyArgs args) throws OzoneException {
|
|
|
|
+ lock.writeLock().lock();
|
|
|
|
+ try {
|
|
|
|
+ String fileNameHash = DigestUtils.sha256Hex(args.getResourceName());
|
|
|
|
+
|
|
|
|
+ // Please don't try trillion objects unless the physical file system
|
|
|
|
+ // is capable of doing that in a single directory.
|
|
|
|
+
|
|
|
|
+ String fullPath = storageRoot + OBJECT_DIR + fileNameHash;
|
|
|
|
+ File f = new File(fullPath);
|
|
|
|
+
|
|
|
|
+ // In real ozone it would not be this way, a file will be overwritten
|
|
|
|
+ // only if the upload is successful.
|
|
|
|
+ if (f.exists()) {
|
|
|
|
+ LOG.debug("we are overwriting a file. This is by design.");
|
|
|
|
+ if(!f.delete()) {
|
|
|
|
+ LOG.error("Unable to delete the file: {}", fullPath);
|
|
|
|
+ throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // f.createNewFile();
|
|
|
|
+ FileOutputStream fsStream = new FileOutputStream(f);
|
|
|
|
+ inProgressObjects.put(fsStream, fullPath);
|
|
|
|
+
|
|
|
|
+ return fsStream;
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args, e);
|
|
|
|
+ } finally {
|
|
|
|
+ lock.writeLock().unlock();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * commit keys moves an In progress object into the metadata store
|
|
|
|
+ * so that key is visible in the metadata operations from that point
|
|
|
|
+ * onwards.
|
|
|
|
+ *
|
|
|
|
+ * @param args Object args
|
|
|
|
+ *
|
|
|
|
+ * @throws OzoneException
|
|
|
|
+ */
|
|
|
|
+ public void commitKey(KeyArgs args, OutputStream stream)
|
|
|
|
+ throws OzoneException {
|
|
|
|
+ SimpleDateFormat format =
|
|
|
|
+ new SimpleDateFormat("EEE, dd MMM yyyy HH:mm:ss ZZZ", Locale.US);
|
|
|
|
+ lock.writeLock().lock();
|
|
|
|
+
|
|
|
|
+ try {
|
|
|
|
+ byte[] bucketInfo = metadataDB.get(args.getParentName()
|
|
|
|
+ .getBytes(encoding));
|
|
|
|
+ if (bucketInfo == null) {
|
|
|
|
+ throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, args);
|
|
|
|
+ }
|
|
|
|
+ BucketInfo bInfo = BucketInfo.parse(new String(bucketInfo, encoding));
|
|
|
|
+ bInfo.setKeyCount(bInfo.getKeyCount() + 1);
|
|
|
|
+
|
|
|
|
+ String fileNameHash = inProgressObjects.get(stream);
|
|
|
|
+ inProgressObjects.remove(stream);
|
|
|
|
+ if (fileNameHash == null) {
|
|
|
|
+ throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ ListKeys keyList;
|
|
|
|
+ byte[] bucketListBytes = userDB.get(args.getParentName()
|
|
|
|
+ .getBytes(encoding));
|
|
|
|
+ if (bucketListBytes == null) {
|
|
|
|
+ keyList = new ListKeys();
|
|
|
|
+ } else {
|
|
|
|
+ keyList = ListKeys.parse(new String(bucketListBytes, encoding));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ KeyInfo keyInfo;
|
|
|
|
+
|
|
|
|
+ byte[] objectBytes = metadataDB.get(args.getResourceName()
|
|
|
|
+ .getBytes(encoding));
|
|
|
|
+
|
|
|
|
+ if (objectBytes != null) {
|
|
|
|
+ // we are overwriting an existing object.
|
|
|
|
+ // TODO : Emit info for Accounting
|
|
|
|
+ keyInfo = KeyInfo.parse(new String(objectBytes, encoding));
|
|
|
|
+ keyList.getKeyList().remove(keyInfo);
|
|
|
|
+ } else {
|
|
|
|
+ keyInfo = new KeyInfo();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ keyInfo.setCreatedOn(format.format(new Date(System.currentTimeMillis())));
|
|
|
|
+
|
|
|
|
+ // TODO : support version, we need to check if versioning
|
|
|
|
+ // is switched on the bucket and make appropriate calls.
|
|
|
|
+ keyInfo.setVersion(0);
|
|
|
|
+
|
|
|
|
+ keyInfo.setDataFileName(fileNameHash);
|
|
|
|
+ keyInfo.setKeyName(args.getKeyName());
|
|
|
|
+ keyInfo.setMd5hash(args.getHash());
|
|
|
|
+ keyInfo.setSize(args.getSize());
|
|
|
|
+
|
|
|
|
+ keyList.getKeyList().add(keyInfo);
|
|
|
|
+
|
|
|
|
+ // if the key exists, we overwrite happily :). since the
|
|
|
|
+ // earlier call - createObject - has overwritten the data.
|
|
|
|
+
|
|
|
|
+ metadataDB.put(args.getResourceName().getBytes(encoding),
|
|
|
|
+ keyInfo.toDBString().getBytes(encoding));
|
|
|
|
+
|
|
|
|
+ metadataDB.put(args.getParentName().getBytes(encoding),
|
|
|
|
+ bInfo.toDBString().getBytes(encoding));
|
|
|
|
+
|
|
|
|
+ userDB.put(args.getParentName().getBytes(encoding),
|
|
|
|
+ keyList.toDBString().getBytes(encoding));
|
|
|
|
+
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args, e);
|
|
|
|
+ } finally {
|
|
|
|
+ lock.writeLock().unlock();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * deletes an key from a given bucket.
|
|
|
|
+ *
|
|
|
|
+ * @param args - ObjectArgs
|
|
|
|
+ *
|
|
|
|
+ * @throws OzoneException
|
|
|
|
+ */
|
|
|
|
+ public void deleteKey(KeyArgs args) throws OzoneException {
|
|
|
|
+ lock.writeLock().lock();
|
|
|
|
+ try {
|
|
|
|
+ byte[] bucketInfo = metadataDB.get(args.getParentName()
|
|
|
|
+ .getBytes(encoding));
|
|
|
|
+ if (bucketInfo == null) {
|
|
|
|
+ throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, args);
|
|
|
|
+ }
|
|
|
|
+ BucketInfo bInfo = BucketInfo.parse(new String(bucketInfo, encoding));
|
|
|
|
+ bInfo.setKeyCount(bInfo.getKeyCount() - 1);
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+ byte[] bucketListBytes = userDB.get(args.getParentName()
|
|
|
|
+ .getBytes(encoding));
|
|
|
|
+ if (bucketListBytes == null) {
|
|
|
|
+ throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, args);
|
|
|
|
+ }
|
|
|
|
+ ListKeys keyList = ListKeys.parse(new String(bucketListBytes, encoding));
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+ byte[] objectBytes = metadataDB.get(args.getResourceName()
|
|
|
|
+ .getBytes(encoding));
|
|
|
|
+ if (objectBytes == null) {
|
|
|
|
+ throw ErrorTable.newError(ErrorTable.INVALID_KEY, args);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ KeyInfo oInfo = KeyInfo.parse(new String(objectBytes, encoding));
|
|
|
|
+ keyList.getKeyList().remove(oInfo);
|
|
|
|
+
|
|
|
|
+ String fileNameHash = DigestUtils.sha256Hex(args.getResourceName());
|
|
|
|
+
|
|
|
|
+ String fullPath = storageRoot + OBJECT_DIR + fileNameHash;
|
|
|
|
+ File f = new File(fullPath);
|
|
|
|
+
|
|
|
|
+ if (f.exists()) {
|
|
|
|
+ if(!f.delete()) {
|
|
|
|
+ throw ErrorTable.newError(ErrorTable.KEY_OPERATION_CONFLICT, args);
|
|
|
|
+ }
|
|
|
|
+ } else {
|
|
|
|
+ throw ErrorTable.newError(ErrorTable.INVALID_KEY, args);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+ metadataDB.delete(args.getResourceName().getBytes(encoding));
|
|
|
|
+ metadataDB.put(args.getParentName().getBytes(encoding),
|
|
|
|
+ bInfo.toDBString().getBytes(encoding));
|
|
|
|
+ userDB.put(args.getParentName().getBytes(encoding),
|
|
|
|
+ keyList.toDBString().getBytes(encoding));
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args, e);
|
|
|
|
+ } finally {
|
|
|
|
+ lock.writeLock().unlock();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Returns a Stream for the file.
|
|
|
|
+ *
|
|
|
|
+ * @param args - Object args
|
|
|
|
+ *
|
|
|
|
+ * @return Stream
|
|
|
|
+ *
|
|
|
|
+ * @throws IOException
|
|
|
|
+ * @throws OzoneException
|
|
|
|
+ */
|
|
|
|
+ public LengthInputStream newKeyReader(KeyArgs args)
|
|
|
|
+ throws IOException, OzoneException {
|
|
|
|
+ lock.readLock().lock();
|
|
|
|
+ try {
|
|
|
|
+ String fileNameHash = DigestUtils.sha256Hex(args.getResourceName());
|
|
|
|
+ String fullPath = storageRoot + OBJECT_DIR + fileNameHash;
|
|
|
|
+ File f = new File(fullPath);
|
|
|
|
+ if (!f.exists()) {
|
|
|
|
+ throw ErrorTable.newError(ErrorTable.INVALID_KEY, args);
|
|
|
|
+ }
|
|
|
|
+ long size = f.length();
|
|
|
|
+
|
|
|
|
+ FileInputStream fileStream = new FileInputStream(f);
|
|
|
|
+ return new LengthInputStream(fileStream, size);
|
|
|
|
+ } finally {
|
|
|
|
+ lock.readLock().unlock();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Returns keys in a bucket.
|
|
|
|
+ * @param args
|
|
|
|
+ * @return List of keys.
|
|
|
|
+ * @throws IOException
|
|
|
|
+ * @throws OzoneException
|
|
|
|
+ */
|
|
|
|
+ public ListKeys listKeys(ListArgs args) throws IOException, OzoneException {
|
|
|
|
+ lock.readLock().lock();
|
|
|
|
+ try {
|
|
|
|
+ byte[] bucketInfo = metadataDB.get(args.getResourceName()
|
|
|
|
+ .getBytes(encoding));
|
|
|
|
+ if (bucketInfo == null) {
|
|
|
|
+ throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, args);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ byte[] bucketListBytes = userDB.get(args.getResourceName()
|
|
|
|
+ .getBytes(encoding));
|
|
|
|
+ if (bucketListBytes == null) {
|
|
|
|
+ throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, args);
|
|
|
|
+ }
|
|
|
|
+ return ListKeys.parse(new String(bucketListBytes, encoding));
|
|
|
|
+ } finally {
|
|
|
|
+ lock.readLock().unlock();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* This is used in updates to volume metadata.
|
|
* This is used in updates to volume metadata.
|
|
*/
|
|
*/
|