HDDS-793. Support custom key/value annotations on volume/bucket/key. Contributed by Elek, Marton.
This commit is contained in:
parent
a448b05287
commit
9fc7df8afb
@ -21,7 +21,9 @@
|
|||||||
import org.apache.hadoop.hdds.protocol.StorageType;
|
import org.apache.hadoop.hdds.protocol.StorageType;
|
||||||
import org.apache.hadoop.ozone.OzoneAcl;
|
import org.apache.hadoop.ozone.OzoneAcl;
|
||||||
|
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class encapsulates the arguments that are
|
* This class encapsulates the arguments that are
|
||||||
@ -43,6 +45,11 @@ public final class BucketArgs {
|
|||||||
*/
|
*/
|
||||||
private StorageType storageType;
|
private StorageType storageType;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Custom key/value metadata.
|
||||||
|
*/
|
||||||
|
private Map<String, String> metadata;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Private constructor, constructed via builder.
|
* Private constructor, constructed via builder.
|
||||||
* @param versioning Bucket version flag.
|
* @param versioning Bucket version flag.
|
||||||
@ -50,10 +57,11 @@ public final class BucketArgs {
|
|||||||
* @param acls list of ACLs.
|
* @param acls list of ACLs.
|
||||||
*/
|
*/
|
||||||
private BucketArgs(Boolean versioning, StorageType storageType,
|
private BucketArgs(Boolean versioning, StorageType storageType,
|
||||||
List<OzoneAcl> acls) {
|
List<OzoneAcl> acls, Map<String, String> metadata) {
|
||||||
this.acls = acls;
|
this.acls = acls;
|
||||||
this.versioning = versioning;
|
this.versioning = versioning;
|
||||||
this.storageType = storageType;
|
this.storageType = storageType;
|
||||||
|
this.metadata = metadata;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -80,6 +88,15 @@ public List<OzoneAcl> getAcls() {
|
|||||||
return acls;
|
return acls;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Custom metadata for the buckets.
|
||||||
|
*
|
||||||
|
* @return key value map
|
||||||
|
*/
|
||||||
|
public Map<String, String> getMetadata() {
|
||||||
|
return metadata;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns new builder class that builds a OmBucketInfo.
|
* Returns new builder class that builds a OmBucketInfo.
|
||||||
*
|
*
|
||||||
@ -96,6 +113,11 @@ public static class Builder {
|
|||||||
private Boolean versioning;
|
private Boolean versioning;
|
||||||
private StorageType storageType;
|
private StorageType storageType;
|
||||||
private List<OzoneAcl> acls;
|
private List<OzoneAcl> acls;
|
||||||
|
private Map<String, String> metadata;
|
||||||
|
|
||||||
|
public Builder() {
|
||||||
|
metadata = new HashMap<>();
|
||||||
|
}
|
||||||
|
|
||||||
public BucketArgs.Builder setVersioning(Boolean versionFlag) {
|
public BucketArgs.Builder setVersioning(Boolean versionFlag) {
|
||||||
this.versioning = versionFlag;
|
this.versioning = versionFlag;
|
||||||
@ -112,12 +134,16 @@ public BucketArgs.Builder setAcls(List<OzoneAcl> listOfAcls) {
|
|||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public BucketArgs.Builder addMetadata(String key, String value) {
|
||||||
|
this.metadata.put(key, value);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
/**
|
/**
|
||||||
* Constructs the BucketArgs.
|
* Constructs the BucketArgs.
|
||||||
* @return instance of BucketArgs.
|
* @return instance of BucketArgs.
|
||||||
*/
|
*/
|
||||||
public BucketArgs build() {
|
public BucketArgs build() {
|
||||||
return new BucketArgs(versioning, storageType, acls);
|
return new BucketArgs(versioning, storageType, acls, metadata);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -32,8 +32,10 @@
|
|||||||
import org.apache.hadoop.ozone.OzoneAcl;
|
import org.apache.hadoop.ozone.OzoneAcl;
|
||||||
import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
|
||||||
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
|
||||||
|
import org.apache.hadoop.ozone.om.helpers.WithMetadata;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
@ -42,7 +44,7 @@
|
|||||||
/**
|
/**
|
||||||
* A class that encapsulates OzoneBucket.
|
* A class that encapsulates OzoneBucket.
|
||||||
*/
|
*/
|
||||||
public class OzoneBucket {
|
public class OzoneBucket extends WithMetadata {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The proxy used for connecting to the cluster and perform
|
* The proxy used for connecting to the cluster and perform
|
||||||
@ -107,7 +109,8 @@ public class OzoneBucket {
|
|||||||
public OzoneBucket(Configuration conf, ClientProtocol proxy,
|
public OzoneBucket(Configuration conf, ClientProtocol proxy,
|
||||||
String volumeName, String bucketName,
|
String volumeName, String bucketName,
|
||||||
List<OzoneAcl> acls, StorageType storageType,
|
List<OzoneAcl> acls, StorageType storageType,
|
||||||
Boolean versioning, long creationTime) {
|
Boolean versioning, long creationTime,
|
||||||
|
Map<String, String> metadata) {
|
||||||
Preconditions.checkNotNull(proxy, "Client proxy is not set.");
|
Preconditions.checkNotNull(proxy, "Client proxy is not set.");
|
||||||
this.proxy = proxy;
|
this.proxy = proxy;
|
||||||
this.volumeName = volumeName;
|
this.volumeName = volumeName;
|
||||||
@ -123,6 +126,7 @@ public OzoneBucket(Configuration conf, ClientProtocol proxy,
|
|||||||
this.defaultReplicationType = ReplicationType.valueOf(conf.get(
|
this.defaultReplicationType = ReplicationType.valueOf(conf.get(
|
||||||
OzoneConfigKeys.OZONE_REPLICATION_TYPE,
|
OzoneConfigKeys.OZONE_REPLICATION_TYPE,
|
||||||
OzoneConfigKeys.OZONE_REPLICATION_TYPE_DEFAULT));
|
OzoneConfigKeys.OZONE_REPLICATION_TYPE_DEFAULT));
|
||||||
|
this.metadata = metadata;
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
@ -248,7 +252,8 @@ public void setVersioning(Boolean newVersioning) throws IOException {
|
|||||||
*/
|
*/
|
||||||
public OzoneOutputStream createKey(String key, long size)
|
public OzoneOutputStream createKey(String key, long size)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return createKey(key, size, defaultReplicationType, defaultReplication);
|
return createKey(key, size, defaultReplicationType, defaultReplication,
|
||||||
|
new HashMap<>());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -262,9 +267,11 @@ public OzoneOutputStream createKey(String key, long size)
|
|||||||
*/
|
*/
|
||||||
public OzoneOutputStream createKey(String key, long size,
|
public OzoneOutputStream createKey(String key, long size,
|
||||||
ReplicationType type,
|
ReplicationType type,
|
||||||
ReplicationFactor factor)
|
ReplicationFactor factor,
|
||||||
|
Map<String, String> keyMetadata)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return proxy.createKey(volumeName, name, key, size, type, factor);
|
return proxy
|
||||||
|
.createKey(volumeName, name, key, size, type, factor, keyMetadata);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -21,6 +21,7 @@
|
|||||||
import org.apache.hadoop.hdds.client.ReplicationType;
|
import org.apache.hadoop.hdds.client.ReplicationType;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A class that encapsulates OzoneKeyLocation.
|
* A class that encapsulates OzoneKeyLocation.
|
||||||
@ -32,6 +33,8 @@ public class OzoneKeyDetails extends OzoneKey {
|
|||||||
*/
|
*/
|
||||||
private List<OzoneKeyLocation> ozoneKeyLocations;
|
private List<OzoneKeyLocation> ozoneKeyLocations;
|
||||||
|
|
||||||
|
private Map<String, String> metadata;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructs OzoneKeyDetails from OmKeyInfo.
|
* Constructs OzoneKeyDetails from OmKeyInfo.
|
||||||
*/
|
*/
|
||||||
@ -39,10 +42,11 @@ public class OzoneKeyDetails extends OzoneKey {
|
|||||||
public OzoneKeyDetails(String volumeName, String bucketName, String keyName,
|
public OzoneKeyDetails(String volumeName, String bucketName, String keyName,
|
||||||
long size, long creationTime, long modificationTime,
|
long size, long creationTime, long modificationTime,
|
||||||
List<OzoneKeyLocation> ozoneKeyLocations,
|
List<OzoneKeyLocation> ozoneKeyLocations,
|
||||||
ReplicationType type) {
|
ReplicationType type, Map<String, String> metadata) {
|
||||||
super(volumeName, bucketName, keyName, size, creationTime,
|
super(volumeName, bucketName, keyName, size, creationTime,
|
||||||
modificationTime, type);
|
modificationTime, type);
|
||||||
this.ozoneKeyLocations = ozoneKeyLocations;
|
this.ozoneKeyLocations = ozoneKeyLocations;
|
||||||
|
this.metadata = metadata;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -52,6 +56,10 @@ public List<OzoneKeyLocation> getOzoneKeyLocations() {
|
|||||||
return ozoneKeyLocations;
|
return ozoneKeyLocations;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Map<String, String> getMetadata() {
|
||||||
|
return metadata;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set details of key location.
|
* Set details of key location.
|
||||||
* @param ozoneKeyLocations - details of key location
|
* @param ozoneKeyLocations - details of key location
|
||||||
|
@ -19,8 +19,10 @@
|
|||||||
package org.apache.hadoop.ozone.client;
|
package org.apache.hadoop.ozone.client;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.NoSuchElementException;
|
import java.util.NoSuchElementException;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
@ -28,6 +30,7 @@
|
|||||||
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
|
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
|
||||||
import org.apache.hadoop.ozone.OzoneAcl;
|
import org.apache.hadoop.ozone.OzoneAcl;
|
||||||
import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
|
import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
|
||||||
|
import org.apache.hadoop.ozone.om.helpers.WithMetadata;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
@ -35,7 +38,7 @@
|
|||||||
/**
|
/**
|
||||||
* A class that encapsulates OzoneVolume.
|
* A class that encapsulates OzoneVolume.
|
||||||
*/
|
*/
|
||||||
public class OzoneVolume {
|
public class OzoneVolume extends WithMetadata {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The proxy used for connecting to the cluster and perform
|
* The proxy used for connecting to the cluster and perform
|
||||||
@ -81,11 +84,13 @@ public class OzoneVolume {
|
|||||||
* @param quotaInBytes Volume quota in bytes.
|
* @param quotaInBytes Volume quota in bytes.
|
||||||
* @param creationTime creation time of the volume
|
* @param creationTime creation time of the volume
|
||||||
* @param acls ACLs associated with the volume.
|
* @param acls ACLs associated with the volume.
|
||||||
|
* @param metadata custom key value metadata.
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings("parameternumber")
|
@SuppressWarnings("parameternumber")
|
||||||
public OzoneVolume(Configuration conf, ClientProtocol proxy, String name,
|
public OzoneVolume(Configuration conf, ClientProtocol proxy, String name,
|
||||||
String admin, String owner, long quotaInBytes,
|
String admin, String owner, long quotaInBytes,
|
||||||
long creationTime, List<OzoneAcl> acls) {
|
long creationTime, List<OzoneAcl> acls,
|
||||||
|
Map<String, String> metadata) {
|
||||||
Preconditions.checkNotNull(proxy, "Client proxy is not set.");
|
Preconditions.checkNotNull(proxy, "Client proxy is not set.");
|
||||||
this.proxy = proxy;
|
this.proxy = proxy;
|
||||||
this.name = name;
|
this.name = name;
|
||||||
@ -95,6 +100,15 @@ public OzoneVolume(Configuration conf, ClientProtocol proxy, String name,
|
|||||||
this.creationTime = creationTime;
|
this.creationTime = creationTime;
|
||||||
this.acls = acls;
|
this.acls = acls;
|
||||||
this.listCacheSize = HddsClientUtils.getListCacheSize(conf);
|
this.listCacheSize = HddsClientUtils.getListCacheSize(conf);
|
||||||
|
this.metadata = metadata;
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("parameternumber")
|
||||||
|
public OzoneVolume(Configuration conf, ClientProtocol proxy, String name,
|
||||||
|
String admin, String owner, long quotaInBytes,
|
||||||
|
long creationTime, List<OzoneAcl> acls) {
|
||||||
|
this(conf, proxy, name, admin, owner, quotaInBytes, creationTime, acls,
|
||||||
|
new HashMap<>());
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
@ -108,6 +122,7 @@ protected OzoneVolume(String name, String admin, String owner,
|
|||||||
this.quotaInBytes = quotaInBytes;
|
this.quotaInBytes = quotaInBytes;
|
||||||
this.creationTime = creationTime;
|
this.creationTime = creationTime;
|
||||||
this.acls = acls;
|
this.acls = acls;
|
||||||
|
this.metadata = new HashMap<>();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -21,7 +21,9 @@
|
|||||||
import org.apache.hadoop.ozone.OzoneAcl;
|
import org.apache.hadoop.ozone.OzoneAcl;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class encapsulates the arguments that are
|
* This class encapsulates the arguments that are
|
||||||
@ -33,6 +35,7 @@ public final class VolumeArgs {
|
|||||||
private final String owner;
|
private final String owner;
|
||||||
private final String quota;
|
private final String quota;
|
||||||
private final List<OzoneAcl> acls;
|
private final List<OzoneAcl> acls;
|
||||||
|
private Map<String, String> metadata;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Private constructor, constructed via builder.
|
* Private constructor, constructed via builder.
|
||||||
@ -41,12 +44,16 @@ public final class VolumeArgs {
|
|||||||
* @param quota Volume Quota.
|
* @param quota Volume Quota.
|
||||||
* @param acls User to access rights map.
|
* @param acls User to access rights map.
|
||||||
*/
|
*/
|
||||||
private VolumeArgs(String admin, String owner,
|
private VolumeArgs(String admin,
|
||||||
String quota, List<OzoneAcl> acls) {
|
String owner,
|
||||||
|
String quota,
|
||||||
|
List<OzoneAcl> acls,
|
||||||
|
Map<String, String> metadata) {
|
||||||
this.admin = admin;
|
this.admin = admin;
|
||||||
this.owner = owner;
|
this.owner = owner;
|
||||||
this.quota = quota;
|
this.quota = quota;
|
||||||
this.acls = acls;
|
this.acls = acls;
|
||||||
|
this.metadata = metadata;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -73,6 +80,15 @@ public String getQuota() {
|
|||||||
return quota;
|
return quota;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return custom key value map.
|
||||||
|
*
|
||||||
|
* @return metadata
|
||||||
|
*/
|
||||||
|
public Map<String, String> getMetadata() {
|
||||||
|
return metadata;
|
||||||
|
}
|
||||||
|
|
||||||
public List<OzoneAcl> getAcls() {
|
public List<OzoneAcl> getAcls() {
|
||||||
return acls;
|
return acls;
|
||||||
}
|
}
|
||||||
@ -93,6 +109,7 @@ public static class Builder {
|
|||||||
private String ownerName;
|
private String ownerName;
|
||||||
private String volumeQuota;
|
private String volumeQuota;
|
||||||
private List<OzoneAcl> listOfAcls;
|
private List<OzoneAcl> listOfAcls;
|
||||||
|
private Map<String, String> metadata = new HashMap<>();
|
||||||
|
|
||||||
|
|
||||||
public VolumeArgs.Builder setAdmin(String admin) {
|
public VolumeArgs.Builder setAdmin(String admin) {
|
||||||
@ -110,6 +127,10 @@ public VolumeArgs.Builder setQuota(String quota) {
|
|||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public VolumeArgs.Builder addMetadata(String key, String value) {
|
||||||
|
metadata.put(key, value);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
public VolumeArgs.Builder setAcls(List<OzoneAcl> acls)
|
public VolumeArgs.Builder setAcls(List<OzoneAcl> acls)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
this.listOfAcls = acls;
|
this.listOfAcls = acls;
|
||||||
@ -121,7 +142,8 @@ public VolumeArgs.Builder setAcls(List<OzoneAcl> acls)
|
|||||||
* @return CreateVolumeArgs.
|
* @return CreateVolumeArgs.
|
||||||
*/
|
*/
|
||||||
public VolumeArgs build() {
|
public VolumeArgs build() {
|
||||||
return new VolumeArgs(adminName, ownerName, volumeQuota, listOfAcls);
|
return new VolumeArgs(adminName, ownerName, volumeQuota, listOfAcls,
|
||||||
|
metadata);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -262,12 +262,14 @@ List<OzoneBucket> listBuckets(String volumeName, String bucketPrefix,
|
|||||||
* @param bucketName Name of the Bucket
|
* @param bucketName Name of the Bucket
|
||||||
* @param keyName Name of the Key
|
* @param keyName Name of the Key
|
||||||
* @param size Size of the data
|
* @param size Size of the data
|
||||||
|
* @param metadata custom key value metadata
|
||||||
* @return {@link OzoneOutputStream}
|
* @return {@link OzoneOutputStream}
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
OzoneOutputStream createKey(String volumeName, String bucketName,
|
OzoneOutputStream createKey(String volumeName, String bucketName,
|
||||||
String keyName, long size, ReplicationType type,
|
String keyName, long size, ReplicationType type,
|
||||||
ReplicationFactor factor)
|
ReplicationFactor factor,
|
||||||
|
Map<String, String> metadata)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -83,6 +83,7 @@
|
|||||||
import java.net.URISyntaxException;
|
import java.net.URISyntaxException;
|
||||||
import java.text.ParseException;
|
import java.text.ParseException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
@ -572,7 +573,8 @@ public OzoneBucket getBucketDetails(String volumeName, String bucketName)
|
|||||||
bucketInfo.getAcls(),
|
bucketInfo.getAcls(),
|
||||||
bucketInfo.getStorageType(),
|
bucketInfo.getStorageType(),
|
||||||
getBucketVersioningFlag(bucketInfo.getVersioning()),
|
getBucketVersioningFlag(bucketInfo.getVersioning()),
|
||||||
HddsClientUtils.formatDateTime(bucketInfo.getCreatedOn()));
|
HddsClientUtils.formatDateTime(bucketInfo.getCreatedOn()),
|
||||||
|
new HashMap<>());
|
||||||
EntityUtils.consume(response);
|
EntityUtils.consume(response);
|
||||||
return bucket;
|
return bucket;
|
||||||
} catch (URISyntaxException | ParseException e) {
|
} catch (URISyntaxException | ParseException e) {
|
||||||
@ -611,7 +613,8 @@ public List<OzoneBucket> listBuckets(String volumeName, String bucketPrefix,
|
|||||||
return new OzoneBucket(conf, this, volumeName,
|
return new OzoneBucket(conf, this, volumeName,
|
||||||
bucketInfo.getBucketName(), bucketInfo.getAcls(),
|
bucketInfo.getBucketName(), bucketInfo.getAcls(),
|
||||||
bucketInfo.getStorageType(),
|
bucketInfo.getStorageType(),
|
||||||
getBucketVersioningFlag(bucketInfo.getVersioning()), creationTime);
|
getBucketVersioningFlag(bucketInfo.getVersioning()), creationTime,
|
||||||
|
new HashMap<>());
|
||||||
}).collect(Collectors.toList());
|
}).collect(Collectors.toList());
|
||||||
} catch (URISyntaxException e) {
|
} catch (URISyntaxException e) {
|
||||||
throw new IOException(e);
|
throw new IOException(e);
|
||||||
@ -631,7 +634,8 @@ public List<OzoneBucket> listBuckets(String volumeName, String bucketPrefix,
|
|||||||
@Override
|
@Override
|
||||||
public OzoneOutputStream createKey(
|
public OzoneOutputStream createKey(
|
||||||
String volumeName, String bucketName, String keyName, long size,
|
String volumeName, String bucketName, String keyName, long size,
|
||||||
ReplicationType type, ReplicationFactor factor)
|
ReplicationType type, ReplicationFactor factor,
|
||||||
|
Map<String, String> metadata)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
// TODO: Once ReplicationType and ReplicationFactor are supported in
|
// TODO: Once ReplicationType and ReplicationFactor are supported in
|
||||||
// OzoneHandler (in Datanode), set them in header.
|
// OzoneHandler (in Datanode), set them in header.
|
||||||
@ -865,7 +869,8 @@ public OzoneKeyDetails getKeyDetails(
|
|||||||
HddsClientUtils.formatDateTime(keyInfo.getCreatedOn()),
|
HddsClientUtils.formatDateTime(keyInfo.getCreatedOn()),
|
||||||
HddsClientUtils.formatDateTime(keyInfo.getModifiedOn()),
|
HddsClientUtils.formatDateTime(keyInfo.getModifiedOn()),
|
||||||
ozoneKeyLocations, ReplicationType.valueOf(
|
ozoneKeyLocations, ReplicationType.valueOf(
|
||||||
keyInfo.getType().toString()));
|
keyInfo.getType().toString()),
|
||||||
|
new HashMap<>());
|
||||||
EntityUtils.consume(response);
|
EntityUtils.consume(response);
|
||||||
return key;
|
return key;
|
||||||
} catch (URISyntaxException | ParseException e) {
|
} catch (URISyntaxException | ParseException e) {
|
||||||
|
@ -247,6 +247,7 @@ public void createVolume(String volumeName, VolumeArgs volArgs)
|
|||||||
builder.setAdminName(admin);
|
builder.setAdminName(admin);
|
||||||
builder.setOwnerName(owner);
|
builder.setOwnerName(owner);
|
||||||
builder.setQuotaInBytes(quota);
|
builder.setQuotaInBytes(quota);
|
||||||
|
builder.addAllMetadata(volArgs.getMetadata());
|
||||||
|
|
||||||
//Remove duplicates and add ACLs
|
//Remove duplicates and add ACLs
|
||||||
for (OzoneAcl ozoneAcl :
|
for (OzoneAcl ozoneAcl :
|
||||||
@ -290,7 +291,8 @@ public OzoneVolume getVolumeDetails(String volumeName)
|
|||||||
volume.getQuotaInBytes(),
|
volume.getQuotaInBytes(),
|
||||||
volume.getCreationTime(),
|
volume.getCreationTime(),
|
||||||
volume.getAclMap().ozoneAclGetProtobuf().stream().
|
volume.getAclMap().ozoneAclGetProtobuf().stream().
|
||||||
map(OMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
|
map(OMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
|
||||||
|
volume.getMetadata());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -341,7 +343,8 @@ public List<OzoneVolume> listVolumes(String user, String volumePrefix,
|
|||||||
volume.getQuotaInBytes(),
|
volume.getQuotaInBytes(),
|
||||||
volume.getCreationTime(),
|
volume.getCreationTime(),
|
||||||
volume.getAclMap().ozoneAclGetProtobuf().stream().
|
volume.getAclMap().ozoneAclGetProtobuf().stream().
|
||||||
map(OMPBHelper::convertOzoneAcl).collect(Collectors.toList())))
|
map(OMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
|
||||||
|
volume.getMetadata()))
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -380,6 +383,7 @@ public void createBucket(
|
|||||||
builder.setVolumeName(volumeName)
|
builder.setVolumeName(volumeName)
|
||||||
.setBucketName(bucketName)
|
.setBucketName(bucketName)
|
||||||
.setIsVersionEnabled(isVersionEnabled)
|
.setIsVersionEnabled(isVersionEnabled)
|
||||||
|
.addAllMetadata(bucketArgs.getMetadata())
|
||||||
.setStorageType(storageType)
|
.setStorageType(storageType)
|
||||||
.setAcls(listOfAcls.stream().distinct().collect(Collectors.toList()));
|
.setAcls(listOfAcls.stream().distinct().collect(Collectors.toList()));
|
||||||
|
|
||||||
@ -510,17 +514,18 @@ public void checkBucketAccess(
|
|||||||
public OzoneBucket getBucketDetails(
|
public OzoneBucket getBucketDetails(
|
||||||
String volumeName, String bucketName) throws IOException {
|
String volumeName, String bucketName) throws IOException {
|
||||||
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
||||||
OmBucketInfo bucketArgs =
|
OmBucketInfo bucketInfo =
|
||||||
ozoneManagerClient.getBucketInfo(volumeName, bucketName);
|
ozoneManagerClient.getBucketInfo(volumeName, bucketName);
|
||||||
return new OzoneBucket(
|
return new OzoneBucket(
|
||||||
conf,
|
conf,
|
||||||
this,
|
this,
|
||||||
bucketArgs.getVolumeName(),
|
bucketInfo.getVolumeName(),
|
||||||
bucketArgs.getBucketName(),
|
bucketInfo.getBucketName(),
|
||||||
bucketArgs.getAcls(),
|
bucketInfo.getAcls(),
|
||||||
bucketArgs.getStorageType(),
|
bucketInfo.getStorageType(),
|
||||||
bucketArgs.getIsVersionEnabled(),
|
bucketInfo.getIsVersionEnabled(),
|
||||||
bucketArgs.getCreationTime());
|
bucketInfo.getCreationTime(),
|
||||||
|
bucketInfo.getMetadata());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -538,14 +543,16 @@ public List<OzoneBucket> listBuckets(String volumeName, String bucketPrefix,
|
|||||||
bucket.getAcls(),
|
bucket.getAcls(),
|
||||||
bucket.getStorageType(),
|
bucket.getStorageType(),
|
||||||
bucket.getIsVersionEnabled(),
|
bucket.getIsVersionEnabled(),
|
||||||
bucket.getCreationTime()))
|
bucket.getCreationTime(),
|
||||||
|
bucket.getMetadata()))
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public OzoneOutputStream createKey(
|
public OzoneOutputStream createKey(
|
||||||
String volumeName, String bucketName, String keyName, long size,
|
String volumeName, String bucketName, String keyName, long size,
|
||||||
ReplicationType type, ReplicationFactor factor)
|
ReplicationType type, ReplicationFactor factor,
|
||||||
|
Map<String, String> metadata)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
||||||
HddsClientUtils.checkNotNull(keyName, type, factor);
|
HddsClientUtils.checkNotNull(keyName, type, factor);
|
||||||
@ -557,6 +564,7 @@ public OzoneOutputStream createKey(
|
|||||||
.setDataSize(size)
|
.setDataSize(size)
|
||||||
.setType(HddsProtos.ReplicationType.valueOf(type.toString()))
|
.setType(HddsProtos.ReplicationType.valueOf(type.toString()))
|
||||||
.setFactor(HddsProtos.ReplicationFactor.valueOf(factor.getValue()))
|
.setFactor(HddsProtos.ReplicationFactor.valueOf(factor.getValue()))
|
||||||
|
.addAllMetadata(metadata)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
OpenKeySession openKey = ozoneManagerClient.openKey(keyArgs);
|
OpenKeySession openKey = ozoneManagerClient.openKey(keyArgs);
|
||||||
@ -669,7 +677,7 @@ public OzoneKeyDetails getKeyDetails(
|
|||||||
return new OzoneKeyDetails(keyInfo.getVolumeName(), keyInfo.getBucketName(),
|
return new OzoneKeyDetails(keyInfo.getVolumeName(), keyInfo.getBucketName(),
|
||||||
keyInfo.getKeyName(), keyInfo.getDataSize(), keyInfo.getCreationTime(),
|
keyInfo.getKeyName(), keyInfo.getDataSize(), keyInfo.getCreationTime(),
|
||||||
keyInfo.getModificationTime(), ozoneKeyLocations, ReplicationType
|
keyInfo.getModificationTime(), ozoneKeyLocations, ReplicationType
|
||||||
.valueOf(keyInfo.getType().toString()));
|
.valueOf(keyInfo.getType().toString()), keyInfo.getMetadata());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -728,7 +736,8 @@ public List<OzoneBucket> listS3Buckets(String userName, String bucketPrefix,
|
|||||||
bucket.getAcls(),
|
bucket.getAcls(),
|
||||||
bucket.getStorageType(),
|
bucket.getStorageType(),
|
||||||
bucket.getIsVersionEnabled(),
|
bucket.getIsVersionEnabled(),
|
||||||
bucket.getCreationTime()))
|
bucket.getCreationTime(),
|
||||||
|
bucket.getMetadata()))
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -0,0 +1,54 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.apache.hadoop.ozone.om.helpers;
|
||||||
|
|
||||||
|
import java.util.LinkedList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.KeyValue;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convert from/to hdds KeyValue protobuf structure.
|
||||||
|
*/
|
||||||
|
public final class KeyValueUtil {
|
||||||
|
private KeyValueUtil() {
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Parse Key,Value map data from protobuf representation.
|
||||||
|
*/
|
||||||
|
public static Map<String, String> getFromProtobuf(List<KeyValue> metadata) {
|
||||||
|
return metadata.stream()
|
||||||
|
.collect(Collectors.toMap(KeyValue::getKey,
|
||||||
|
KeyValue::getValue));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Encode key value map to protobuf.
|
||||||
|
*/
|
||||||
|
public static List<KeyValue> toProtobuf(Map<String, String> keyValueMap) {
|
||||||
|
List<KeyValue> metadataList = new LinkedList<>();
|
||||||
|
for (Map.Entry<String, String> entry : keyValueMap.entrySet()) {
|
||||||
|
metadataList.add(KeyValue.newBuilder().setKey(entry.getKey()).
|
||||||
|
setValue(entry.getValue()).build());
|
||||||
|
}
|
||||||
|
return metadataList;
|
||||||
|
}
|
||||||
|
}
|
@ -17,6 +17,7 @@
|
|||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.om.helpers;
|
package org.apache.hadoop.ozone.om.helpers;
|
||||||
|
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.LinkedHashMap;
|
import java.util.LinkedHashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
@ -34,7 +35,7 @@
|
|||||||
/**
|
/**
|
||||||
* A class that encapsulates Bucket Arguments.
|
* A class that encapsulates Bucket Arguments.
|
||||||
*/
|
*/
|
||||||
public final class OmBucketArgs implements Auditable {
|
public final class OmBucketArgs extends WithMetadata implements Auditable {
|
||||||
/**
|
/**
|
||||||
* Name of the volume in which the bucket belongs to.
|
* Name of the volume in which the bucket belongs to.
|
||||||
*/
|
*/
|
||||||
@ -72,13 +73,15 @@ public final class OmBucketArgs implements Auditable {
|
|||||||
*/
|
*/
|
||||||
private OmBucketArgs(String volumeName, String bucketName,
|
private OmBucketArgs(String volumeName, String bucketName,
|
||||||
List<OzoneAcl> addAcls, List<OzoneAcl> removeAcls,
|
List<OzoneAcl> addAcls, List<OzoneAcl> removeAcls,
|
||||||
Boolean isVersionEnabled, StorageType storageType) {
|
Boolean isVersionEnabled, StorageType storageType,
|
||||||
|
Map<String, String> metadata) {
|
||||||
this.volumeName = volumeName;
|
this.volumeName = volumeName;
|
||||||
this.bucketName = bucketName;
|
this.bucketName = bucketName;
|
||||||
this.addAcls = addAcls;
|
this.addAcls = addAcls;
|
||||||
this.removeAcls = removeAcls;
|
this.removeAcls = removeAcls;
|
||||||
this.isVersionEnabled = isVersionEnabled;
|
this.isVersionEnabled = isVersionEnabled;
|
||||||
this.storageType = storageType;
|
this.storageType = storageType;
|
||||||
|
this.metadata = metadata;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -167,6 +170,7 @@ public static class Builder {
|
|||||||
private List<OzoneAcl> removeAcls;
|
private List<OzoneAcl> removeAcls;
|
||||||
private Boolean isVersionEnabled;
|
private Boolean isVersionEnabled;
|
||||||
private StorageType storageType;
|
private StorageType storageType;
|
||||||
|
private Map<String, String> metadata;
|
||||||
|
|
||||||
public Builder setVolumeName(String volume) {
|
public Builder setVolumeName(String volume) {
|
||||||
this.volumeName = volume;
|
this.volumeName = volume;
|
||||||
@ -193,6 +197,11 @@ public Builder setIsVersionEnabled(Boolean versionFlag) {
|
|||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Builder addMetadata(Map<String, String> metadataMap) {
|
||||||
|
this.metadata = metadataMap;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
public Builder setStorageType(StorageType storage) {
|
public Builder setStorageType(StorageType storage) {
|
||||||
this.storageType = storage;
|
this.storageType = storage;
|
||||||
return this;
|
return this;
|
||||||
@ -206,7 +215,7 @@ public OmBucketArgs build() {
|
|||||||
Preconditions.checkNotNull(volumeName);
|
Preconditions.checkNotNull(volumeName);
|
||||||
Preconditions.checkNotNull(bucketName);
|
Preconditions.checkNotNull(bucketName);
|
||||||
return new OmBucketArgs(volumeName, bucketName, addAcls,
|
return new OmBucketArgs(volumeName, bucketName, addAcls,
|
||||||
removeAcls, isVersionEnabled, storageType);
|
removeAcls, isVersionEnabled, storageType, metadata);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -249,6 +258,7 @@ public static OmBucketArgs getFromProtobuf(BucketArgs bucketArgs) {
|
|||||||
bucketArgs.hasIsVersionEnabled() ?
|
bucketArgs.hasIsVersionEnabled() ?
|
||||||
bucketArgs.getIsVersionEnabled() : null,
|
bucketArgs.getIsVersionEnabled() : null,
|
||||||
bucketArgs.hasStorageType() ? StorageType.valueOf(
|
bucketArgs.hasStorageType() ? StorageType.valueOf(
|
||||||
bucketArgs.getStorageType()) : null);
|
bucketArgs.getStorageType()) : null,
|
||||||
|
KeyValueUtil.getFromProtobuf(bucketArgs.getMetadataList()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -17,10 +17,13 @@
|
|||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.om.helpers;
|
package org.apache.hadoop.ozone.om.helpers;
|
||||||
|
|
||||||
|
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.LinkedHashMap;
|
import java.util.LinkedHashMap;
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import org.apache.hadoop.hdds.protocol.StorageType;
|
import org.apache.hadoop.hdds.protocol.StorageType;
|
||||||
@ -35,7 +38,7 @@
|
|||||||
/**
|
/**
|
||||||
* A class that encapsulates Bucket Info.
|
* A class that encapsulates Bucket Info.
|
||||||
*/
|
*/
|
||||||
public final class OmBucketInfo implements Auditable {
|
public final class OmBucketInfo extends WithMetadata implements Auditable {
|
||||||
/**
|
/**
|
||||||
* Name of the volume in which the bucket belongs to.
|
* Name of the volume in which the bucket belongs to.
|
||||||
*/
|
*/
|
||||||
@ -71,15 +74,20 @@ public final class OmBucketInfo implements Auditable {
|
|||||||
* @param storageType - Storage type to be used.
|
* @param storageType - Storage type to be used.
|
||||||
* @param creationTime - Bucket creation time.
|
* @param creationTime - Bucket creation time.
|
||||||
*/
|
*/
|
||||||
private OmBucketInfo(String volumeName, String bucketName,
|
private OmBucketInfo(String volumeName,
|
||||||
List<OzoneAcl> acls, boolean isVersionEnabled,
|
String bucketName,
|
||||||
StorageType storageType, long creationTime) {
|
List<OzoneAcl> acls,
|
||||||
|
boolean isVersionEnabled,
|
||||||
|
StorageType storageType,
|
||||||
|
long creationTime,
|
||||||
|
Map<String, String> metadata) {
|
||||||
this.volumeName = volumeName;
|
this.volumeName = volumeName;
|
||||||
this.bucketName = bucketName;
|
this.bucketName = bucketName;
|
||||||
this.acls = acls;
|
this.acls = acls;
|
||||||
this.isVersionEnabled = isVersionEnabled;
|
this.isVersionEnabled = isVersionEnabled;
|
||||||
this.storageType = storageType;
|
this.storageType = storageType;
|
||||||
this.creationTime = creationTime;
|
this.creationTime = creationTime;
|
||||||
|
this.metadata = metadata;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -165,12 +173,14 @@ public static class Builder {
|
|||||||
private Boolean isVersionEnabled;
|
private Boolean isVersionEnabled;
|
||||||
private StorageType storageType;
|
private StorageType storageType;
|
||||||
private long creationTime;
|
private long creationTime;
|
||||||
|
private Map<String, String> metadata;
|
||||||
|
|
||||||
public Builder() {
|
public Builder() {
|
||||||
//Default values
|
//Default values
|
||||||
this.acls = new LinkedList<>();
|
this.acls = new LinkedList<>();
|
||||||
this.isVersionEnabled = false;
|
this.isVersionEnabled = false;
|
||||||
this.storageType = StorageType.DISK;
|
this.storageType = StorageType.DISK;
|
||||||
|
this.metadata = new HashMap<>();
|
||||||
}
|
}
|
||||||
|
|
||||||
public Builder setVolumeName(String volume) {
|
public Builder setVolumeName(String volume) {
|
||||||
@ -203,6 +213,16 @@ public Builder setCreationTime(long createdOn) {
|
|||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Builder addMetadata(String key, String value) {
|
||||||
|
metadata.put(key, value);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder addAllMetadata(Map<String, String> additionalMetadata) {
|
||||||
|
metadata.putAll(additionalMetadata);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructs the OmBucketInfo.
|
* Constructs the OmBucketInfo.
|
||||||
* @return instance of OmBucketInfo.
|
* @return instance of OmBucketInfo.
|
||||||
@ -215,7 +235,8 @@ public OmBucketInfo build() {
|
|||||||
Preconditions.checkNotNull(storageType);
|
Preconditions.checkNotNull(storageType);
|
||||||
|
|
||||||
return new OmBucketInfo(volumeName, bucketName, acls,
|
return new OmBucketInfo(volumeName, bucketName, acls,
|
||||||
isVersionEnabled, storageType, creationTime);
|
isVersionEnabled, storageType, creationTime, metadata
|
||||||
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -231,6 +252,7 @@ public BucketInfo getProtobuf() {
|
|||||||
.setIsVersionEnabled(isVersionEnabled)
|
.setIsVersionEnabled(isVersionEnabled)
|
||||||
.setStorageType(storageType.toProto())
|
.setStorageType(storageType.toProto())
|
||||||
.setCreationTime(creationTime)
|
.setCreationTime(creationTime)
|
||||||
|
.addAllMetadata(KeyValueUtil.toProtobuf(metadata))
|
||||||
.build();
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -247,6 +269,30 @@ public static OmBucketInfo getFromProtobuf(BucketInfo bucketInfo) {
|
|||||||
OMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
|
OMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
|
||||||
bucketInfo.getIsVersionEnabled(),
|
bucketInfo.getIsVersionEnabled(),
|
||||||
StorageType.valueOf(bucketInfo.getStorageType()),
|
StorageType.valueOf(bucketInfo.getStorageType()),
|
||||||
bucketInfo.getCreationTime());
|
bucketInfo.getCreationTime(),
|
||||||
|
KeyValueUtil.getFromProtobuf(bucketInfo.getMetadataList()));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o) {
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
OmBucketInfo that = (OmBucketInfo) o;
|
||||||
|
return creationTime == that.creationTime &&
|
||||||
|
volumeName.equals(that.volumeName) &&
|
||||||
|
bucketName.equals(that.bucketName) &&
|
||||||
|
Objects.equals(acls, that.acls) &&
|
||||||
|
Objects.equals(isVersionEnabled, that.isVersionEnabled) &&
|
||||||
|
storageType == that.storageType &&
|
||||||
|
Objects.equals(metadata, that.metadata);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(volumeName, bucketName);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -23,6 +23,7 @@
|
|||||||
import org.apache.hadoop.ozone.audit.Auditable;
|
import org.apache.hadoop.ozone.audit.Auditable;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.LinkedHashMap;
|
import java.util.LinkedHashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
@ -42,12 +43,14 @@ public final class OmKeyArgs implements Auditable {
|
|||||||
private final boolean isMultipartKey;
|
private final boolean isMultipartKey;
|
||||||
private final String multipartUploadID;
|
private final String multipartUploadID;
|
||||||
private final int multipartUploadPartNumber;
|
private final int multipartUploadPartNumber;
|
||||||
|
private Map<String, String> metadata;
|
||||||
|
|
||||||
@SuppressWarnings("parameternumber")
|
@SuppressWarnings("parameternumber")
|
||||||
private OmKeyArgs(String volumeName, String bucketName, String keyName,
|
private OmKeyArgs(String volumeName, String bucketName, String keyName,
|
||||||
long dataSize, ReplicationType type, ReplicationFactor factor,
|
long dataSize, ReplicationType type, ReplicationFactor factor,
|
||||||
List<OmKeyLocationInfo> locationInfoList, boolean isMultipart,
|
List<OmKeyLocationInfo> locationInfoList, boolean isMultipart,
|
||||||
String uploadID, int partNumber) {
|
String uploadID, int partNumber,
|
||||||
|
Map<String, String> metadataMap) {
|
||||||
this.volumeName = volumeName;
|
this.volumeName = volumeName;
|
||||||
this.bucketName = bucketName;
|
this.bucketName = bucketName;
|
||||||
this.keyName = keyName;
|
this.keyName = keyName;
|
||||||
@ -58,6 +61,7 @@ private OmKeyArgs(String volumeName, String bucketName, String keyName,
|
|||||||
this.isMultipartKey = isMultipart;
|
this.isMultipartKey = isMultipart;
|
||||||
this.multipartUploadID = uploadID;
|
this.multipartUploadID = uploadID;
|
||||||
this.multipartUploadPartNumber = partNumber;
|
this.multipartUploadPartNumber = partNumber;
|
||||||
|
this.metadata = metadataMap;
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean getIsMultipartKey() {
|
public boolean getIsMultipartKey() {
|
||||||
@ -100,6 +104,14 @@ public void setDataSize(long size) {
|
|||||||
dataSize = size;
|
dataSize = size;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Map<String, String> getMetadata() {
|
||||||
|
return metadata;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setMetadata(Map<String, String> metadata) {
|
||||||
|
this.metadata = metadata;
|
||||||
|
}
|
||||||
|
|
||||||
public void setLocationInfoList(List<OmKeyLocationInfo> locationInfoList) {
|
public void setLocationInfoList(List<OmKeyLocationInfo> locationInfoList) {
|
||||||
this.locationInfoList = locationInfoList;
|
this.locationInfoList = locationInfoList;
|
||||||
}
|
}
|
||||||
@ -146,6 +158,7 @@ public static class Builder {
|
|||||||
private boolean isMultipartKey;
|
private boolean isMultipartKey;
|
||||||
private String multipartUploadID;
|
private String multipartUploadID;
|
||||||
private int multipartUploadPartNumber;
|
private int multipartUploadPartNumber;
|
||||||
|
private Map<String, String> metadata = new HashMap<>();
|
||||||
|
|
||||||
public Builder setVolumeName(String volume) {
|
public Builder setVolumeName(String volume) {
|
||||||
this.volumeName = volume;
|
this.volumeName = volume;
|
||||||
@ -197,10 +210,21 @@ public Builder setMultipartUploadPartNumber(int partNumber) {
|
|||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Builder addMetadata(String key, String value) {
|
||||||
|
this.metadata.put(key, value);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder addAllMetadata(Map<String, String> metadatamap) {
|
||||||
|
this.metadata.putAll(metadatamap);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
public OmKeyArgs build() {
|
public OmKeyArgs build() {
|
||||||
return new OmKeyArgs(volumeName, bucketName, keyName, dataSize, type,
|
return new OmKeyArgs(volumeName, bucketName, keyName, dataSize, type,
|
||||||
factor, locationInfoList, isMultipartKey, multipartUploadID,
|
factor, locationInfoList, isMultipartKey, multipartUploadID,
|
||||||
multipartUploadPartNumber);
|
multipartUploadPartNumber, metadata);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -19,7 +19,10 @@
|
|||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
@ -33,7 +36,7 @@
|
|||||||
* This is returned from OM to client, and client use class to talk to
|
* This is returned from OM to client, and client use class to talk to
|
||||||
* datanode. Also, this is the metadata written to om.db on server side.
|
* datanode. Also, this is the metadata written to om.db on server side.
|
||||||
*/
|
*/
|
||||||
public final class OmKeyInfo {
|
public final class OmKeyInfo extends WithMetadata {
|
||||||
private final String volumeName;
|
private final String volumeName;
|
||||||
private final String bucketName;
|
private final String bucketName;
|
||||||
// name of key client specified
|
// name of key client specified
|
||||||
@ -46,11 +49,12 @@ public final class OmKeyInfo {
|
|||||||
private HddsProtos.ReplicationFactor factor;
|
private HddsProtos.ReplicationFactor factor;
|
||||||
|
|
||||||
@SuppressWarnings("parameternumber")
|
@SuppressWarnings("parameternumber")
|
||||||
private OmKeyInfo(String volumeName, String bucketName, String keyName,
|
OmKeyInfo(String volumeName, String bucketName, String keyName,
|
||||||
List<OmKeyLocationInfoGroup> versions, long dataSize,
|
List<OmKeyLocationInfoGroup> versions, long dataSize,
|
||||||
long creationTime, long modificationTime,
|
long creationTime, long modificationTime,
|
||||||
HddsProtos.ReplicationType type,
|
HddsProtos.ReplicationType type,
|
||||||
HddsProtos.ReplicationFactor factor) {
|
HddsProtos.ReplicationFactor factor,
|
||||||
|
Map<String, String> metadata) {
|
||||||
this.volumeName = volumeName;
|
this.volumeName = volumeName;
|
||||||
this.bucketName = bucketName;
|
this.bucketName = bucketName;
|
||||||
this.keyName = keyName;
|
this.keyName = keyName;
|
||||||
@ -71,6 +75,7 @@ private OmKeyInfo(String volumeName, String bucketName, String keyName,
|
|||||||
this.modificationTime = modificationTime;
|
this.modificationTime = modificationTime;
|
||||||
this.factor = factor;
|
this.factor = factor;
|
||||||
this.type = type;
|
this.type = type;
|
||||||
|
this.metadata = metadata;
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getVolumeName() {
|
public String getVolumeName() {
|
||||||
@ -217,6 +222,12 @@ public static class Builder {
|
|||||||
private HddsProtos.ReplicationType type;
|
private HddsProtos.ReplicationType type;
|
||||||
private HddsProtos.ReplicationFactor factor;
|
private HddsProtos.ReplicationFactor factor;
|
||||||
private boolean isMultipartKey;
|
private boolean isMultipartKey;
|
||||||
|
private Map<String, String> metadata;
|
||||||
|
|
||||||
|
public Builder() {
|
||||||
|
this.metadata = new HashMap<>();
|
||||||
|
omKeyLocationInfoGroups = new ArrayList<>();
|
||||||
|
}
|
||||||
|
|
||||||
public Builder setVolumeName(String volume) {
|
public Builder setVolumeName(String volume) {
|
||||||
this.volumeName = volume;
|
this.volumeName = volume;
|
||||||
@ -269,10 +280,20 @@ public Builder setIsMultipartKey(boolean isMultipart) {
|
|||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Builder addMetadata(String key, String value) {
|
||||||
|
metadata.put(key, value);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder addAllMetadata(Map<String, String> newMetadata) {
|
||||||
|
metadata.putAll(newMetadata);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
public OmKeyInfo build() {
|
public OmKeyInfo build() {
|
||||||
return new OmKeyInfo(
|
return new OmKeyInfo(
|
||||||
volumeName, bucketName, keyName, omKeyLocationInfoGroups,
|
volumeName, bucketName, keyName, omKeyLocationInfoGroups,
|
||||||
dataSize, creationTime, modificationTime, type, factor);
|
dataSize, creationTime, modificationTime, type, factor, metadata);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -292,6 +313,7 @@ public KeyInfo getProtobuf() {
|
|||||||
.setLatestVersion(latestVersion)
|
.setLatestVersion(latestVersion)
|
||||||
.setCreationTime(creationTime)
|
.setCreationTime(creationTime)
|
||||||
.setModificationTime(modificationTime)
|
.setModificationTime(modificationTime)
|
||||||
|
.addAllMetadata(KeyValueUtil.toProtobuf(metadata))
|
||||||
.build();
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -307,7 +329,34 @@ public static OmKeyInfo getFromProtobuf(KeyInfo keyInfo) {
|
|||||||
keyInfo.getCreationTime(),
|
keyInfo.getCreationTime(),
|
||||||
keyInfo.getModificationTime(),
|
keyInfo.getModificationTime(),
|
||||||
keyInfo.getType(),
|
keyInfo.getType(),
|
||||||
keyInfo.getFactor());
|
keyInfo.getFactor(),
|
||||||
|
KeyValueUtil.getFromProtobuf(keyInfo.getMetadataList()));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o) {
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
OmKeyInfo omKeyInfo = (OmKeyInfo) o;
|
||||||
|
return dataSize == omKeyInfo.dataSize &&
|
||||||
|
creationTime == omKeyInfo.creationTime &&
|
||||||
|
modificationTime == omKeyInfo.modificationTime &&
|
||||||
|
volumeName.equals(omKeyInfo.volumeName) &&
|
||||||
|
bucketName.equals(omKeyInfo.bucketName) &&
|
||||||
|
keyName.equals(omKeyInfo.keyName) &&
|
||||||
|
Objects
|
||||||
|
.equals(keyLocationVersions, omKeyInfo.keyLocationVersions) &&
|
||||||
|
type == omKeyInfo.type &&
|
||||||
|
factor == omKeyInfo.factor &&
|
||||||
|
Objects.equals(metadata, omKeyInfo.metadata);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(volumeName, bucketName, keyName);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -17,34 +17,29 @@
|
|||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.om.helpers;
|
package org.apache.hadoop.ozone.om.helpers;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
|
||||||
import org.apache.hadoop.ozone.OzoneConsts;
|
|
||||||
import org.apache.hadoop.ozone.audit.Auditable;
|
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
|
||||||
.OzoneManagerProtocolProtos.OzoneAclInfo;
|
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
|
||||||
.OzoneManagerProtocolProtos.VolumeInfo;
|
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.KeyValue;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.LinkedHashMap;
|
import java.util.LinkedHashMap;
|
||||||
import java.util.LinkedList;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
import org.apache.hadoop.ozone.OzoneConsts;
|
||||||
|
import org.apache.hadoop.ozone.audit.Auditable;
|
||||||
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneAclInfo;
|
||||||
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeInfo;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A class that encapsulates the OmVolumeArgs Args.
|
* A class that encapsulates the OmVolumeArgs Args.
|
||||||
*/
|
*/
|
||||||
public final class OmVolumeArgs implements Auditable{
|
public final class OmVolumeArgs extends WithMetadata implements Auditable {
|
||||||
private final String adminName;
|
private final String adminName;
|
||||||
private final String ownerName;
|
private final String ownerName;
|
||||||
private final String volume;
|
private final String volume;
|
||||||
private final long creationTime;
|
private final long creationTime;
|
||||||
private final long quotaInBytes;
|
private final long quotaInBytes;
|
||||||
private final Map<String, String> keyValueMap;
|
|
||||||
private final OmOzoneAclMap aclMap;
|
private final OmOzoneAclMap aclMap;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -53,18 +48,18 @@ public final class OmVolumeArgs implements Auditable{
|
|||||||
* @param ownerName - Volume owner's name
|
* @param ownerName - Volume owner's name
|
||||||
* @param volume - volume name
|
* @param volume - volume name
|
||||||
* @param quotaInBytes - Volume Quota in bytes.
|
* @param quotaInBytes - Volume Quota in bytes.
|
||||||
* @param keyValueMap - keyValue map.
|
* @param metadata - metadata map for custom key/value data.
|
||||||
* @param aclMap - User to access rights map.
|
* @param aclMap - User to access rights map.
|
||||||
* @param creationTime - Volume creation time.
|
* @param creationTime - Volume creation time.
|
||||||
*/
|
*/
|
||||||
private OmVolumeArgs(String adminName, String ownerName, String volume,
|
private OmVolumeArgs(String adminName, String ownerName, String volume,
|
||||||
long quotaInBytes, Map<String, String> keyValueMap,
|
long quotaInBytes, Map<String, String> metadata,
|
||||||
OmOzoneAclMap aclMap, long creationTime) {
|
OmOzoneAclMap aclMap, long creationTime) {
|
||||||
this.adminName = adminName;
|
this.adminName = adminName;
|
||||||
this.ownerName = ownerName;
|
this.ownerName = ownerName;
|
||||||
this.volume = volume;
|
this.volume = volume;
|
||||||
this.quotaInBytes = quotaInBytes;
|
this.quotaInBytes = quotaInBytes;
|
||||||
this.keyValueMap = keyValueMap;
|
this.metadata = metadata;
|
||||||
this.aclMap = aclMap;
|
this.aclMap = aclMap;
|
||||||
this.creationTime = creationTime;
|
this.creationTime = creationTime;
|
||||||
}
|
}
|
||||||
@ -109,10 +104,6 @@ public long getQuotaInBytes() {
|
|||||||
return quotaInBytes;
|
return quotaInBytes;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Map<String, String> getKeyValueMap() {
|
|
||||||
return keyValueMap;
|
|
||||||
}
|
|
||||||
|
|
||||||
public OmOzoneAclMap getAclMap() {
|
public OmOzoneAclMap getAclMap() {
|
||||||
return aclMap;
|
return aclMap;
|
||||||
}
|
}
|
||||||
@ -145,14 +136,14 @@ public static class Builder {
|
|||||||
private String volume;
|
private String volume;
|
||||||
private long creationTime;
|
private long creationTime;
|
||||||
private long quotaInBytes;
|
private long quotaInBytes;
|
||||||
private Map<String, String> keyValueMap;
|
private Map<String, String> metadata;
|
||||||
private OmOzoneAclMap aclMap;
|
private OmOzoneAclMap aclMap;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructs a builder.
|
* Constructs a builder.
|
||||||
*/
|
*/
|
||||||
public Builder() {
|
public Builder() {
|
||||||
keyValueMap = new HashMap<>();
|
metadata = new HashMap<>();
|
||||||
aclMap = new OmOzoneAclMap();
|
aclMap = new OmOzoneAclMap();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -182,7 +173,14 @@ public Builder setQuotaInBytes(long quota) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public Builder addMetadata(String key, String value) {
|
public Builder addMetadata(String key, String value) {
|
||||||
keyValueMap.put(key, value); // overwrite if present.
|
metadata.put(key, value); // overwrite if present.
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder addAllMetadata(Map<String, String> additionalMetaData) {
|
||||||
|
if (additionalMetaData != null) {
|
||||||
|
metadata.putAll(additionalMetaData);
|
||||||
|
}
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -200,16 +198,13 @@ public OmVolumeArgs build() {
|
|||||||
Preconditions.checkNotNull(ownerName);
|
Preconditions.checkNotNull(ownerName);
|
||||||
Preconditions.checkNotNull(volume);
|
Preconditions.checkNotNull(volume);
|
||||||
return new OmVolumeArgs(adminName, ownerName, volume, quotaInBytes,
|
return new OmVolumeArgs(adminName, ownerName, volume, quotaInBytes,
|
||||||
keyValueMap, aclMap, creationTime);
|
metadata, aclMap, creationTime);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public VolumeInfo getProtobuf() {
|
public VolumeInfo getProtobuf() {
|
||||||
List<KeyValue> metadataList = new LinkedList<>();
|
|
||||||
for (Map.Entry<String, String> entry : keyValueMap.entrySet()) {
|
|
||||||
metadataList.add(KeyValue.newBuilder().setKey(entry.getKey()).
|
|
||||||
setValue(entry.getValue()).build());
|
|
||||||
}
|
|
||||||
List<OzoneAclInfo> aclList = aclMap.ozoneAclGetProtobuf();
|
List<OzoneAclInfo> aclList = aclMap.ozoneAclGetProtobuf();
|
||||||
|
|
||||||
return VolumeInfo.newBuilder()
|
return VolumeInfo.newBuilder()
|
||||||
@ -217,7 +212,7 @@ public VolumeInfo getProtobuf() {
|
|||||||
.setOwnerName(ownerName)
|
.setOwnerName(ownerName)
|
||||||
.setVolume(volume)
|
.setVolume(volume)
|
||||||
.setQuotaInBytes(quotaInBytes)
|
.setQuotaInBytes(quotaInBytes)
|
||||||
.addAllMetadata(metadataList)
|
.addAllMetadata(KeyValueUtil.toProtobuf(metadata))
|
||||||
.addAllVolumeAcls(aclList)
|
.addAllVolumeAcls(aclList)
|
||||||
.setCreationTime(
|
.setCreationTime(
|
||||||
creationTime == 0 ? System.currentTimeMillis() : creationTime)
|
creationTime == 0 ? System.currentTimeMillis() : creationTime)
|
||||||
@ -225,14 +220,17 @@ public VolumeInfo getProtobuf() {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public static OmVolumeArgs getFromProtobuf(VolumeInfo volInfo) {
|
public static OmVolumeArgs getFromProtobuf(VolumeInfo volInfo) {
|
||||||
Map<String, String> kvMap = volInfo.getMetadataList().stream()
|
|
||||||
.collect(Collectors.toMap(KeyValue::getKey,
|
|
||||||
KeyValue::getValue));
|
|
||||||
OmOzoneAclMap aclMap =
|
OmOzoneAclMap aclMap =
|
||||||
OmOzoneAclMap.ozoneAclGetFromProtobuf(volInfo.getVolumeAclsList());
|
OmOzoneAclMap.ozoneAclGetFromProtobuf(volInfo.getVolumeAclsList());
|
||||||
|
|
||||||
return new OmVolumeArgs(volInfo.getAdminName(), volInfo.getOwnerName(),
|
return new OmVolumeArgs(
|
||||||
volInfo.getVolume(), volInfo.getQuotaInBytes(), kvMap, aclMap,
|
volInfo.getAdminName(),
|
||||||
|
volInfo.getOwnerName(),
|
||||||
|
volInfo.getVolume(),
|
||||||
|
volInfo.getQuotaInBytes(),
|
||||||
|
KeyValueUtil.getFromProtobuf(volInfo.getMetadataList()),
|
||||||
|
aclMap,
|
||||||
volInfo.getCreationTime());
|
volInfo.getCreationTime());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -0,0 +1,44 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.apache.hadoop.ozone.om.helpers;
|
||||||
|
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Mixin class to handle custom metadata.
|
||||||
|
*/
|
||||||
|
public class WithMetadata {
|
||||||
|
|
||||||
|
protected Map<String, String> metadata = new HashMap<>();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Custom key value metadata.
|
||||||
|
*/
|
||||||
|
public Map<String, String> getMetadata() {
|
||||||
|
return metadata;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set custom key value metadata.
|
||||||
|
*/
|
||||||
|
public void setMetadata(Map<String, String> metadata) {
|
||||||
|
this.metadata = metadata;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -26,6 +26,7 @@
|
|||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.ipc.ProtobufHelper;
|
import org.apache.hadoop.ipc.ProtobufHelper;
|
||||||
import org.apache.hadoop.ipc.ProtocolTranslator;
|
import org.apache.hadoop.ipc.ProtocolTranslator;
|
||||||
|
import org.apache.hadoop.ozone.om.helpers.KeyValueUtil;
|
||||||
import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
|
||||||
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||||
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
||||||
@ -652,6 +653,11 @@ public OpenKeySession openKey(OmKeyArgs args) throws IOException {
|
|||||||
keyArgs.setDataSize(args.getDataSize());
|
keyArgs.setDataSize(args.getDataSize());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (args.getMetadata() != null && args.getMetadata().size() > 0) {
|
||||||
|
keyArgs.addAllMetadata(KeyValueUtil.toProtobuf(args.getMetadata()));
|
||||||
|
}
|
||||||
|
req.setKeyArgs(keyArgs.build());
|
||||||
|
|
||||||
if (args.getMultipartUploadID() != null) {
|
if (args.getMultipartUploadID() != null) {
|
||||||
keyArgs.setMultipartUploadID(args.getMultipartUploadID());
|
keyArgs.setMultipartUploadID(args.getMultipartUploadID());
|
||||||
}
|
}
|
||||||
|
@ -317,6 +317,8 @@ message BucketInfo {
|
|||||||
required bool isVersionEnabled = 4 [default = false];
|
required bool isVersionEnabled = 4 [default = false];
|
||||||
required StorageTypeProto storageType = 5 [default = DISK];
|
required StorageTypeProto storageType = 5 [default = DISK];
|
||||||
required uint64 creationTime = 6;
|
required uint64 creationTime = 6;
|
||||||
|
repeated hadoop.hdds.KeyValue metadata = 7;
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
enum StorageTypeProto {
|
enum StorageTypeProto {
|
||||||
@ -333,6 +335,7 @@ message BucketArgs {
|
|||||||
repeated OzoneAclInfo removeAcls = 4;
|
repeated OzoneAclInfo removeAcls = 4;
|
||||||
optional bool isVersionEnabled = 5;
|
optional bool isVersionEnabled = 5;
|
||||||
optional StorageTypeProto storageType = 6;
|
optional StorageTypeProto storageType = 6;
|
||||||
|
repeated hadoop.hdds.KeyValue metadata = 7;
|
||||||
}
|
}
|
||||||
|
|
||||||
message OzoneAclInfo {
|
message OzoneAclInfo {
|
||||||
@ -409,6 +412,7 @@ message KeyArgs {
|
|||||||
optional bool isMultipartKey = 8;
|
optional bool isMultipartKey = 8;
|
||||||
optional string multipartUploadID = 9;
|
optional string multipartUploadID = 9;
|
||||||
optional uint32 multipartNumber = 10;
|
optional uint32 multipartNumber = 10;
|
||||||
|
repeated hadoop.hdds.KeyValue metadata = 11;
|
||||||
}
|
}
|
||||||
|
|
||||||
message KeyLocation {
|
message KeyLocation {
|
||||||
@ -436,6 +440,8 @@ message KeyInfo {
|
|||||||
required uint64 creationTime = 8;
|
required uint64 creationTime = 8;
|
||||||
required uint64 modificationTime = 9;
|
required uint64 modificationTime = 9;
|
||||||
optional uint64 latestVersion = 10;
|
optional uint64 latestVersion = 10;
|
||||||
|
repeated hadoop.hdds.KeyValue metadata = 11;
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
message CreateKeyRequest {
|
message CreateKeyRequest {
|
||||||
|
@ -0,0 +1,46 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.apache.hadoop.ozone.om.helpers;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hdds.protocol.StorageType;
|
||||||
|
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test BucketInfo.
|
||||||
|
*/
|
||||||
|
public class TestOmBucketInfo {
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void protobufConversion() {
|
||||||
|
OmBucketInfo bucket = OmBucketInfo.newBuilder()
|
||||||
|
.setBucketName("bucket")
|
||||||
|
.setVolumeName("vol1")
|
||||||
|
.setCreationTime(1L)
|
||||||
|
.setIsVersionEnabled(false)
|
||||||
|
.setStorageType(StorageType.ARCHIVE)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
OmBucketInfo afterSerialization =
|
||||||
|
OmBucketInfo.getFromProtobuf(bucket.getProtobuf());
|
||||||
|
|
||||||
|
Assert.assertEquals(bucket, afterSerialization);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,52 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.apache.hadoop.ozone.om.helpers;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
||||||
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
||||||
|
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo.Builder;
|
||||||
|
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test OmKeyInfo.
|
||||||
|
*/
|
||||||
|
public class TestOmKeyInfo {
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void protobufConversion() {
|
||||||
|
OmKeyInfo key = new Builder()
|
||||||
|
.setKeyName("key1")
|
||||||
|
.setBucketName("bucket")
|
||||||
|
.setVolumeName("vol1")
|
||||||
|
.setCreationTime(123L)
|
||||||
|
.setModificationTime(123L)
|
||||||
|
.setDataSize(123L)
|
||||||
|
.setReplicationFactor(ReplicationFactor.THREE)
|
||||||
|
.setReplicationType(ReplicationType.RATIS)
|
||||||
|
.addMetadata("key1", "value1")
|
||||||
|
.addMetadata("key2", "value2")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
OmKeyInfo keyAfterSerialization =
|
||||||
|
OmKeyInfo.getFromProtobuf(key.getProtobuf());
|
||||||
|
|
||||||
|
Assert.assertEquals(key, keyAfterSerialization);
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,21 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF 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
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* 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.apache.hadoop.ozone.om.helpers;
|
||||||
|
/**
|
||||||
|
* Unit tests of helpers.
|
||||||
|
*/
|
@ -46,6 +46,7 @@
|
|||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.InetSocketAddress;
|
import java.net.InetSocketAddress;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
|
|
||||||
@ -350,7 +351,7 @@ public void testPutKey()
|
|||||||
|
|
||||||
OzoneOutputStream out = bucket.createKey(keyName,
|
OzoneOutputStream out = bucket.createKey(keyName,
|
||||||
value.getBytes().length, ReplicationType.STAND_ALONE,
|
value.getBytes().length, ReplicationType.STAND_ALONE,
|
||||||
ReplicationFactor.ONE);
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
out.write(value.getBytes());
|
out.write(value.getBytes());
|
||||||
out.close();
|
out.close();
|
||||||
OzoneKey key = bucket.getKey(keyName);
|
OzoneKey key = bucket.getKey(keyName);
|
||||||
@ -376,7 +377,7 @@ public void testDeleteKey()
|
|||||||
OzoneBucket bucket = volume.getBucket(bucketName);
|
OzoneBucket bucket = volume.getBucket(bucketName);
|
||||||
OzoneOutputStream out = bucket.createKey(keyName,
|
OzoneOutputStream out = bucket.createKey(keyName,
|
||||||
value.getBytes().length, ReplicationType.STAND_ALONE,
|
value.getBytes().length, ReplicationType.STAND_ALONE,
|
||||||
ReplicationFactor.ONE);
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
out.write(value.getBytes());
|
out.write(value.getBytes());
|
||||||
out.close();
|
out.close();
|
||||||
OzoneKey key = bucket.getKey(keyName);
|
OzoneKey key = bucket.getKey(keyName);
|
||||||
@ -398,7 +399,7 @@ public void testRenameKey()
|
|||||||
OzoneBucket bucket = volume.getBucket(bucketName);
|
OzoneBucket bucket = volume.getBucket(bucketName);
|
||||||
OzoneOutputStream out = bucket.createKey(fromKeyName,
|
OzoneOutputStream out = bucket.createKey(fromKeyName,
|
||||||
value.getBytes().length, ReplicationType.STAND_ALONE,
|
value.getBytes().length, ReplicationType.STAND_ALONE,
|
||||||
ReplicationFactor.ONE);
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
out.write(value.getBytes());
|
out.write(value.getBytes());
|
||||||
out.close();
|
out.close();
|
||||||
OzoneKey key = bucket.getKey(fromKeyName);
|
OzoneKey key = bucket.getKey(fromKeyName);
|
||||||
@ -427,7 +428,7 @@ public void testGetKeyDetails() throws IOException, OzoneException {
|
|||||||
String keyValue = RandomStringUtils.random(128);
|
String keyValue = RandomStringUtils.random(128);
|
||||||
OzoneOutputStream out = bucket.createKey(keyName,
|
OzoneOutputStream out = bucket.createKey(keyName,
|
||||||
keyValue.getBytes().length, ReplicationType.STAND_ALONE,
|
keyValue.getBytes().length, ReplicationType.STAND_ALONE,
|
||||||
ReplicationFactor.ONE);
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
out.write(keyValue.getBytes());
|
out.write(keyValue.getBytes());
|
||||||
out.close();
|
out.close();
|
||||||
|
|
||||||
|
@ -37,6 +37,7 @@
|
|||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
@ -109,7 +110,7 @@ public void testBCSID() throws Exception {
|
|||||||
OzoneOutputStream key =
|
OzoneOutputStream key =
|
||||||
objectStore.getVolume(volumeName).getBucket(bucketName)
|
objectStore.getVolume(volumeName).getBucket(bucketName)
|
||||||
.createKey("ratis", 1024, ReplicationType.RATIS,
|
.createKey("ratis", 1024, ReplicationType.RATIS,
|
||||||
ReplicationFactor.ONE);
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
key.write("ratis".getBytes());
|
key.write("ratis".getBytes());
|
||||||
key.close();
|
key.close();
|
||||||
|
|
||||||
|
@ -39,6 +39,7 @@
|
|||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
@ -117,7 +118,7 @@ public void testContainerStateMachineFailures() throws Exception {
|
|||||||
OzoneOutputStream key =
|
OzoneOutputStream key =
|
||||||
objectStore.getVolume(volumeName).getBucket(bucketName)
|
objectStore.getVolume(volumeName).getBucket(bucketName)
|
||||||
.createKey("ratis", 1024, ReplicationType.RATIS,
|
.createKey("ratis", 1024, ReplicationType.RATIS,
|
||||||
ReplicationFactor.ONE);
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
// First write and flush creates a container in the datanode
|
// First write and flush creates a container in the datanode
|
||||||
key.write("ratis".getBytes());
|
key.write("ratis".getBytes());
|
||||||
key.flush();
|
key.flush();
|
||||||
|
@ -39,6 +39,7 @@
|
|||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
@ -115,7 +116,7 @@ public void testContainerStateMachineFailures() throws Exception {
|
|||||||
OzoneOutputStream key =
|
OzoneOutputStream key =
|
||||||
objectStore.getVolume(volumeName).getBucket(bucketName)
|
objectStore.getVolume(volumeName).getBucket(bucketName)
|
||||||
.createKey("ratis", 1024, ReplicationType.RATIS,
|
.createKey("ratis", 1024, ReplicationType.RATIS,
|
||||||
ReplicationFactor.ONE);
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
// First write and flush creates a container in the datanode
|
// First write and flush creates a container in the datanode
|
||||||
key.write("ratis".getBytes());
|
key.write("ratis".getBytes());
|
||||||
key.flush();
|
key.flush();
|
||||||
|
@ -21,6 +21,7 @@
|
|||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
@ -59,6 +60,7 @@
|
|||||||
import org.apache.hadoop.ozone.client.OzoneKeyDetails;
|
import org.apache.hadoop.ozone.client.OzoneKeyDetails;
|
||||||
import org.apache.hadoop.ozone.client.OzoneKeyLocation;
|
import org.apache.hadoop.ozone.client.OzoneKeyLocation;
|
||||||
import org.apache.hadoop.ozone.client.OzoneVolume;
|
import org.apache.hadoop.ozone.client.OzoneVolume;
|
||||||
|
import org.apache.hadoop.ozone.client.VolumeArgs;
|
||||||
import org.apache.hadoop.ozone.client.io.KeyOutputStream;
|
import org.apache.hadoop.ozone.client.io.KeyOutputStream;
|
||||||
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
|
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
|
||||||
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
||||||
@ -200,6 +202,39 @@ public void testDeleteVolume()
|
|||||||
store.getVolume(volumeName);
|
store.getVolume(volumeName);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCreateVolumeWithMetadata()
|
||||||
|
throws IOException, OzoneException {
|
||||||
|
String volumeName = UUID.randomUUID().toString();
|
||||||
|
VolumeArgs volumeArgs = VolumeArgs.newBuilder()
|
||||||
|
.addMetadata("key1", "val1")
|
||||||
|
.build();
|
||||||
|
store.createVolume(volumeName, volumeArgs);
|
||||||
|
OzoneVolume volume = store.getVolume(volumeName);
|
||||||
|
|
||||||
|
Assert.assertEquals("val1", volume.getMetadata().get("key1"));
|
||||||
|
Assert.assertEquals(volumeName, volume.getName());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCreateBucketWithMetadata()
|
||||||
|
throws IOException, OzoneException {
|
||||||
|
long currentTime = Time.now();
|
||||||
|
String volumeName = UUID.randomUUID().toString();
|
||||||
|
String bucketName = UUID.randomUUID().toString();
|
||||||
|
store.createVolume(volumeName);
|
||||||
|
OzoneVolume volume = store.getVolume(volumeName);
|
||||||
|
BucketArgs args = BucketArgs.newBuilder()
|
||||||
|
.addMetadata("key1", "value1").build();
|
||||||
|
volume.createBucket(bucketName, args);
|
||||||
|
OzoneBucket bucket = volume.getBucket(bucketName);
|
||||||
|
Assert.assertEquals(bucketName, bucket.getName());
|
||||||
|
Assert.assertNotNull(bucket.getMetadata());
|
||||||
|
Assert.assertEquals("value1", bucket.getMetadata().get("key1"));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCreateBucket()
|
public void testCreateBucket()
|
||||||
throws IOException, OzoneException {
|
throws IOException, OzoneException {
|
||||||
@ -518,7 +553,7 @@ public void testPutKey()
|
|||||||
|
|
||||||
OzoneOutputStream out = bucket.createKey(keyName,
|
OzoneOutputStream out = bucket.createKey(keyName,
|
||||||
value.getBytes().length, ReplicationType.STAND_ALONE,
|
value.getBytes().length, ReplicationType.STAND_ALONE,
|
||||||
ReplicationFactor.ONE);
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
out.write(value.getBytes());
|
out.write(value.getBytes());
|
||||||
out.close();
|
out.close();
|
||||||
OzoneKey key = bucket.getKey(keyName);
|
OzoneKey key = bucket.getKey(keyName);
|
||||||
@ -549,7 +584,7 @@ public void testValidateBlockLengthWithCommitKey() throws IOException {
|
|||||||
|
|
||||||
// create the initial key with size 0, write will allocate the first block.
|
// create the initial key with size 0, write will allocate the first block.
|
||||||
OzoneOutputStream out = bucket.createKey(keyName, 0,
|
OzoneOutputStream out = bucket.createKey(keyName, 0,
|
||||||
ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
ReplicationType.STAND_ALONE, ReplicationFactor.ONE, new HashMap<>());
|
||||||
out.write(value.getBytes());
|
out.write(value.getBytes());
|
||||||
out.close();
|
out.close();
|
||||||
OmKeyArgs.Builder builder = new OmKeyArgs.Builder();
|
OmKeyArgs.Builder builder = new OmKeyArgs.Builder();
|
||||||
@ -587,7 +622,7 @@ public void testPutKeyRatisOneNode()
|
|||||||
|
|
||||||
OzoneOutputStream out = bucket.createKey(keyName,
|
OzoneOutputStream out = bucket.createKey(keyName,
|
||||||
value.getBytes().length, ReplicationType.RATIS,
|
value.getBytes().length, ReplicationType.RATIS,
|
||||||
ReplicationFactor.ONE);
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
out.write(value.getBytes());
|
out.write(value.getBytes());
|
||||||
out.close();
|
out.close();
|
||||||
OzoneKey key = bucket.getKey(keyName);
|
OzoneKey key = bucket.getKey(keyName);
|
||||||
@ -622,7 +657,7 @@ public void testPutKeyRatisThreeNodes()
|
|||||||
|
|
||||||
OzoneOutputStream out = bucket.createKey(keyName,
|
OzoneOutputStream out = bucket.createKey(keyName,
|
||||||
value.getBytes().length, ReplicationType.RATIS,
|
value.getBytes().length, ReplicationType.RATIS,
|
||||||
ReplicationFactor.THREE);
|
ReplicationFactor.THREE, new HashMap<>());
|
||||||
out.write(value.getBytes());
|
out.write(value.getBytes());
|
||||||
out.close();
|
out.close();
|
||||||
OzoneKey key = bucket.getKey(keyName);
|
OzoneKey key = bucket.getKey(keyName);
|
||||||
@ -656,7 +691,7 @@ public void testPutKeyAndGetKeyThreeNodes()
|
|||||||
|
|
||||||
OzoneOutputStream out = bucket
|
OzoneOutputStream out = bucket
|
||||||
.createKey(keyName, value.getBytes().length, ReplicationType.RATIS,
|
.createKey(keyName, value.getBytes().length, ReplicationType.RATIS,
|
||||||
ReplicationFactor.THREE);
|
ReplicationFactor.THREE, new HashMap<>());
|
||||||
KeyOutputStream groupOutputStream =
|
KeyOutputStream groupOutputStream =
|
||||||
(KeyOutputStream) out.getOutputStream();
|
(KeyOutputStream) out.getOutputStream();
|
||||||
XceiverClientManager manager = groupOutputStream.getXceiverClientManager();
|
XceiverClientManager manager = groupOutputStream.getXceiverClientManager();
|
||||||
@ -757,7 +792,7 @@ public void testGetKeyDetails() throws IOException, OzoneException {
|
|||||||
// create the initial key with size 0, write will allocate the first block.
|
// create the initial key with size 0, write will allocate the first block.
|
||||||
OzoneOutputStream out = bucket.createKey(keyName,
|
OzoneOutputStream out = bucket.createKey(keyName,
|
||||||
keyValue.getBytes().length, ReplicationType.STAND_ALONE,
|
keyValue.getBytes().length, ReplicationType.STAND_ALONE,
|
||||||
ReplicationFactor.ONE);
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
out.write(keyValue.getBytes());
|
out.write(keyValue.getBytes());
|
||||||
out.close();
|
out.close();
|
||||||
|
|
||||||
@ -844,7 +879,7 @@ public void testReadKeyWithCorruptedData() throws IOException {
|
|||||||
// Write data into a key
|
// Write data into a key
|
||||||
OzoneOutputStream out = bucket.createKey(keyName,
|
OzoneOutputStream out = bucket.createKey(keyName,
|
||||||
value.getBytes().length, ReplicationType.STAND_ALONE,
|
value.getBytes().length, ReplicationType.STAND_ALONE,
|
||||||
ReplicationFactor.ONE);
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
out.write(value.getBytes());
|
out.write(value.getBytes());
|
||||||
out.close();
|
out.close();
|
||||||
|
|
||||||
@ -925,7 +960,7 @@ public void testDeleteKey()
|
|||||||
OzoneBucket bucket = volume.getBucket(bucketName);
|
OzoneBucket bucket = volume.getBucket(bucketName);
|
||||||
OzoneOutputStream out = bucket.createKey(keyName,
|
OzoneOutputStream out = bucket.createKey(keyName,
|
||||||
value.getBytes().length, ReplicationType.STAND_ALONE,
|
value.getBytes().length, ReplicationType.STAND_ALONE,
|
||||||
ReplicationFactor.ONE);
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
out.write(value.getBytes());
|
out.write(value.getBytes());
|
||||||
out.close();
|
out.close();
|
||||||
OzoneKey key = bucket.getKey(keyName);
|
OzoneKey key = bucket.getKey(keyName);
|
||||||
@ -947,7 +982,7 @@ public void testRenameKey()
|
|||||||
OzoneBucket bucket = volume.getBucket(bucketName);
|
OzoneBucket bucket = volume.getBucket(bucketName);
|
||||||
OzoneOutputStream out = bucket.createKey(fromKeyName,
|
OzoneOutputStream out = bucket.createKey(fromKeyName,
|
||||||
value.getBytes().length, ReplicationType.STAND_ALONE,
|
value.getBytes().length, ReplicationType.STAND_ALONE,
|
||||||
ReplicationFactor.ONE);
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
out.write(value.getBytes());
|
out.write(value.getBytes());
|
||||||
out.close();
|
out.close();
|
||||||
OzoneKey key = bucket.getKey(fromKeyName);
|
OzoneKey key = bucket.getKey(fromKeyName);
|
||||||
@ -1141,22 +1176,26 @@ public void testListKey()
|
|||||||
byte[] value = RandomStringUtils.randomAscii(10240).getBytes();
|
byte[] value = RandomStringUtils.randomAscii(10240).getBytes();
|
||||||
OzoneOutputStream one = volAbucketA.createKey(
|
OzoneOutputStream one = volAbucketA.createKey(
|
||||||
keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
|
keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
|
||||||
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE,
|
||||||
|
new HashMap<>());
|
||||||
one.write(value);
|
one.write(value);
|
||||||
one.close();
|
one.close();
|
||||||
OzoneOutputStream two = volAbucketB.createKey(
|
OzoneOutputStream two = volAbucketB.createKey(
|
||||||
keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
|
keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
|
||||||
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE,
|
||||||
|
new HashMap<>());
|
||||||
two.write(value);
|
two.write(value);
|
||||||
two.close();
|
two.close();
|
||||||
OzoneOutputStream three = volBbucketA.createKey(
|
OzoneOutputStream three = volBbucketA.createKey(
|
||||||
keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
|
keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
|
||||||
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE,
|
||||||
|
new HashMap<>());
|
||||||
three.write(value);
|
three.write(value);
|
||||||
three.close();
|
three.close();
|
||||||
OzoneOutputStream four = volBbucketB.createKey(
|
OzoneOutputStream four = volBbucketB.createKey(
|
||||||
keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
|
keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
|
||||||
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE,
|
||||||
|
new HashMap<>());
|
||||||
four.write(value);
|
four.write(value);
|
||||||
four.close();
|
four.close();
|
||||||
}
|
}
|
||||||
@ -1170,22 +1209,26 @@ public void testListKey()
|
|||||||
byte[] value = RandomStringUtils.randomAscii(10240).getBytes();
|
byte[] value = RandomStringUtils.randomAscii(10240).getBytes();
|
||||||
OzoneOutputStream one = volAbucketA.createKey(
|
OzoneOutputStream one = volAbucketA.createKey(
|
||||||
keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
|
keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
|
||||||
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE,
|
||||||
|
new HashMap<>());
|
||||||
one.write(value);
|
one.write(value);
|
||||||
one.close();
|
one.close();
|
||||||
OzoneOutputStream two = volAbucketB.createKey(
|
OzoneOutputStream two = volAbucketB.createKey(
|
||||||
keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
|
keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
|
||||||
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE,
|
||||||
|
new HashMap<>());
|
||||||
two.write(value);
|
two.write(value);
|
||||||
two.close();
|
two.close();
|
||||||
OzoneOutputStream three = volBbucketA.createKey(
|
OzoneOutputStream three = volBbucketA.createKey(
|
||||||
keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
|
keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
|
||||||
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE,
|
||||||
|
new HashMap<>());
|
||||||
three.write(value);
|
three.write(value);
|
||||||
three.close();
|
three.close();
|
||||||
OzoneOutputStream four = volBbucketB.createKey(
|
OzoneOutputStream four = volBbucketB.createKey(
|
||||||
keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
|
keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
|
||||||
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
value.length, ReplicationType.STAND_ALONE, ReplicationFactor.ONE,
|
||||||
|
new HashMap<>());
|
||||||
four.write(value);
|
four.write(value);
|
||||||
four.close();
|
four.close();
|
||||||
}
|
}
|
||||||
|
@ -51,6 +51,7 @@
|
|||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
|
|
||||||
import static org.apache.hadoop.hdds.HddsConfigKeys.OZONE_METADATA_DIRS;
|
import static org.apache.hadoop.hdds.HddsConfigKeys.OZONE_METADATA_DIRS;
|
||||||
@ -134,7 +135,7 @@ public void testPutKeySuccessWithBlockToken() throws Exception {
|
|||||||
|
|
||||||
try (OzoneOutputStream out = bucket.createKey(keyName,
|
try (OzoneOutputStream out = bucket.createKey(keyName,
|
||||||
value.getBytes().length, ReplicationType.STAND_ALONE,
|
value.getBytes().length, ReplicationType.STAND_ALONE,
|
||||||
ReplicationFactor.ONE)) {
|
ReplicationFactor.ONE, new HashMap<>())) {
|
||||||
out.write(value.getBytes());
|
out.write(value.getBytes());
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -177,7 +178,7 @@ public void testKeyOpFailureWithoutBlockToken() throws Exception {
|
|||||||
|
|
||||||
try(OzoneOutputStream out = bucket.createKey(keyName,
|
try(OzoneOutputStream out = bucket.createKey(keyName,
|
||||||
value.getBytes().length, ReplicationType.STAND_ALONE,
|
value.getBytes().length, ReplicationType.STAND_ALONE,
|
||||||
ReplicationFactor.ONE)) {
|
ReplicationFactor.ONE, new HashMap<>())) {
|
||||||
LambdaTestUtils.intercept(IOException.class, "UNAUTHENTICATED: Fail " +
|
LambdaTestUtils.intercept(IOException.class, "UNAUTHENTICATED: Fail " +
|
||||||
"to find any token ",
|
"to find any token ",
|
||||||
() -> out.write(value.getBytes()));
|
() -> out.write(value.getBytes()));
|
||||||
|
@ -18,13 +18,33 @@
|
|||||||
|
|
||||||
package org.apache.hadoop.ozone.container;
|
package org.apache.hadoop.ozone.container;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import java.io.IOException;
|
||||||
|
import java.net.ServerSocket;
|
||||||
import java.security.MessageDigest;
|
import java.security.MessageDigest;
|
||||||
|
import java.security.NoSuchAlgorithmException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.LinkedList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.Random;
|
||||||
|
import java.util.UUID;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.StorageUnit;
|
import org.apache.hadoop.conf.StorageUnit;
|
||||||
import org.apache.hadoop.hdds.HddsUtils;
|
import org.apache.hadoop.hdds.HddsUtils;
|
||||||
|
import org.apache.hadoop.hdds.client.BlockID;
|
||||||
import org.apache.hadoop.hdds.client.ReplicationType;
|
import org.apache.hadoop.hdds.client.ReplicationType;
|
||||||
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
||||||
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto.Builder;
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto.Builder;
|
||||||
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
|
||||||
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
||||||
|
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
||||||
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
|
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
|
||||||
import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
|
import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
|
||||||
import org.apache.hadoop.ozone.HddsDatanodeService;
|
import org.apache.hadoop.ozone.HddsDatanodeService;
|
||||||
@ -35,31 +55,18 @@
|
|||||||
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
||||||
import org.apache.hadoop.ozone.common.Checksum;
|
import org.apache.hadoop.ozone.common.Checksum;
|
||||||
import org.apache.hadoop.ozone.common.OzoneChecksumException;
|
import org.apache.hadoop.ozone.common.OzoneChecksumException;
|
||||||
|
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
|
||||||
|
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
|
||||||
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
|
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
|
||||||
import org.apache.hadoop.ozone.container.common.interfaces.Container;
|
import org.apache.hadoop.ozone.container.common.interfaces.Container;
|
||||||
import org.apache.hadoop.security.token.Token;
|
import org.apache.hadoop.security.token.Token;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
|
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
|
||||||
import org.apache.hadoop.hdds.client.BlockID;
|
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
|
||||||
.ContainerCommandRequestProto;
|
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
|
||||||
.ContainerCommandResponseProto;
|
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
|
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
|
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
|
|
||||||
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.net.ServerSocket;
|
|
||||||
import java.security.NoSuchAlgorithmException;
|
|
||||||
import java.util.*;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Helpers for container tests.
|
* Helpers for container tests.
|
||||||
*/
|
*/
|
||||||
@ -668,7 +675,7 @@ public static OzoneOutputStream createKey(String keyName,
|
|||||||
org.apache.hadoop.hdds.client.ReplicationFactor.ONE :
|
org.apache.hadoop.hdds.client.ReplicationFactor.ONE :
|
||||||
org.apache.hadoop.hdds.client.ReplicationFactor.THREE;
|
org.apache.hadoop.hdds.client.ReplicationFactor.THREE;
|
||||||
return objectStore.getVolume(volumeName).getBucket(bucketName)
|
return objectStore.getVolume(volumeName).getBucket(bucketName)
|
||||||
.createKey(keyName, size, type, factor);
|
.createKey(keyName, size, type, factor, new HashMap<>());
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void validateData(String keyName, byte[] data,
|
public static void validateData(String keyName, byte[] data,
|
||||||
|
@ -58,6 +58,7 @@
|
|||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
@ -133,7 +134,7 @@ public void testBlockDeletion() throws Exception {
|
|||||||
String keyName = UUID.randomUUID().toString();
|
String keyName = UUID.randomUUID().toString();
|
||||||
|
|
||||||
OzoneOutputStream out = bucket.createKey(keyName, value.getBytes().length,
|
OzoneOutputStream out = bucket.createKey(keyName, value.getBytes().length,
|
||||||
ReplicationType.STAND_ALONE, ReplicationFactor.ONE);
|
ReplicationType.STAND_ALONE, ReplicationFactor.ONE, new HashMap<>());
|
||||||
for (int i = 0; i < 100; i++) {
|
for (int i = 0; i < 100; i++) {
|
||||||
out.write(value.getBytes());
|
out.write(value.getBytes());
|
||||||
}
|
}
|
||||||
|
@ -44,6 +44,7 @@
|
|||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
|
|
||||||
@ -92,7 +93,7 @@ public static void shutdown() {
|
|||||||
public void testIfCloseContainerCommandHandlerIsInvoked() throws Exception {
|
public void testIfCloseContainerCommandHandlerIsInvoked() throws Exception {
|
||||||
OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
|
OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
|
||||||
.createKey("standalone", 1024, ReplicationType.RATIS,
|
.createKey("standalone", 1024, ReplicationType.RATIS,
|
||||||
ReplicationFactor.ONE);
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
key.write("standalone".getBytes());
|
key.write("standalone".getBytes());
|
||||||
key.close();
|
key.close();
|
||||||
|
|
||||||
@ -146,7 +147,7 @@ public void testCloseContainerViaStandAlone()
|
|||||||
|
|
||||||
OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
|
OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
|
||||||
.createKey("standalone", 1024, ReplicationType.RATIS,
|
.createKey("standalone", 1024, ReplicationType.RATIS,
|
||||||
ReplicationFactor.ONE);
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
key.write("standalone".getBytes());
|
key.write("standalone".getBytes());
|
||||||
key.close();
|
key.close();
|
||||||
|
|
||||||
@ -195,7 +196,7 @@ public void testCloseContainerViaRatis() throws IOException,
|
|||||||
|
|
||||||
OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
|
OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
|
||||||
.createKey("ratis", 1024, ReplicationType.RATIS,
|
.createKey("ratis", 1024, ReplicationType.RATIS,
|
||||||
ReplicationFactor.THREE);
|
ReplicationFactor.THREE, new HashMap<>());
|
||||||
key.write("ratis".getBytes());
|
key.write("ratis".getBytes());
|
||||||
key.close();
|
key.close();
|
||||||
|
|
||||||
|
@ -41,6 +41,7 @@
|
|||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -66,7 +67,7 @@ public void test() throws IOException, TimeoutException, InterruptedException,
|
|||||||
objectStore.getVolume("test").createBucket("test");
|
objectStore.getVolume("test").createBucket("test");
|
||||||
OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
|
OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
|
||||||
.createKey("test", 1024, ReplicationType.STAND_ALONE,
|
.createKey("test", 1024, ReplicationType.STAND_ALONE,
|
||||||
ReplicationFactor.ONE);
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
key.write("test".getBytes());
|
key.write("test".getBytes());
|
||||||
key.close();
|
key.close();
|
||||||
|
|
||||||
|
@ -40,6 +40,7 @@
|
|||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.HashMap;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class tests container report with DN container state info.
|
* This class tests container report with DN container state info.
|
||||||
@ -93,7 +94,7 @@ public void testContainerReportKeyWrite() throws Exception {
|
|||||||
OzoneOutputStream key =
|
OzoneOutputStream key =
|
||||||
objectStore.getVolume(volumeName).getBucket(bucketName)
|
objectStore.getVolume(volumeName).getBucket(bucketName)
|
||||||
.createKey(keyName, keySize, ReplicationType.STAND_ALONE,
|
.createKey(keyName, keySize, ReplicationType.STAND_ALONE,
|
||||||
ReplicationFactor.ONE);
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
String dataString = RandomStringUtils.randomAlphabetic(keySize);
|
String dataString = RandomStringUtils.randomAlphabetic(keySize);
|
||||||
key.write(dataString.getBytes());
|
key.write(dataString.getBytes());
|
||||||
key.close();
|
key.close();
|
||||||
|
@ -47,6 +47,7 @@
|
|||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
|
import java.util.HashMap;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* End-to-end testing of Ozone REST operations.
|
* End-to-end testing of Ozone REST operations.
|
||||||
@ -107,7 +108,8 @@ private void putKey(OzoneBucket bucket, String keyName, String keyData)
|
|||||||
throws IOException {
|
throws IOException {
|
||||||
try (
|
try (
|
||||||
OzoneOutputStream ozoneOutputStream = bucket
|
OzoneOutputStream ozoneOutputStream = bucket
|
||||||
.createKey(keyName, 0, replicationType, replicationFactor);
|
.createKey(keyName, 0, replicationType, replicationFactor,
|
||||||
|
new HashMap<>());
|
||||||
InputStream inputStream = IOUtils.toInputStream(keyData, UTF_8)) {
|
InputStream inputStream = IOUtils.toInputStream(keyData, UTF_8)) {
|
||||||
IOUtils.copy(inputStream, ozoneOutputStream);
|
IOUtils.copy(inputStream, ozoneOutputStream);
|
||||||
}
|
}
|
||||||
|
@ -80,6 +80,7 @@
|
|||||||
import java.nio.file.Path;
|
import java.nio.file.Path;
|
||||||
import java.nio.file.Paths;
|
import java.nio.file.Paths;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
@ -259,7 +260,8 @@ private String putKey() throws Exception {
|
|||||||
|
|
||||||
try (
|
try (
|
||||||
OzoneOutputStream ozoneOutputStream = bucket
|
OzoneOutputStream ozoneOutputStream = bucket
|
||||||
.createKey(keyName, 0, replicationType, replicationFactor);
|
.createKey(keyName, 0, replicationType, replicationFactor,
|
||||||
|
new HashMap<>());
|
||||||
InputStream fileInputStream = new FileInputStream(file)) {
|
InputStream fileInputStream = new FileInputStream(file)) {
|
||||||
IOUtils.copy(fileInputStream, ozoneOutputStream);
|
IOUtils.copy(fileInputStream, ozoneOutputStream);
|
||||||
}
|
}
|
||||||
@ -291,7 +293,7 @@ static void runTestPutKey(PutHelper helper) throws Exception {
|
|||||||
String newkeyName = OzoneUtils.getRequestID().toLowerCase();
|
String newkeyName = OzoneUtils.getRequestID().toLowerCase();
|
||||||
OzoneOutputStream ozoneOutputStream = helperClient
|
OzoneOutputStream ozoneOutputStream = helperClient
|
||||||
.createKey(helper.getVol().getName(), helper.getBucket().getName(),
|
.createKey(helper.getVol().getName(), helper.getBucket().getName(),
|
||||||
newkeyName, 0, replicationType, replicationFactor);
|
newkeyName, 0, replicationType, replicationFactor, new HashMap<>());
|
||||||
ozoneOutputStream.close();
|
ozoneOutputStream.close();
|
||||||
keyList = helperClient
|
keyList = helperClient
|
||||||
.listKeys(helper.getVol().getName(), helper.getBucket().getName(), null,
|
.listKeys(helper.getVol().getName(), helper.getBucket().getName(), null,
|
||||||
@ -302,7 +304,7 @@ static void runTestPutKey(PutHelper helper) throws Exception {
|
|||||||
try {
|
try {
|
||||||
ozoneOutputStream = helperClient
|
ozoneOutputStream = helperClient
|
||||||
.createKey("invalid-volume", helper.getBucket().getName(), newkeyName,
|
.createKey("invalid-volume", helper.getBucket().getName(), newkeyName,
|
||||||
0, replicationType, replicationFactor);
|
0, replicationType, replicationFactor, new HashMap<>());
|
||||||
ozoneOutputStream.close();
|
ozoneOutputStream.close();
|
||||||
fail("Put key should have thrown"
|
fail("Put key should have thrown"
|
||||||
+ " when using invalid volume name.");
|
+ " when using invalid volume name.");
|
||||||
@ -314,7 +316,7 @@ static void runTestPutKey(PutHelper helper) throws Exception {
|
|||||||
try {
|
try {
|
||||||
ozoneOutputStream = helperClient
|
ozoneOutputStream = helperClient
|
||||||
.createKey(helper.getVol().getName(), "invalid-bucket", newkeyName, 0,
|
.createKey(helper.getVol().getName(), "invalid-bucket", newkeyName, 0,
|
||||||
replicationType, replicationFactor);
|
replicationType, replicationFactor, new HashMap<>());
|
||||||
ozoneOutputStream.close();
|
ozoneOutputStream.close();
|
||||||
fail("Put key should have thrown "
|
fail("Put key should have thrown "
|
||||||
+ "when using invalid bucket name.");
|
+ "when using invalid bucket name.");
|
||||||
@ -488,7 +490,8 @@ static void runTestPutAndListKey(PutHelper helper) throws Exception {
|
|||||||
String newkeyName = "list-key" + x;
|
String newkeyName = "list-key" + x;
|
||||||
try (
|
try (
|
||||||
OzoneOutputStream ozoneOutputStream = helper.getBucket()
|
OzoneOutputStream ozoneOutputStream = helper.getBucket()
|
||||||
.createKey(newkeyName, 0, replicationType, replicationFactor);
|
.createKey(newkeyName, 0, replicationType, replicationFactor,
|
||||||
|
new HashMap<>());
|
||||||
InputStream fileInputStream = new FileInputStream(helper.getFile())) {
|
InputStream fileInputStream = new FileInputStream(helper.getFile())) {
|
||||||
IOUtils.copy(fileInputStream, ozoneOutputStream);
|
IOUtils.copy(fileInputStream, ozoneOutputStream);
|
||||||
}
|
}
|
||||||
|
@ -107,6 +107,7 @@ public void createBucket(OmBucketInfo bucketInfo) throws IOException {
|
|||||||
.setStorageType(bucketInfo.getStorageType())
|
.setStorageType(bucketInfo.getStorageType())
|
||||||
.setIsVersionEnabled(bucketInfo.getIsVersionEnabled())
|
.setIsVersionEnabled(bucketInfo.getIsVersionEnabled())
|
||||||
.setCreationTime(Time.now())
|
.setCreationTime(Time.now())
|
||||||
|
.addAllMetadata(bucketInfo.getMetadata())
|
||||||
.build();
|
.build();
|
||||||
metadataManager.getBucketTable().put(bucketKey,
|
metadataManager.getBucketTable().put(bucketKey,
|
||||||
omBucketInfo);
|
omBucketInfo);
|
||||||
@ -182,6 +183,7 @@ public void setBucketProperty(OmBucketArgs args) throws IOException {
|
|||||||
OmBucketInfo.Builder bucketInfoBuilder = OmBucketInfo.newBuilder();
|
OmBucketInfo.Builder bucketInfoBuilder = OmBucketInfo.newBuilder();
|
||||||
bucketInfoBuilder.setVolumeName(oldBucketInfo.getVolumeName())
|
bucketInfoBuilder.setVolumeName(oldBucketInfo.getVolumeName())
|
||||||
.setBucketName(oldBucketInfo.getBucketName());
|
.setBucketName(oldBucketInfo.getBucketName());
|
||||||
|
bucketInfoBuilder.addAllMetadata(args.getMetadata());
|
||||||
|
|
||||||
//Check ACLs to update
|
//Check ACLs to update
|
||||||
if (args.getAddAcls() != null || args.getRemoveAcls() != null) {
|
if (args.getAddAcls() != null || args.getRemoveAcls() != null) {
|
||||||
|
@ -334,6 +334,7 @@ public OpenKeySession openKey(OmKeyArgs args) throws IOException {
|
|||||||
// For this upload part we don't need to check in KeyTable. As this
|
// For this upload part we don't need to check in KeyTable. As this
|
||||||
// is not an actual key, it is a part of the key.
|
// is not an actual key, it is a part of the key.
|
||||||
keyInfo = createKeyInfo(args, locations, factor, type, size);
|
keyInfo = createKeyInfo(args, locations, factor, type, size);
|
||||||
|
//TODO args.getMetadata
|
||||||
openVersion = 0;
|
openVersion = 0;
|
||||||
} else {
|
} else {
|
||||||
keyInfo = metadataManager.getKeyTable().get(objectKey);
|
keyInfo = metadataManager.getKeyTable().get(objectKey);
|
||||||
|
@ -321,6 +321,8 @@ public void run() {
|
|||||||
private void saveOmMetrics() {
|
private void saveOmMetrics() {
|
||||||
try {
|
try {
|
||||||
boolean success;
|
boolean success;
|
||||||
|
Files.createDirectories(
|
||||||
|
getTempMetricsStorageFile().getParentFile().toPath());
|
||||||
try (BufferedWriter writer = new BufferedWriter(
|
try (BufferedWriter writer = new BufferedWriter(
|
||||||
new OutputStreamWriter(new FileOutputStream(
|
new OutputStreamWriter(new FileOutputStream(
|
||||||
getTempMetricsStorageFile()), "UTF-8"))) {
|
getTempMetricsStorageFile()), "UTF-8"))) {
|
||||||
|
@ -20,6 +20,7 @@
|
|||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.FileInputStream;
|
import java.io.FileInputStream;
|
||||||
|
import java.util.HashMap;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.conf.StorageUnit;
|
import org.apache.hadoop.conf.StorageUnit;
|
||||||
@ -105,7 +106,7 @@ public Void call() throws Exception {
|
|||||||
OzoneBucket bucket = vol.getBucket(bucketName);
|
OzoneBucket bucket = vol.getBucket(bucketName);
|
||||||
OzoneOutputStream outputStream = bucket
|
OzoneOutputStream outputStream = bucket
|
||||||
.createKey(keyName, dataFile.length(), replicationType,
|
.createKey(keyName, dataFile.length(), replicationType,
|
||||||
replicationFactor);
|
replicationFactor, new HashMap<>());
|
||||||
FileInputStream fileInputStream = new FileInputStream(dataFile);
|
FileInputStream fileInputStream = new FileInputStream(dataFile);
|
||||||
IOUtils.copyBytes(fileInputStream, outputStream, (int) conf
|
IOUtils.copyBytes(fileInputStream, outputStream, (int) conf
|
||||||
.getStorageSize(OZONE_SCM_CHUNK_SIZE_KEY, OZONE_SCM_CHUNK_SIZE_DEFAULT,
|
.getStorageSize(OZONE_SCM_CHUNK_SIZE_KEY, OZONE_SCM_CHUNK_SIZE_DEFAULT,
|
||||||
|
@ -207,7 +207,8 @@ public FSDataOutputStream create(Path f, FsPermission permission,
|
|||||||
}
|
}
|
||||||
|
|
||||||
OzoneOutputStream ozoneOutputStream =
|
OzoneOutputStream ozoneOutputStream =
|
||||||
bucket.createKey(key, 0, replicationType, replicationFactor);
|
bucket.createKey(key, 0, replicationType, replicationFactor,
|
||||||
|
new HashMap<>());
|
||||||
// We pass null to FSDataOutputStream so it won't count writes that
|
// We pass null to FSDataOutputStream so it won't count writes that
|
||||||
// are being buffered to a file
|
// are being buffered to a file
|
||||||
return new FSDataOutputStream(
|
return new FSDataOutputStream(
|
||||||
@ -770,7 +771,8 @@ private boolean isDirectory(OzoneKey key) {
|
|||||||
private boolean createDirectory(String keyName) {
|
private boolean createDirectory(String keyName) {
|
||||||
try {
|
try {
|
||||||
LOG.trace("creating dir for key:{}", keyName);
|
LOG.trace("creating dir for key:{}", keyName);
|
||||||
bucket.createKey(keyName, 0, replicationType, replicationFactor).close();
|
bucket.createKey(keyName, 0, replicationType, replicationFactor,
|
||||||
|
new HashMap<>()).close();
|
||||||
return true;
|
return true;
|
||||||
} catch (IOException ioe) {
|
} catch (IOException ioe) {
|
||||||
LOG.error("create key failed for key:{}", keyName, ioe);
|
LOG.error("create key failed for key:{}", keyName, ioe);
|
||||||
|
@ -41,6 +41,7 @@
|
|||||||
import java.time.ZoneId;
|
import java.time.ZoneId;
|
||||||
import java.time.ZonedDateTime;
|
import java.time.ZonedDateTime;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.TreeMap;
|
import java.util.TreeMap;
|
||||||
@ -163,7 +164,7 @@ public Response put(
|
|||||||
OzoneBucket bucket = getBucket(bucketName);
|
OzoneBucket bucket = getBucket(bucketName);
|
||||||
|
|
||||||
output = bucket.createKey(keyPath, length, replicationType,
|
output = bucket.createKey(keyPath, length, replicationType,
|
||||||
replicationFactor);
|
replicationFactor, new HashMap<>());
|
||||||
|
|
||||||
if ("STREAMING-AWS4-HMAC-SHA256-PAYLOAD"
|
if ("STREAMING-AWS4-HMAC-SHA256-PAYLOAD"
|
||||||
.equals(headers.getHeaderString("x-amz-content-sha256"))) {
|
.equals(headers.getHeaderString("x-amz-content-sha256"))) {
|
||||||
@ -619,7 +620,7 @@ private CopyObjectResponse copyObject(String copyHeader,
|
|||||||
sourceInputStream = sourceOzoneBucket.readKey(sourceKey);
|
sourceInputStream = sourceOzoneBucket.readKey(sourceKey);
|
||||||
|
|
||||||
destOutputStream = destOzoneBucket.createKey(destkey, sourceKeyLen,
|
destOutputStream = destOzoneBucket.createKey(destkey, sourceKeyLen,
|
||||||
replicationType, replicationFactor);
|
replicationType, replicationFactor, new HashMap<>());
|
||||||
|
|
||||||
IOUtils.copy(sourceInputStream, destOutputStream);
|
IOUtils.copy(sourceInputStream, destOutputStream);
|
||||||
|
|
||||||
|
@ -83,12 +83,15 @@ public OzoneBucketStub(
|
|||||||
@Override
|
@Override
|
||||||
public OzoneOutputStream createKey(String key, long size) throws IOException {
|
public OzoneOutputStream createKey(String key, long size) throws IOException {
|
||||||
return createKey(key, size, ReplicationType.STAND_ALONE,
|
return createKey(key, size, ReplicationType.STAND_ALONE,
|
||||||
ReplicationFactor.ONE);
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public OzoneOutputStream createKey(String key, long size,
|
public OzoneOutputStream createKey(String key, long size,
|
||||||
ReplicationType type, ReplicationFactor factor) throws IOException {
|
ReplicationType type,
|
||||||
|
ReplicationFactor factor,
|
||||||
|
Map<String, String> metadata)
|
||||||
|
throws IOException {
|
||||||
ByteArrayOutputStream byteArrayOutputStream =
|
ByteArrayOutputStream byteArrayOutputStream =
|
||||||
new ByteArrayOutputStream((int) size) {
|
new ByteArrayOutputStream((int) size) {
|
||||||
@Override
|
@Override
|
||||||
@ -101,7 +104,7 @@ public void close() throws IOException {
|
|||||||
size,
|
size,
|
||||||
System.currentTimeMillis(),
|
System.currentTimeMillis(),
|
||||||
System.currentTimeMillis(),
|
System.currentTimeMillis(),
|
||||||
new ArrayList<>(), type
|
new ArrayList<>(), type, metadata
|
||||||
));
|
));
|
||||||
super.close();
|
super.close();
|
||||||
}
|
}
|
||||||
|
@ -22,6 +22,7 @@
|
|||||||
import javax.ws.rs.core.Response;
|
import javax.ws.rs.core.Response;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.time.format.DateTimeFormatter;
|
import java.time.format.DateTimeFormatter;
|
||||||
|
import java.util.HashMap;
|
||||||
|
|
||||||
import org.apache.hadoop.hdds.client.ReplicationFactor;
|
import org.apache.hadoop.hdds.client.ReplicationFactor;
|
||||||
import org.apache.hadoop.hdds.client.ReplicationType;
|
import org.apache.hadoop.hdds.client.ReplicationType;
|
||||||
@ -71,7 +72,7 @@ public void testHeadObject() throws Exception {
|
|||||||
String value = RandomStringUtils.randomAlphanumeric(32);
|
String value = RandomStringUtils.randomAlphanumeric(32);
|
||||||
OzoneOutputStream out = bucket.createKey("key1",
|
OzoneOutputStream out = bucket.createKey("key1",
|
||||||
value.getBytes(UTF_8).length, ReplicationType.STAND_ALONE,
|
value.getBytes(UTF_8).length, ReplicationType.STAND_ALONE,
|
||||||
ReplicationFactor.ONE);
|
ReplicationFactor.ONE, new HashMap<>());
|
||||||
out.write(value.getBytes(UTF_8));
|
out.write(value.getBytes(UTF_8));
|
||||||
out.close();
|
out.close();
|
||||||
|
|
||||||
|
@ -24,6 +24,7 @@
|
|||||||
import java.text.SimpleDateFormat;
|
import java.text.SimpleDateFormat;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
import java.util.concurrent.ArrayBlockingQueue;
|
import java.util.concurrent.ArrayBlockingQueue;
|
||||||
import java.util.concurrent.BlockingQueue;
|
import java.util.concurrent.BlockingQueue;
|
||||||
@ -592,7 +593,7 @@ public void run() {
|
|||||||
key, bucket, volume);
|
key, bucket, volume);
|
||||||
long keyCreateStart = System.nanoTime();
|
long keyCreateStart = System.nanoTime();
|
||||||
OzoneOutputStream os =
|
OzoneOutputStream os =
|
||||||
bucket.createKey(key, keySize, type, factor);
|
bucket.createKey(key, keySize, type, factor, new HashMap<>());
|
||||||
long keyCreationDuration = System.nanoTime() - keyCreateStart;
|
long keyCreationDuration = System.nanoTime() - keyCreateStart;
|
||||||
histograms.get(FreonOps.KEY_CREATE.ordinal())
|
histograms.get(FreonOps.KEY_CREATE.ordinal())
|
||||||
.update(keyCreationDuration);
|
.update(keyCreationDuration);
|
||||||
|
Loading…
Reference in New Issue
Block a user