@@ -125,9 +125,6 @@ public final class OzoneConfigKeys {
* */
public static final String OZONE_ADMINISTRATORS_WILDCARD = "*";
- public static final String OZONE_CLIENT_PROTOCOL =
- "ozone.client.protocol";
-
public static final String OZONE_CLIENT_STREAM_BUFFER_FLUSH_SIZE =
"ozone.client.stream.buffer.flush.size";
@@ -461,19 +461,6 @@
there is no wait.
</description>
</property>
- <property>
- <name>ozone.client.protocol</name>
- <value>org.apache.hadoop.ozone.client.rpc.RpcClient</value>
- <tag>OZONE, CLIENT, MANAGEMENT</tag>
- <description>Protocol class to be used by the client to connect to ozone
- cluster.
- The build-in implementation includes:
- org.apache.hadoop.ozone.client.rpc.RpcClient for RPC
- org.apache.hadoop.ozone.client.rest.RestClient for REST
- The default is the RpClient. Please do not change this unless you have a
- very good understanding of what you are doing.
- </description>
- </property>
<property>
<name>ozone.client.socket.timeout</name>
<value>5000ms</value>
@@ -1261,7 +1248,7 @@
<name>hdds.datanode.plugins</name>
- <value>org.apache.hadoop.ozone.web.OzoneHddsDatanodeService</value>
+ <value></value>
<description>
Comma-separated list of HDDS datanode plug-ins to be activated when
HDDS service starts as part of datanode.
@@ -18,6 +18,7 @@
package org.apache.hadoop.ozone.client;
+import com.fasterxml.jackson.annotation.JsonIgnore;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
@@ -211,6 +212,7 @@ public class OzoneBucket extends WithMetadata {
*
* @return acls
*/
+ @JsonIgnore
public List<OzoneAcl> getAcls() throws IOException {
return proxy.getAcl(ozoneObj);
}
@@ -17,38 +17,27 @@
-import org.apache.hadoop.ozone.client.rest.OzoneException;
/**
* This exception is thrown by the Ozone Clients.
-public class OzoneClientException extends OzoneException {
- /**
- * Constructor that allows the shortMessage.
- *
- * @param shortMessage Short Message
- */
- public OzoneClientException(String shortMessage) {
- super(0, shortMessage, shortMessage);
+public class OzoneClientException extends Exception {
+ public OzoneClientException() {
+ }
+
+ public OzoneClientException(String s) {
+ super(s);
+ public OzoneClientException(String s, Throwable throwable) {
+ super(s, throwable);
- * Constructor that allows a shortMessage and an exception.
- * @param shortMessage short message
- * @param ex exception
- public OzoneClientException(String shortMessage, Exception ex) {
- super(0, shortMessage, shortMessage, ex);
+ public OzoneClientException(Throwable throwable) {
+ super(throwable);
- * Constructor that allows the shortMessage and a longer message.
- * @param message long error message
- public OzoneClientException(String shortMessage, String message) {
- super(0, shortMessage, message);
+ public OzoneClientException(String s, Throwable throwable, boolean b,
+ boolean b1) {
+ super(s, throwable, b, b1);
@@ -18,37 +18,22 @@
-import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.lang.reflect.Proxy;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OmUtils;
import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
-import org.apache.hadoop.ozone.client.rest.RestClient;
import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import com.google.common.base.Preconditions;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Proxy;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CLIENT_PROTOCOL;
-import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
-import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY;
- * Factory class to create different types of OzoneClients.
- * Based on <code>ozone.client.protocol</code>, it decides which
- * protocol to use for the communication.
- * Default value is
- * <code>org.apache.hadoop.ozone.client.rpc.RpcClient</code>.<br>
- * OzoneClientFactory constructs a proxy using
- * {@link OzoneClientInvocationHandler}
- * and creates OzoneClient instance with it.
- * {@link OzoneClientInvocationHandler} dispatches the call to
- * underlying {@link ClientProtocol} implementation.
+ * Factory class to create OzoneClients.
public final class OzoneClientFactory {
@@ -87,9 +72,7 @@ public final class OzoneClientFactory {
public static OzoneClient getClient(Configuration config)
throws IOException {
Preconditions.checkNotNull(config);
- Class<? extends ClientProtocol> clazz = (Class<? extends ClientProtocol>)
- config.getClass(OZONE_CLIENT_PROTOCOL, RpcClient.class);
- return getClient(getClientProtocol(clazz, config), config);
+ return getClient(getClientProtocol(config), config);
@@ -166,85 +149,7 @@ public final class OzoneClientFactory {
public static OzoneClient getRpcClient(Configuration config)
- return getClient(getClientProtocol(RpcClient.class, config),
- config);
- }
- * Returns an OzoneClient which will use REST protocol.
- * @param omHost
- * hostname of OzoneManager to connect.
- * @return OzoneClient
- * @throws IOException
- public static OzoneClient getRestClient(String omHost)
- throws IOException {
- Configuration config = new OzoneConfiguration();
- int port = OmUtils.getOmRestPort(config);
- return getRestClient(omHost, port, config);
- * @param omHttpPort
- * HTTP port of OzoneManager.
- public static OzoneClient getRestClient(String omHost, Integer omHttpPort)
- return getRestClient(omHost, omHttpPort, new OzoneConfiguration());
- * @param config
- * Configuration to be used for OzoneClient creation
- public static OzoneClient getRestClient(String omHost, Integer omHttpPort,
- Configuration config)
- Preconditions.checkNotNull(omHost);
- Preconditions.checkNotNull(omHttpPort);
- Preconditions.checkNotNull(config);
- config.set(OZONE_OM_HTTP_ADDRESS_KEY, omHost + ":" + omHttpPort);
- return getRestClient(config);
- public static OzoneClient getRestClient(Configuration config)
- return getClient(getClientProtocol(RestClient.class, config),
+ return getClient(getClientProtocol(config),
config);
@@ -270,8 +175,6 @@ public final class OzoneClientFactory {
* Returns an instance of Protocol class.
- * @param protocolClass
- * Class object of the ClientProtocol.
* @param config
* Configuration used to initialize ClientProtocol.
@@ -280,23 +183,15 @@ public final class OzoneClientFactory {
* @throws IOException
- private static ClientProtocol getClientProtocol(
- Class<? extends ClientProtocol> protocolClass, Configuration config)
+ private static ClientProtocol getClientProtocol(Configuration config)
try {
- LOG.debug("Using {} as client protocol.",
- protocolClass.getCanonicalName());
- Constructor<? extends ClientProtocol> ctor =
- protocolClass.getConstructor(Configuration.class);
- return ctor.newInstance(config);
+ return new RpcClient(config);
} catch (Exception e) {
- final String message = "Couldn't create protocol " + protocolClass;
+ final String message = "Couldn't create RpcClient protocol";
LOG.error(message + " exception: ", e);
if (e.getCause() instanceof IOException) {
throw (IOException) e.getCause();
- } else if (e instanceof InvocationTargetException) {
- throw new IOException(message,
- ((InvocationTargetException) e).getTargetException());
} else {
throw new IOException(message, e);
@@ -17,106 +17,15 @@
-import java.util.ArrayList;
-import java.util.List;
import java.util.concurrent.TimeUnit;
-import org.apache.hadoop.hdds.client.OzoneQuota;
-import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
import org.apache.hadoop.io.retry.RetryPolicies;
import org.apache.hadoop.io.retry.RetryPolicy;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.client.rest.response.BucketInfo;
-import org.apache.hadoop.ozone.client.rest.response.KeyInfo;
-import org.apache.hadoop.ozone.client.rest.response.KeyInfoDetails;
-import org.apache.hadoop.ozone.client.rest.response.KeyLocation;
-import org.apache.hadoop.ozone.client.rest.response.VolumeInfo;
-import org.apache.hadoop.ozone.client.rest.response.VolumeOwner;
/** A utility class for OzoneClient. */
public final class OzoneClientUtils {
private OzoneClientUtils() {}
- * Returns a BucketInfo object constructed using fields of the input
- * OzoneBucket object.
- * @param bucket OzoneBucket instance from which BucketInfo object needs to
- * be created.
- * @return BucketInfo instance
- public static BucketInfo asBucketInfo(OzoneBucket bucket) throws IOException {
- BucketInfo bucketInfo =
- new BucketInfo(bucket.getVolumeName(), bucket.getName());
- bucketInfo
- .setCreatedOn(HddsClientUtils.formatDateTime(bucket.getCreationTime()));
- bucketInfo.setStorageType(bucket.getStorageType());
- bucketInfo.setVersioning(
- OzoneConsts.Versioning.getVersioning(bucket.getVersioning()));
- bucketInfo.setEncryptionKeyName(
- bucket.getEncryptionKeyName()==null? "N/A" :
- bucket.getEncryptionKeyName());
- return bucketInfo;
- * Returns a VolumeInfo object constructed using fields of the input
- * OzoneVolume object.
- * @param volume OzoneVolume instance from which VolumeInfo object needs to
- * @return VolumeInfo instance
- public static VolumeInfo asVolumeInfo(OzoneVolume volume) {
- VolumeInfo volumeInfo = new VolumeInfo(volume.getName(),
- HddsClientUtils.formatDateTime(volume.getCreationTime()),
- volume.getOwner());
- volumeInfo.setQuota(OzoneQuota.getOzoneQuota(volume.getQuota()));
- volumeInfo.setOwner(new VolumeOwner(volume.getOwner()));
- return volumeInfo;
- * Returns a KeyInfo object constructed using fields of the input
- * OzoneKey object.
- * @param key OzoneKey instance from which KeyInfo object needs to
- * @return KeyInfo instance
- public static KeyInfo asKeyInfo(OzoneKey key) {
- KeyInfo keyInfo = new KeyInfo();
- keyInfo.setKeyName(key.getName());
- keyInfo.setCreatedOn(HddsClientUtils.formatDateTime(key.getCreationTime()));
- keyInfo.setModifiedOn(
- HddsClientUtils.formatDateTime(key.getModificationTime()));
- keyInfo.setSize(key.getDataSize());
- return keyInfo;
- * Returns a KeyInfoDetails object constructed using fields of the input
- * OzoneKeyDetails object.
- * @param key OzoneKeyDetails instance from which KeyInfo object needs to
- * @return KeyInfoDetails instance
- public static KeyInfoDetails asKeyInfoDetails(OzoneKeyDetails key) {
- KeyInfoDetails keyInfo = new KeyInfoDetails();
- List<KeyLocation> keyLocations = new ArrayList<>();
- key.getOzoneKeyLocations().forEach((a) -> keyLocations.add(new KeyLocation(
- a.getContainerID(), a.getLocalID(), a.getLength(), a.getOffset())));
- keyInfo.setKeyLocation(keyLocations);
- keyInfo.setFileEncryptionInfo(key.getFileEncryptionInfo());
public static RetryPolicy createRetryPolicy(int maxRetryCount,
long retryInterval) {
@@ -1,36 +0,0 @@
-/**
- * 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
- * http://www.apache.org/licenses/LICENSE-2.0
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-package org.apache.hadoop.ozone.client.rest;
-import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
-import java.util.Random;
- * Default selector randomly picks one of the REST Server from the list.
-public class DefaultRestServerSelector implements RestServerSelector {
- @Override
- public ServiceInfo getRestServer(List<ServiceInfo> restServices) {
- return restServices.get(
- new Random().nextInt(restServices.size()));
-}
@@ -1,45 +0,0 @@
-/*
-import javax.ws.rs.core.Response;
-import javax.ws.rs.ext.ExceptionMapper;
-import org.slf4j.MDC;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
- * Class the represents various errors returned by the
- * Object Layer.
-public class OzoneExceptionMapper implements ExceptionMapper<OzoneException> {
- private static final Logger LOG =
- LoggerFactory.getLogger(OzoneExceptionMapper.class);
- public Response toResponse(OzoneException exception) {
- LOG.debug("Returning exception. ex: {}", exception.toJsonString());
- MDC.clear();
- return Response.status((int)exception.getHttpCode())
- .entity(exception.toJsonString()).build();
@@ -1,1130 +0,0 @@
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.base.Strings;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.key.KeyProvider;
-import org.apache.hadoop.hdds.protocol.StorageType;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.ozone.OzoneAcl;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.client.*;
-import org.apache.hadoop.hdds.client.ReplicationFactor;
-import org.apache.hadoop.hdds.client.ReplicationType;
-import org.apache.hadoop.ozone.client.VolumeArgs;
-import org.apache.hadoop.ozone.client.io.OzoneInputStream;
-import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
-import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
-import org.apache.hadoop.ozone.client.rest.headers.Header;
-import org.apache.hadoop.ozone.client.rpc.OzoneKMSUtil;
-import org.apache.hadoop.ozone.om.OMConfigKeys;
-import org.apache.hadoop.ozone.om.ha.OMFailoverProxyProvider;
-import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
-import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
-import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
-import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServicePort;
-import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
-import org.apache.hadoop.ozone.security.acl.OzoneObj;
-import org.apache.hadoop.ozone.web.response.ListBuckets;
-import org.apache.hadoop.ozone.web.response.ListKeys;
-import org.apache.hadoop.ozone.web.response.ListVolumes;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.Time;
-import org.apache.http.HttpEntity;
-import org.apache.http.HttpHeaders;
-import org.apache.http.HttpResponse;
-import org.apache.http.client.config.RequestConfig;
-import org.apache.http.client.methods.HttpDelete;
-import org.apache.http.client.methods.HttpGet;
-import org.apache.http.client.methods.HttpPost;
-import org.apache.http.client.methods.HttpPut;
-import org.apache.http.client.methods.HttpUriRequest;
-import org.apache.http.client.utils.URIBuilder;
-import org.apache.http.entity.InputStreamEntity;
-import org.apache.http.impl.client.CloseableHttpClient;
-import org.apache.http.impl.client.HttpClients;
-import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
-import org.apache.http.util.EntityUtils;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.PipedInputStream;
-import java.io.PipedOutputStream;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.text.ParseException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.FutureTask;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-import static java.net.HttpURLConnection.HTTP_CREATED;
-import static java.net.HttpURLConnection.HTTP_OK;
-import static org.apache.hadoop.ozone.OzoneConsts.OZONE_OM_SERVICE_LIST_HTTP_ENDPOINT;
- * Ozone Client REST protocol implementation. It uses REST protocol to
- * connect to Ozone Handler that executes client calls.
-public class RestClient implements ClientProtocol {
- private static final String PATH_SEPARATOR = "/";
- private static final Logger LOG = LoggerFactory.getLogger(RestClient.class);
- private final Configuration conf;
- private final URI ozoneRestUri;
- private final CloseableHttpClient httpClient;
- private final UserGroupInformation ugi;
- // private final OzoneAcl.OzoneACLRights userRights;
- * Creates RestClient instance with the given configuration.
- * @param conf Configuration
- public RestClient(Configuration conf)
- try {
- Preconditions.checkNotNull(conf);
- this.conf = conf;
- this.ugi = UserGroupInformation.getCurrentUser();
- long socketTimeout = conf.getTimeDuration(
- OzoneConfigKeys.OZONE_CLIENT_SOCKET_TIMEOUT,
- OzoneConfigKeys.OZONE_CLIENT_SOCKET_TIMEOUT_DEFAULT,
- TimeUnit.MILLISECONDS);
- long connectionTimeout = conf.getTimeDuration(
- OzoneConfigKeys.OZONE_CLIENT_CONNECTION_TIMEOUT,
- OzoneConfigKeys.OZONE_CLIENT_CONNECTION_TIMEOUT_DEFAULT,
- int maxConnection = conf.getInt(
- OzoneConfigKeys.OZONE_REST_CLIENT_HTTP_CONNECTION_MAX,
- OzoneConfigKeys.OZONE_REST_CLIENT_HTTP_CONNECTION_DEFAULT);
- int maxConnectionPerRoute = conf.getInt(
- OzoneConfigKeys.OZONE_REST_CLIENT_HTTP_CONNECTION_PER_ROUTE_MAX,
- OzoneConfigKeys
- .OZONE_REST_CLIENT_HTTP_CONNECTION_PER_ROUTE_MAX_DEFAULT
- );
- /*
- To make RestClient Thread safe, creating the HttpClient with
- ThreadSafeClientConnManager.
- PoolingHttpClientConnectionManager connManager =
- new PoolingHttpClientConnectionManager();
- connManager.setMaxTotal(maxConnection);
- connManager.setDefaultMaxPerRoute(maxConnectionPerRoute);
- this.httpClient = HttpClients.custom()
- .setConnectionManager(connManager)
- .setDefaultRequestConfig(
- RequestConfig.custom()
- .setSocketTimeout(Math.toIntExact(socketTimeout))
- .setConnectTimeout(Math.toIntExact(connectionTimeout))
- .build())
- .build();
-// this.userRights = conf.getEnum(OMConfigKeys.OZONE_OM_USER_RIGHTS,
-// OMConfigKeys.OZONE_OM_USER_RIGHTS_DEFAULT);
- // TODO: Add new configuration parameter to configure RestServerSelector.
- RestServerSelector defaultSelector = new DefaultRestServerSelector();
- InetSocketAddress restServer = getOzoneRestServerAddress(defaultSelector);
- URIBuilder uriBuilder = new URIBuilder()
- .setScheme("http")
- .setHost(restServer.getHostName())
- .setPort(restServer.getPort());
- this.ozoneRestUri = uriBuilder.build();
- } catch (URISyntaxException e) {
- throw new IOException(e);
- private InetSocketAddress getOzoneRestServerAddress(
- RestServerSelector selector) throws IOException {
- String httpAddress = conf.get(OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY);
- if (httpAddress == null) {
- throw new IllegalArgumentException(
- OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY + " must be defined. See" +
- " https://wiki.apache.org/hadoop/Ozone#Configuration for" +
- " details on configuring Ozone.");
- HttpGet httpGet = new HttpGet("http://" + httpAddress +
- OZONE_OM_SERVICE_LIST_HTTP_ENDPOINT);
- HttpEntity entity = executeHttpRequest(httpGet);
- String serviceListJson = EntityUtils.toString(entity);
- ObjectMapper objectMapper = new ObjectMapper();
- TypeReference<List<ServiceInfo>> serviceInfoReference =
- new TypeReference<List<ServiceInfo>>() {
- };
- List<ServiceInfo> services = objectMapper.readValue(
- serviceListJson, serviceInfoReference);
- List<ServiceInfo> dataNodeInfos = services.stream().filter(
- a -> a.getNodeType().equals(HddsProtos.NodeType.DATANODE))
- .collect(Collectors.toList());
- ServiceInfo restServer = selector.getRestServer(dataNodeInfos);
- return NetUtils.createSocketAddr(
- NetUtils.normalizeHostName(restServer.getHostname()) + ":"
- + restServer.getPort(ServicePort.Type.HTTP));
- } finally {
- EntityUtils.consume(entity);
- public void createVolume(String volumeName) throws IOException {
- createVolume(volumeName, VolumeArgs.newBuilder().build());
- public void createVolume(String volumeName, VolumeArgs volArgs)
- HddsClientUtils.verifyResourceName(volumeName);
- Preconditions.checkNotNull(volArgs);
- URIBuilder builder = new URIBuilder(ozoneRestUri);
- String owner = volArgs.getOwner() == null ?
- ugi.getUserName() : volArgs.getOwner();
- //TODO: support for ACLs has to be done in OzoneHandler (rest server)
- List<OzoneAcl> listOfAcls = new ArrayList<>();
- //User ACL
- listOfAcls.add(new OzoneAcl(OzoneAcl.OzoneACLType.USER,
- owner, userRights));
- //ACLs from VolumeArgs
- if(volArgs.getAcls() != null) {
- listOfAcls.addAll(volArgs.getAcls());
- builder.setPath(PATH_SEPARATOR + volumeName);
- String quota = volArgs.getQuota();
- if(quota != null) {
- builder.setParameter(Header.OZONE_QUOTA_QUERY_TAG, quota);
- HttpPost httpPost = new HttpPost(builder.build());
- addOzoneHeaders(httpPost);
- //use admin from VolumeArgs, if it's present
- if(volArgs.getAdmin() != null) {
- httpPost.removeHeaders(HttpHeaders.AUTHORIZATION);
- httpPost.addHeader(HttpHeaders.AUTHORIZATION,
- Header.OZONE_SIMPLE_AUTHENTICATION_SCHEME + " " +
- volArgs.getAdmin());
- httpPost.addHeader(Header.OZONE_USER, owner);
- LOG.info("Creating Volume: {}, with {} as owner and quota set to {}.",
- volumeName, owner, quota == null ? "default" : quota);
- EntityUtils.consume(executeHttpRequest(httpPost));
- public void setVolumeOwner(String volumeName, String owner)
- Preconditions.checkNotNull(owner);
- HttpPut httpPut = new HttpPut(builder.build());
- addOzoneHeaders(httpPut);
- httpPut.addHeader(Header.OZONE_USER, owner);
- EntityUtils.consume(executeHttpRequest(httpPut));
- public void setVolumeQuota(String volumeName, OzoneQuota quota)
- Preconditions.checkNotNull(quota);
- String quotaString = quota.toString();
- builder.setParameter(Header.OZONE_QUOTA_QUERY_TAG, quotaString);
- public OzoneVolume getVolumeDetails(String volumeName)
- builder.setParameter(Header.OZONE_INFO_QUERY_TAG,
- Header.OZONE_INFO_QUERY_VOLUME);
- HttpGet httpGet = new HttpGet(builder.build());
- addOzoneHeaders(httpGet);
- HttpEntity response = executeHttpRequest(httpGet);
- VolumeInfo volInfo =
- VolumeInfo.parse(EntityUtils.toString(response));
- //TODO: OzoneHandler in datanode has to be modified to send ACLs
- OzoneVolume volume = new OzoneVolume(conf,
- this,
- volInfo.getVolumeName(),
- volInfo.getCreatedBy(),
- volInfo.getOwner().getName(),
- volInfo.getQuota().sizeInBytes(),
- HddsClientUtils.formatDateTime(volInfo.getCreatedOn()),
- null);
- EntityUtils.consume(response);
- return volume;
- } catch (URISyntaxException | ParseException e) {
- public boolean checkVolumeAccess(String volumeName, OzoneAcl acl)
- throw new UnsupportedOperationException("Not yet implemented.");
- public void deleteVolume(String volumeName) throws IOException {
- HttpDelete httpDelete = new HttpDelete(builder.build());
- addOzoneHeaders(httpDelete);
- EntityUtils.consume(executeHttpRequest(httpDelete));
- public List<OzoneVolume> listVolumes(String volumePrefix, String prevKey,
- int maxListResult)
- return listVolumes(null, volumePrefix, prevKey, maxListResult);
- public List<OzoneVolume> listVolumes(String user, String volumePrefix,
- String prevKey, int maxListResult)
- builder.setPath(PATH_SEPARATOR);
- builder.addParameter(Header.OZONE_INFO_QUERY_TAG,
- Header.OZONE_LIST_QUERY_SERVICE);
- builder.addParameter(Header.OZONE_LIST_QUERY_MAXKEYS,
- String.valueOf(maxListResult));
- addQueryParamter(Header.OZONE_LIST_QUERY_PREFIX, volumePrefix, builder);
- addQueryParamter(Header.OZONE_LIST_QUERY_PREVKEY, prevKey, builder);
- if (!Strings.isNullOrEmpty(user)) {
- httpGet.addHeader(Header.OZONE_USER, user);
- ListVolumes volumeList =
- ListVolumes.parse(EntityUtils.toString(response));
- return volumeList.getVolumes().stream().map(volInfo -> {
- long creationTime = 0;
- creationTime = HddsClientUtils.formatDateTime(volInfo.getCreatedOn());
- } catch (ParseException e) {
- LOG.warn("Parse exception in getting creation time for volume", e);
- return new OzoneVolume(conf, this, volInfo.getVolumeName(),
- volInfo.getCreatedBy(), volInfo.getOwner().getName(),
- volInfo.getQuota().sizeInBytes(), creationTime, null);
- }).collect(Collectors.toList());
- public void createBucket(String volumeName, String bucketName)
- createBucket(volumeName, bucketName, BucketArgs.newBuilder().build());
- public void createBucket(
- String volumeName, String bucketName, BucketArgs bucketArgs)
- HddsClientUtils.verifyResourceName(volumeName, bucketName);
- Preconditions.checkNotNull(bucketArgs);
- OzoneConsts.Versioning versioning = OzoneConsts.Versioning.DISABLED;
- if(bucketArgs.getVersioning() != null &&
- bucketArgs.getVersioning()) {
- versioning = OzoneConsts.Versioning.ENABLED;
- StorageType storageType = bucketArgs.getStorageType() == null ?
- StorageType.DEFAULT : bucketArgs.getStorageType();
- builder.setPath(PATH_SEPARATOR + volumeName +
- PATH_SEPARATOR + bucketName);
- //ACLs from BucketArgs
- if(bucketArgs.getAcls() != null) {
- for (OzoneAcl acl : bucketArgs.getAcls()) {
- httpPost.addHeader(
- Header.OZONE_ACLS, Header.OZONE_ACL_ADD + " " + acl.toString());
- httpPost.addHeader(Header.OZONE_STORAGE_TYPE, storageType.toString());
- httpPost.addHeader(Header.OZONE_BUCKET_VERSIONING,
- versioning.toString());
- LOG.info("Creating Bucket: {}/{}, with Versioning {} and Storage Type" +
- " set to {}", volumeName, bucketName, versioning,
- storageType);
- public void setBucketVersioning(
- String volumeName, String bucketName, Boolean versioning)
- Preconditions.checkNotNull(versioning);
- httpPut.addHeader(Header.OZONE_BUCKET_VERSIONING,
- getBucketVersioning(versioning).toString());
- public void setBucketStorageType(
- String volumeName, String bucketName, StorageType storageType)
- Preconditions.checkNotNull(storageType);
- httpPut.addHeader(Header.OZONE_STORAGE_TYPE, storageType.toString());
- public void deleteBucket(String volumeName, String bucketName)
- public void checkBucketAccess(String volumeName, String bucketName)
- public OzoneBucket getBucketDetails(String volumeName, String bucketName)
- Header.OZONE_INFO_QUERY_BUCKET);
- BucketInfo.parse(EntityUtils.toString(response));
- OzoneBucket bucket = new OzoneBucket(conf,
- bucketInfo.getVolumeName(),
- bucketInfo.getBucketName(),
- bucketInfo.getStorageType(),
- getBucketVersioningFlag(bucketInfo.getVersioning()),
- HddsClientUtils.formatDateTime(bucketInfo.getCreatedOn()),
- new HashMap<>());
- return bucket;
- public List<OzoneBucket> listBuckets(String volumeName, String bucketPrefix,
- String prevBucket, int maxListResult)
- addQueryParamter(Header.OZONE_LIST_QUERY_PREFIX, bucketPrefix, builder);
- addQueryParamter(Header.OZONE_LIST_QUERY_PREVKEY, prevBucket, builder);
- ListBuckets bucketList =
- ListBuckets.parse(EntityUtils.toString(response));
- return bucketList.getBuckets().stream().map(bucketInfo -> {
- creationTime =
- HddsClientUtils.formatDateTime(bucketInfo.getCreatedOn());
- return new OzoneBucket(conf, this, volumeName,
- bucketInfo.getBucketName(), bucketInfo.getStorageType(),
- getBucketVersioningFlag(bucketInfo.getVersioning()), creationTime,
- new HashMap<>(), bucketInfo.getEncryptionKeyName());
- * Writes a key in an existing bucket.
- * @param volumeName Name of the Volume
- * @param bucketName Name of the Bucket
- * @param keyName Name of the Key
- * @param size Size of the data
- * @param type
- * @param factor @return {@link OzoneOutputStream}
- public OzoneOutputStream createKey(
- String volumeName, String bucketName, String keyName, long size,
- ReplicationType type, ReplicationFactor factor,
- Map<String, String> metadata)
- // TODO: Once ReplicationType and ReplicationFactor are supported in
- // OzoneHandler (in Datanode), set them in header.
- HddsClientUtils.checkNotNull(keyName, type, factor);
- PATH_SEPARATOR + bucketName +
- PATH_SEPARATOR + keyName);
- HttpPut putRequest = new HttpPut(builder.build());
- addOzoneHeaders(putRequest);
- PipedInputStream in = new PipedInputStream();
- OutputStream out = new PipedOutputStream(in);
- putRequest.setEntity(new InputStreamEntity(in, size));
- FutureTask<HttpEntity> futureTask =
- new FutureTask<>(() -> executeHttpRequest(putRequest));
- new Thread(futureTask).start();
- OzoneOutputStream outputStream = new OzoneOutputStream(
- new OutputStream() {
- public void write(int b) throws IOException {
- out.write(b);
- public void close() throws IOException {
- out.close();
- EntityUtils.consume(futureTask.get());
- } catch (ExecutionException | InterruptedException e) {
- });
- return outputStream;
- * Get a valid Delegation Token. Not supported for RestClient.
- * @param renewer the designated renewer for the token
- * @return Token<OzoneDelegationTokenSelector>
- public Token<OzoneTokenIdentifier> getDelegationToken(Text renewer)
- throw new IOException("Method not supported");
- * Renew an existing delegation token. Not supported for RestClient.
- * @param token delegation token obtained earlier
- * @return the new expiration time
- public long renewDelegationToken(Token<OzoneTokenIdentifier> token)
- * Cancel an existing delegation token. Not supported for RestClient.
- * @param token delegation token
- public void cancelDelegationToken(Token<OzoneTokenIdentifier> token)
- public S3SecretValue getS3Secret(String kerberosID) throws IOException {
- throw new UnsupportedOperationException("Ozone REST protocol does not " +
- "support this operation.");
- public OMFailoverProxyProvider getOMProxyProvider() {
- return null;
- public KeyProvider getKeyProvider() throws IOException {
- // TODO: fix me to support kms instances for difference OMs
- return OzoneKMSUtil.getKeyProvider(conf, getKeyProviderUri());
- public URI getKeyProviderUri() throws IOException {
- return OzoneKMSUtil.getKeyProviderUri(ugi, null, null, conf);
- public OzoneInputStream getKey(
- String volumeName, String bucketName, String keyName)
- Preconditions.checkNotNull(keyName);
- HttpGet getRequest = new HttpGet(builder.build());
- addOzoneHeaders(getRequest);
- HttpEntity entity = executeHttpRequest(getRequest);
- FutureTask<Void> futureTask =
- new FutureTask<>(() -> {
- entity.writeTo(out);
- OzoneInputStream inputStream = new OzoneInputStream(
- new InputStream() {
- public int read() throws IOException {
- return in.read();
- in.close();
- return inputStream;
- public void deleteKey(String volumeName, String bucketName, String keyName)
- PATH_SEPARATOR + bucketName + PATH_SEPARATOR + keyName);
- public void renameKey(String volumeName, String bucketName,
- String fromKeyName, String toKeyName) throws IOException {
- HddsClientUtils.checkNotNull(fromKeyName, toKeyName);
- builder.setPath(PATH_SEPARATOR + volumeName + PATH_SEPARATOR + bucketName
- + PATH_SEPARATOR + fromKeyName);
- builder.addParameter(Header.OZONE_RENAME_TO_KEY_PARAM_NAME, toKeyName);
- public List<OzoneKey> listKeys(String volumeName, String bucketName,
- String keyPrefix, String prevKey,
- builder
- .setPath(PATH_SEPARATOR + volumeName + PATH_SEPARATOR + bucketName);
- Header.OZONE_INFO_QUERY_KEY);
- addQueryParamter(Header.OZONE_LIST_QUERY_PREFIX, keyPrefix, builder);
- ListKeys keyList = ListKeys.parse(EntityUtils.toString(response));
- return keyList.getKeyList().stream().map(keyInfo -> {
- long creationTime = 0, modificationTime = 0;
- creationTime = HddsClientUtils.formatDateTime(keyInfo.getCreatedOn());
- modificationTime =
- HddsClientUtils.formatDateTime(keyInfo.getModifiedOn());
- return new OzoneKey(volumeName, bucketName, keyInfo.getKeyName(),
- keyInfo.getSize(), creationTime, modificationTime,
- ReplicationType.valueOf(keyInfo.getType().toString()));
- public OzoneKeyDetails getKeyDetails(
- Header.OZONE_INFO_QUERY_KEY_DETAIL);
- KeyInfoDetails keyInfo =
- KeyInfoDetails.parse(EntityUtils.toString(response));
- List<OzoneKeyLocation> ozoneKeyLocations = new ArrayList<>();
- keyInfo.getKeyLocations().forEach((a) -> ozoneKeyLocations.add(
- new OzoneKeyLocation(a.getContainerID(), a.getLocalID(),
- a.getLength(), a.getOffset())));
- OzoneKeyDetails key = new OzoneKeyDetails(volumeName,
- bucketName,
- keyInfo.getKeyName(),
- keyInfo.getSize(),
- HddsClientUtils.formatDateTime(keyInfo.getCreatedOn()),
- HddsClientUtils.formatDateTime(keyInfo.getModifiedOn()),
- ozoneKeyLocations, ReplicationType.valueOf(
- keyInfo.getType().toString()),
- new HashMap<>(), keyInfo.getFileEncryptionInfo());
- return key;
- public void createS3Bucket(String userName, String s3BucketName)
- public void deleteS3Bucket(String s3BucketName)
- public String getOzoneBucketMapping(String s3BucketName) throws IOException {
- public String getOzoneVolumeName(String s3BucketName) throws IOException {
- public String getOzoneBucketName(String s3BucketName) throws IOException {
- public List<OzoneBucket> listS3Buckets(String userName, String bucketPrefix,
- * Adds Ozone headers to http request.
- * @param httpRequest Http Request
- private void addOzoneHeaders(HttpUriRequest httpRequest) {
- httpRequest.addHeader(HttpHeaders.AUTHORIZATION,
- ugi.getUserName());
- httpRequest.addHeader(HttpHeaders.DATE,
- HddsClientUtils.formatDateTime(Time.monotonicNow()));
- httpRequest.addHeader(Header.OZONE_VERSION_HEADER,
- Header.OZONE_V1_VERSION_HEADER);
- * Sends the http request to server and returns the response HttpEntity.
- * It's responsibility of the caller to consume and close response HttpEntity
- * by calling {@code EntityUtils.consume}
- * @param httpUriRequest http request
- private HttpEntity executeHttpRequest(HttpUriRequest httpUriRequest)
- HttpResponse response = httpClient.execute(httpUriRequest);
- int errorCode = response.getStatusLine().getStatusCode();
- HttpEntity entity = response.getEntity();
- if ((errorCode == HTTP_OK) || (errorCode == HTTP_CREATED)) {
- return entity;
- if (entity != null) {
- throw new IOException(
- OzoneException.parse(EntityUtils.toString(entity)));
- } else {
- throw new IOException("Unexpected null in http payload," +
- " while processing request");
- * Converts OzoneConts.Versioning to boolean.
- * @param version
- * @return corresponding boolean value
- private Boolean getBucketVersioningFlag(
- OzoneConsts.Versioning version) {
- if(version != null) {
- switch(version) {
- case ENABLED:
- return true;
- case NOT_DEFINED:
- case DISABLED:
- default:
- return false;
- * Converts Bucket versioning flag into OzoneConts.Versioning.
- * @param flag versioning flag
- * @return corresponding OzoneConts.Versionin
- private OzoneConsts.Versioning getBucketVersioning(Boolean flag) {
- if(flag != null) {
- if(flag) {
- return OzoneConsts.Versioning.ENABLED;
- return OzoneConsts.Versioning.DISABLED;
- return OzoneConsts.Versioning.NOT_DEFINED;
- httpClient.close();
- private void addQueryParamter(String param, String value,
- URIBuilder builder) {
- if (!Strings.isNullOrEmpty(value)) {
- builder.addParameter(param, value);
- public OmMultipartInfo initiateMultipartUpload(String volumeName,
- String bucketName,
- String keyName,
- ReplicationType type,
- ReplicationFactor factor)
- public OzoneOutputStream createMultipartKey(String volumeName,
- long size,
- int partNumber,
- String uploadID)
- public OmMultipartUploadCompleteInfo completeMultipartUpload(
- String volumeName, String bucketName, String keyName, String uploadID,
- Map<Integer, String> partsMap) throws IOException {
- public void abortMultipartUpload(String volumeName,
- String bucketName, String keyName, String uploadID) throws IOException {
- public OzoneMultipartUploadPartListParts listParts(String volumeName,
- String bucketName, String keyName, String uploadID, int partNumberMarker,
- int maxParts) throws IOException {
- * Get CanonicalServiceName for ozone delegation token.
- * @return Canonical Service Name of ozone delegation token.
- public String getCanonicalServiceName(){
- public OzoneFileStatus getOzoneFileStatus(String volumeName,
- String bucketName, String keyName) throws IOException {
- public void createDirectory(String volumeName, String bucketName,
- String keyName) {
- throw new UnsupportedOperationException(
- "Ozone REST protocol does not " + "support this operation.");
- public OzoneInputStream readFile(String volumeName, String bucketName,
- public OzoneOutputStream createFile(String volumeName, String bucketName,
- String keyName, long size, ReplicationType type, ReplicationFactor factor,
- boolean overWrite, boolean recursive) {
- public List<OzoneFileStatus> listStatus(String volumeName, String bucketName,
- String keyName, boolean recursive, String startKey, long numEntries)
- * Add acl for Ozone object. Return true if acl is added successfully else
- * false.
- * @param obj Ozone object for which acl should be added.
- * @param acl ozone acl top be added.
- * @throws IOException if there is error.
- public boolean addAcl(OzoneObj obj, OzoneAcl acl) throws IOException {
- throw new UnsupportedOperationException("Ozone REST protocol does not" +
- " support this operation.");
- * Remove acl for Ozone object. Return true if acl is removed successfully
- * else false.
- * @param obj Ozone object.
- * @param acl Ozone acl to be removed.
- public boolean removeAcl(OzoneObj obj, OzoneAcl acl) throws IOException {
- * Acls to be set for given Ozone object. This operations reset ACL for given
- * object to list of ACLs provided in argument.
- * @param acls List of acls.
- public boolean setAcl(OzoneObj obj, List<OzoneAcl> acls) throws IOException {
- * Returns list of ACLs for given Ozone object.
- public List<OzoneAcl> getAcl(OzoneObj obj) throws IOException {
@@ -1,40 +0,0 @@
- * The implementor of this interface should select the REST server which will
- * be used by the client to connect to Ozone Cluster, given list of
- * REST Servers/DataNodes (DataNodes are the ones which hosts REST Service).
-public interface RestServerSelector {
- * Returns the REST Service which will be used by the client for connection.
- * @param restServices list of available REST servers
- * @return ServiceInfo
- ServiceInfo getRestServer(List<ServiceInfo> restServices);
@@ -1,22 +0,0 @@
-package org.apache.hadoop.ozone.client.rest.exceptions;
- * This package contains ozone rest client libraries.
-@InterfaceAudience.Private
-package org.apache.hadoop.ozone.client.rest.headers;
-import org.apache.hadoop.classification.InterfaceAudience;
@@ -1,23 +0,0 @@
- * This package contains Ozone rest client library classes.
@@ -1,267 +0,0 @@
-import com.fasterxml.jackson.annotation.JsonAutoDetect;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.databind.ObjectReader;
- * Ozone Layer.
-public class OzoneException extends Exception {
- private static final ObjectReader READER =
- new ObjectMapper().readerFor(OzoneException.class);
- private static final ObjectMapper MAPPER;
- static {
- MAPPER = new ObjectMapper();
- MAPPER.setVisibility(
- MAPPER.getSerializationConfig().getDefaultVisibilityChecker()
- .withCreatorVisibility(JsonAutoDetect.Visibility.NONE)
- .withFieldVisibility(JsonAutoDetect.Visibility.NONE)
- .withGetterVisibility(JsonAutoDetect.Visibility.NONE)
- .withIsGetterVisibility(JsonAutoDetect.Visibility.NONE)
- .withSetterVisibility(JsonAutoDetect.Visibility.NONE));
- @JsonProperty("httpCode")
- private long httpCode;
- @JsonProperty("shortMessage")
- private String shortMessage;
- @JsonProperty("resource")
- private String resource;
- @JsonProperty("message")
- private String message;
- @JsonProperty("requestID")
- private String requestId;
- @JsonProperty("hostName")
- private String hostID;
- * Constructs a new exception with {@code null} as its detail message. The
- * cause is not initialized, and may subsequently be initialized by a call
- * to {@link #initCause}.
- * This constructor is needed by Json Serializer.
- public OzoneException() {
- * Constructor that allows a shortMessage and exception.
- * @param httpCode Error Code
- * @param ex Exception
- public OzoneException(long httpCode, String shortMessage, Exception ex) {
- super(ex);
- this.message = ex.getMessage();
- this.shortMessage = shortMessage;
- this.httpCode = httpCode;
- * Constructor that allows a shortMessage.
- public OzoneException(long httpCode, String shortMessage) {
- * Constructor that allows a shortMessage and long message.
- public OzoneException(long httpCode, String shortMessage, String message) {
- this.message = message;
- * Constructor that allows a shortMessage, a long message and an exception.
- * @param httpCode Error code
- * @param shortMessage Short message
- * @param message Long error message
- public OzoneException(long httpCode, String shortMessage,
- String message, Exception ex) {
- * Returns the Resource that was involved in the stackTraceString.
- * @return String
- public String getResource() {
- return resource;
- * Sets Resource.
- * @param resourceName - Name of the Resource
- public void setResource(String resourceName) {
- this.resource = resourceName;
- * Gets a detailed message for the error.
- public String getMessage() {
- return message;
- * Sets the error message.
- * @param longMessage - Long message
- public void setMessage(String longMessage) {
- this.message = longMessage;
- * Returns request Id.
- public String getRequestId() {
- return requestId;
- * Sets request ID.
- * @param ozoneRequestId Request ID generated by the Server
- public void setRequestId(String ozoneRequestId) {
- this.requestId = ozoneRequestId;
- * Returns short error string.
- public String getShortMessage() {
- return shortMessage;
- * Sets short error string.
- * @param shortError Short Error Code
- public void setShortMessage(String shortError) {
- this.shortMessage = shortError;
- * Returns hostID.
- public String getHostID() {
- return hostID;
- * Sets host ID.
- * @param hostName host Name
- public void setHostID(String hostName) {
- this.hostID = hostName;
- * Returns http error code.
- * @return long
- public long getHttpCode() {
- return httpCode;
- * Sets http status.
- * @param httpStatus http error code.
- public void setHttpCode(long httpStatus) {
- this.httpCode = httpStatus;
- * Returns a Json String.
- * @return JSON representation of the Error
- public String toJsonString() {
- return MAPPER.writeValueAsString(this);
- } catch (IOException ex) {
- // TODO : Log this error on server side.
- // TODO : Replace this with a JSON Object -- That represents this error.
- return "500 Internal Server Error";
- * Parses an Exception record.
- * @param jsonString - Exception in Json format.
- * @return OzoneException Object
- public static OzoneException parse(String jsonString) throws IOException {
- return READER.readValue(jsonString);
@@ -1,74 +0,0 @@
- * OZONE specific HTTP headers.
-public final class Header {
- public static final String OZONE_QUOTA_BYTES = "BYTES";
- public static final String OZONE_QUOTA_MB = "MB";
- public static final String OZONE_QUOTA_GB = "GB";
- public static final String OZONE_QUOTA_TB = "TB";
- public static final String OZONE_QUOTA_REMOVE = "remove";
- public static final String OZONE_QUOTA_UNDEFINED = "undefined";
- public static final String OZONE_EMPTY_STRING="";
- public static final String OZONE_DEFAULT_LIST_SIZE = "1000";
- public static final String OZONE_USER = "x-ozone-user";
- public static final String OZONE_SIMPLE_AUTHENTICATION_SCHEME = "OZONE";
- public static final String OZONE_VERSION_HEADER = "x-ozone-version";
- public static final String OZONE_V1_VERSION_HEADER ="v1";
- public static final String OZONE_LIST_QUERY_SERVICE = "service";
- public static final String OZONE_INFO_QUERY_VOLUME = "volume";
- public static final String OZONE_INFO_QUERY_BUCKET = "bucket";
- public static final String OZONE_INFO_QUERY_KEY = "key";
- public static final String OZONE_INFO_QUERY_KEY_DETAIL = "key-detail";
- public static final String OZONE_REQUEST_ID = "x-ozone-request-id";
- public static final String OZONE_SERVER_NAME = "x-ozone-server-name";
- public static final String OZONE_STORAGE_TYPE = "x-ozone-storage-type";
- public static final String OZONE_BUCKET_VERSIONING =
- "x-ozone-bucket-versioning";
- public static final String OZONE_ACLS = "x-ozone-acls";
- public static final String OZONE_ACL_ADD = "ADD";
- public static final String OZONE_ACL_REMOVE = "REMOVE";
- public static final String OZONE_INFO_QUERY_TAG ="info";
- public static final String OZONE_QUOTA_QUERY_TAG ="quota";
- public static final String CONTENT_MD5 = "Content-MD5";
- public static final String OZONE_LIST_QUERY_PREFIX="prefix";
- public static final String OZONE_LIST_QUERY_MAXKEYS="max-keys";
- public static final String OZONE_LIST_QUERY_PREVKEY="prev-key";
- public static final String OZONE_LIST_QUERY_ROOTSCAN="root-scan";
- public static final String OZONE_RENAME_TO_KEY_PARAM_NAME = "toKey";
- private Header() {
- // Never constructed.
- * Ozone HTTP Header utility.
- * Ozone REST interface.
@@ -1,248 +0,0 @@
-package org.apache.hadoop.ozone.client.rest.response;
-import java.util.LinkedList;
- * BucketInfo class is used used for parsing json response
- * when BucketInfo Call is made.
-public class BucketInfo implements Comparable<BucketInfo> {
- new ObjectMapper().readerFor(BucketInfo.class);
- private String volumeName;
- private String bucketName;
- private String createdOn;
- private List<OzoneAcl> acls;
- private OzoneConsts.Versioning versioning;
- private StorageType storageType;
- private String bekName;
- * Constructor for BucketInfo.
- * @param volumeName
- * @param bucketName
- public BucketInfo(String volumeName, String bucketName) {
- this.volumeName = volumeName;
- this.bucketName = bucketName;
- * Default constructor for BucketInfo.
- public BucketInfo() {
- acls = new LinkedList<>();
- * Parse a JSON string into BucketInfo Object.
- * @param jsonString Json String
- * @return BucketInfo
- public static BucketInfo parse(String jsonString) throws IOException {
- * Returns a List of ACLs set on the Bucket.
- * @return List of Acl
- public List<OzoneAcl> getAcls() {
- return acls;
- * Sets ACls.
- * @param acls Acl list
- public void setAcls(List<OzoneAcl> acls) {
- this.acls = acls;
- * Returns Storage Type info.
- * @return Storage Type of the bucket
- public StorageType getStorageType() {
- return storageType;
- * Sets the Storage Type.
- * @param storageType Storage Type
- public void setStorageType(StorageType storageType) {
- this.storageType = storageType;
- * Returns versioning.
- * @return versioning Enum
- public OzoneConsts.Versioning getVersioning() {
- return versioning;
- * Sets Versioning.
- * @param versioning
- public void setVersioning(OzoneConsts.Versioning versioning) {
- this.versioning = versioning;
- * Gets bucket Name.
- public String getBucketName() {
- return bucketName;
- * Sets bucket Name.
- * @param bucketName Name of the bucket
- public void setBucketName(String bucketName) {
- * Sets creation time of the bucket.
- * @param creationTime Date String
- public void setCreatedOn(String creationTime) {
- this.createdOn = creationTime;
- * Returns creation time.
- * @return creation time of bucket.
- public String getCreatedOn() {
- return createdOn;
- * Returns Volume Name.
- * @return String volume name
- public String getVolumeName() {
- return volumeName;
- * Sets the Volume Name of bucket.
- * @param volumeName volumeName
- public void setVolumeName(String volumeName) {
- * Return bucket encryption key name.
- * @return bucket encryption key name
- public String getEncryptionKeyName() {
- return bekName;
- * Sets the bucket encryption key name.
- * @param name bucket encryption key name
- public void setEncryptionKeyName(String name) {
- this.bekName = name;
- * Compares this object with the specified object for order. Returns a
- * negative integer, zero, or a positive integer as this object is less
- * than, equal to, or greater than the specified object.
- * Please note : BucketInfo compare functions are used only within the
- * context of a volume, hence volume name is purposefully ignored in
- * compareTo, equal and hashcode functions of this class.
- public int compareTo(BucketInfo o) {
- Preconditions.checkState(o.getVolumeName().equals(this.getVolumeName()));
- return this.bucketName.compareTo(o.getBucketName());
- * Checks if two bucketInfo's are equal.
- * @param o Object BucketInfo
- * @return True or False
- public boolean equals(Object o) {
- if (this == o) {
- if (!(o instanceof BucketInfo)) {
- BucketInfo that = (BucketInfo) o;
- Preconditions.checkState(that.getVolumeName().equals(this.getVolumeName()));
- return bucketName.equals(that.bucketName);
- * Hash Code for this object.
- * @return int
- public int hashCode() {
- return bucketName.hashCode();
@@ -1,236 +0,0 @@
-import org.apache.commons.lang3.builder.EqualsBuilder;
-import org.apache.commons.lang3.builder.HashCodeBuilder;
- * KeyInfo class is used used for parsing json response
- * when KeyInfo Call is made.
-public class KeyInfo implements Comparable<KeyInfo> {
- new ObjectMapper().readerFor(KeyInfo.class);
- private long version;
- private String md5hash;
- private String modifiedOn;
- private long size;
- private String keyName;
- private ReplicationType type;
- * Return replication type of the key.
- * @return replication type
- public ReplicationType getType() {
- return type;
- * Set replication type of the key.
- * @param replicationType
- public void setType(ReplicationType replicationType) {
- this.type = replicationType;
- * When this key was created.
- * @return Date String
- * When this key was modified.
- public String getModifiedOn() {
- return modifiedOn;
- * @param createdOn Date String
- public void setCreatedOn(String createdOn) {
- this.createdOn = createdOn;
- * @param modifiedOn Date String
- public void setModifiedOn(String modifiedOn) {
- this.modifiedOn = modifiedOn;
- * Gets the Key name of this object.
- public String getKeyName() {
- return keyName;
- * Sets the Key name of this object.
- * @param keyName String
- public void setKeyName(String keyName) {
- this.keyName = keyName;
- * Returns the MD5 Hash for the data of this key.
- * @return String MD5
- public String getMd5hash() {
- return md5hash;
- * Sets the MD5 value of this key.
- * @param md5hash Md5 of this file
- public void setMd5hash(String md5hash) {
- this.md5hash = md5hash;
- * Number of bytes stored in the data part of this key.
- * @return long size of the data file
- public long getSize() {
- return size;
- * Sets the size of the data part of this key.
- * @param size Size in long
- public void setSize(long size) {
- this.size = size;
- * Version of this key.
- * @return returns the version of this key.
- public long getVersion() {
- return version;
- * Sets the version of this key.
- * @param version - Version String
- public void setVersion(long version) {
- this.version = version;
- * @param o the object to be compared.
- * @return a negative integer, zero, or a positive integer as this object
- * is less than, equal to, or greater than the specified object.
- * @throws NullPointerException if the specified object is null
- * @throws ClassCastException if the specified object's type prevents it
- * from being compared to this object.
- public int compareTo(KeyInfo o) {
- if (this.keyName.compareTo(o.getKeyName()) != 0) {
- return this.keyName.compareTo(o.getKeyName());
- if (this.getVersion() == o.getVersion()) {
- return 0;
- if (this.getVersion() < o.getVersion()) {
- return -1;
- return 1;
- if (o == null || getClass() != o.getClass()) {
- KeyInfo keyInfo = (KeyInfo) o;
- return new EqualsBuilder()
- .append(version, keyInfo.version)
- .append(keyName, keyInfo.keyName)
- .isEquals();
- return new HashCodeBuilder(17, 37)
- .append(version)
- .append(keyName)
- .toHashCode();
- * Parse a string to return KeyInfo Object.
- * @return keyInfo
- public static KeyInfo parse(String jsonString) throws IOException {
@@ -1,118 +0,0 @@
-import org.apache.hadoop.fs.FileEncryptionInfo;
- * KeyInfoDetails class is used for parsing json response
- * when KeyInfoDetails Call is made.
-public class KeyInfoDetails extends KeyInfo {
- new ObjectMapper().readerFor(KeyInfoDetails.class);
- * a list of Map which maps localID to ContainerID
- * to specify replica locations.
- private List<KeyLocation> keyLocations;
- private FileEncryptionInfo feInfo;
- * Constructor needed for json serialization.
- public KeyInfoDetails() {
- * Set details of key location.
- * @param locations - details of key location
- public void setKeyLocation(List<KeyLocation> locations) {
- this.keyLocations = locations;
- * Returns details of key location.
- * @return volumeName
- public List<KeyLocation> getKeyLocations() {
- return keyLocations;
- public void setFileEncryptionInfo(FileEncryptionInfo info) {
- this.feInfo = info;
- public FileEncryptionInfo getFileEncryptionInfo() {
- return feInfo;
- * Parse a string to return KeyInfoDetails Object.
- * @return KeyInfoDetails
- public static KeyInfoDetails parse(String jsonString) throws IOException {
- KeyInfoDetails that = (KeyInfoDetails) o;
- .append(getVersion(), that.getVersion())
- .append(getKeyName(), that.getKeyName())
- .append(keyLocations, that.keyLocations)
- return new HashCodeBuilder(21, 33)
- .append(getVersion())
- .append(getKeyName())
- .append(keyLocations)
@@ -1,89 +0,0 @@
- * KeyLocation class is used used for parsing json response
-public class KeyLocation {
- * Which container this key stored.
- private long containerID;
- * Which block this key stored inside a container.
- private long localID;
- * Data length of this key replica.
- private long length;
- * Offset of this key.
- private long offset;
- * Empty constructor for Json serialization.
- public KeyLocation() {
- * Constructs KeyLocation.
- public KeyLocation(long containerID, long localID,
- long length, long offset) {
- this.containerID = containerID;
- this.localID = localID;
- this.length = length;
- this.offset = offset;
- * Returns the containerID of this Key.
- public long getContainerID() {
- return containerID;
- * Returns the localID of this Key.
- public long getLocalID() {
- return localID;
- * Returns the length of this Key.
- public long getLength() {
- return length;
- * Returns the offset of this Key.
- public long getOffset() {
- return offset;
@@ -1,215 +0,0 @@
- * VolumeInfo Class is used for parsing json response
- * when VolumeInfo Call is made.
-public class VolumeInfo implements Comparable<VolumeInfo> {
- new ObjectMapper().readerFor(VolumeInfo.class);
- private VolumeOwner owner;
- private OzoneQuota quota;
- private String createdBy;
- * Constructor for VolumeInfo.
- * @param volumeName - Name of the Volume
- * @param createdOn _ Date String
- * @param createdBy - Person who created it
- public VolumeInfo(String volumeName, String createdOn,
- String createdBy) {
- this.createdBy = createdBy;
- public VolumeInfo() {
- * gets the volume name.
- * @return Volume Name
- * Sets the volume name.
- * @param volumeName Volume Name
- * Returns the name of the person who created this volume.
- * @return Name of Admin who created this
- public String getCreatedBy() {
- return createdBy;
- * Sets the user name of the person who created this volume.
- * @param createdBy UserName
- public void setCreatedBy(String createdBy) {
- * Gets the date on which this volume was created.
- * Sets the date string.
- * Returns the owner info.
- * @return OwnerInfo
- public VolumeOwner getOwner() {
- return owner;
- * Sets the owner.
- * @param owner OwnerInfo
- public void setOwner(VolumeOwner owner) {
- this.owner = owner;
- * Returns the quota information on a volume.
- * @return Quota
- public OzoneQuota getQuota() {
- return quota;
- * Sets the quota info.
- * @param quota Quota Info
- public void setQuota(OzoneQuota quota) {
- this.quota = quota;
- * Comparable Interface.
- * @param o VolumeInfo Object.
- * @return Result of comparison
- public int compareTo(VolumeInfo o) {
- return this.volumeName.compareTo(o.getVolumeName());
- * Returns VolumeInfo class from json string.
- * @param data Json String
- * @return VolumeInfo
- public static VolumeInfo parse(String data) throws IOException {
- return READER.readValue(data);
- * Indicates whether some other object is "equal to" this one.
- * @param obj the reference object with which to compare.
- * @return {@code true} if this object is the same as the obj
- * argument; {@code false} otherwise.
- public boolean equals(Object obj) {
- if (obj == null) {
- if (getClass() != obj.getClass()) {
- VolumeInfo otherInfo = (VolumeInfo) obj;
- return otherInfo.getVolumeName().equals(this.getVolumeName());
- * Returns a hash code value for the object. This method is
- * supported for the benefit of hash tables such as those provided by
- * HashMap.
- * @return a hash code value for this object.
- * @see Object#equals(Object)
- * @see System#identityHashCode
- return getVolumeName().hashCode();
@@ -1,61 +0,0 @@
-import com.fasterxml.jackson.annotation.JsonInclude;
- * Volume Owner represents the owner of a volume.
- * This is a class instead of a string since we might need to extend this class
- * to support other forms of authentication.
-public class VolumeOwner {
- @JsonInclude(JsonInclude.Include.NON_NULL)
- private String name;
- * Constructor for VolumeOwner.
- * @param name name of the User
- public VolumeOwner(String name) {
- this.name = name;
- * Constructs Volume Owner.
- public VolumeOwner() {
- name = null;
- * Returns the user name.
- * @return Name
- public String getName() {
- return name;
@@ -1,24 +0,0 @@
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- * This package contains class for ozone rest client library.
@@ -24,13 +24,11 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.ObjectReader;
import com.fasterxml.jackson.databind.ObjectWriter;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
.ServicePort;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
@@ -165,27 +163,6 @@ public final class ServiceInfo {
serviceInfo.getServicePortsList());
- * Returns a JSON string of this object.
- * @return String - json string
- public String toJsonString() throws IOException {
- return WRITER.writeValueAsString(this);
- * Parse a JSON string into ServiceInfo Object.
* Creates a new builder to build {@link ServiceInfo}.
* @return {@link ServiceInfo.Builder}
@@ -1,134 +0,0 @@
-package org.apache.hadoop.ozone.web.handlers;
- * BucketArgs packages all bucket related arguments to
- * file system calls.
-public class BucketArgs extends VolumeArgs {
- private final String bucketName;
- * Constructor for BucketArgs.
- * @param volumeName - volumeName
- * @param bucketName - bucket Name
- * @param userArgs - userArgs
- public BucketArgs(String volumeName, String bucketName, UserArgs userArgs) {
- super(volumeName, userArgs);
- this.versioning = OzoneConsts.Versioning.NOT_DEFINED;
- this.storageType = null;
- * @param volumeArgs - volume Args
- public BucketArgs(String bucketName, VolumeArgs volumeArgs) {
- super(volumeArgs);
- * @param args - Bucket Args
- public BucketArgs(BucketArgs args) {
- this(args.getBucketName(), args);
- * Returns the Bucket Name.
- * @return Bucket Name
- * Returns Versioning Info.
- * @return versioning
- * SetVersioning Info.
- * @param versioning - Enum value
- * returns the current Storage Class.
- * @return Storage Class
- * Sets the Storage Class.
- * @param storageType Set Storage Class
- * returns - Volume/bucketName.
- public String getResourceName() {
- return getVolumeName() + "/" + getBucketName();
- * Returns User/Volume name which is the parent of this
- * bucket.
- public String getParentName() {
- return getUserName() + "/" + getVolumeName();
@@ -1,117 +0,0 @@
- * Class that packages all key Arguments.
-public class KeyArgs extends BucketArgs {
- private String key;
- private String hash;
- * Constructor for Key Args.
- * @param volumeName - Volume Name
- * @param bucketName - Bucket Name
- * @param objectName - Key
- public KeyArgs(String volumeName, String bucketName,
- String objectName, UserArgs args) {
- super(volumeName, bucketName, args);
- this.key = objectName;
- public KeyArgs(String objectName, BucketArgs args) {
- super(args);
- * Get Key Name.
- return this.key;
- * Computed File hash.
- public String getHash() {
- return hash;
- * Sets the hash String.
- * @param hash String
- public void setHash(String hash) {
- this.hash = hash;
- * Returns the file size.
- * @return long - file size
- * Set Size.
- * @param size Size of the file
- * Returns the name of the resource.
- return super.getResourceName() + "/" + getKeyName();
- * Parent name of this resource.
- * @return String.
- return super.getResourceName();
@@ -1,142 +0,0 @@
- * Supports listing keys with pagination.
-public class ListArgs<T extends UserArgs> {
- private String prevKey;
- private String prefix;
- private int maxKeys;
- private boolean rootScan;
- private T args;
- * Constructor for ListArgs.
- * @param args - BucketArgs
- * @param prefix Prefix to start Query from
- * @param maxKeys Max result set
- * @param prevKey - Page token
- public ListArgs(T args, String prefix, int maxKeys,
- String prevKey) {
- setArgs(args);
- setPrefix(prefix);
- setMaxKeys(maxKeys);
- setPrevKey(prevKey);
- * Copy Constructor for ListArgs.
- * @param args - List Args
- public ListArgs(T args, ListArgs listArgs) {
- this(args, listArgs.getPrefix(), listArgs.getMaxKeys(),
- listArgs.getPrevKey());
- * Returns page token.
- public String getPrevKey() {
- return prevKey;
- * Sets page token.
- public void setPrevKey(String prevKey) {
- this.prevKey = prevKey;
- * Gets max keys.
- public int getMaxKeys() {
- return maxKeys;
- * Sets max keys.
- * @param maxKeys - Maximum keys to return
- public void setMaxKeys(int maxKeys) {
- this.maxKeys = maxKeys;
- * Gets prefix.
- public String getPrefix() {
- return prefix;
- * Sets prefix.
- * @param prefix - The prefix that we are looking for
- public void setPrefix(String prefix) {
- this.prefix = prefix;
- * Gets args.
- * @return T
- public T getArgs() {
- return args;
- * Sets args.
- * @param args T
- public void setArgs(T args) {
- this.args = args;
- * Checks if we are doing a rootScan.
- * @return - RootScan.
- public boolean isRootScan() {
- return rootScan;
- * Sets the RootScan property.
- * @param rootScan - Boolean.
- public void setRootScan(boolean rootScan) {
- this.rootScan = rootScan;
@@ -1,172 +0,0 @@
-import javax.ws.rs.core.HttpHeaders;
-import javax.ws.rs.core.Request;
-import javax.ws.rs.core.UriInfo;
-import java.util.Arrays;
- * UserArgs is used to package caller info
- * and pass it down to file system.
-public class UserArgs {
- private String userName;
- private final String requestID;
- private final String hostName;
- private final UriInfo uri;
- private final Request request;
- private final HttpHeaders headers;
- private String[] groups;
- * Constructs user args.
- * @param userName - User name
- * @param requestID - Request ID
- * @param hostName - Host Name
- * @param req - Request
- * @param info - Uri Info
- * @param httpHeaders - http headers
- public UserArgs(String userName, String requestID, String hostName,
- Request req, UriInfo info, HttpHeaders httpHeaders) {
- this.hostName = hostName;
- this.userName = userName;
- this.requestID = requestID;
- this.uri = info;
- this.request = req;
- this.headers = httpHeaders;
- * Constructs user args when we don't know the user name yet.
- * @param requestID _ Request ID
- * @param info - UriInfo
- public UserArgs(String requestID, String hostName, Request req, UriInfo info,
- HttpHeaders httpHeaders) {
- * Returns hostname.
- public String getHostName() {
- return hostName;
- * Returns RequestID.
- * @return Long
- public String getRequestID() {
- return requestID;
- * Returns User Name.
- public String getUserName() {
- return userName;
- * Sets the user name.
- * @param userName Name of the user
- public void setUserName(String userName) {
- * Returns list of groups.
- * @return String[]
- public String[] getGroups() {
- return groups != null ?
- Arrays.copyOf(groups, groups.length) : null;
- * Sets the group list.
- * @param groups list of groups
- public void setGroups(String[] groups) {
- if (groups != null) {
- this.groups = Arrays.copyOf(groups, groups.length);
- * Returns the resource Name.
- * @return String Resource.
- return getUserName();
- * Returns Http Headers for this call.
- * @return httpHeaders
- public HttpHeaders getHeaders() {
- return headers;
- * Returns Request Object.
- * @return Request
- public Request getRequest() {
- return request;
- * Returns UriInfo.
- * @return UriInfo
- public UriInfo getUri() {
- return uri;
@@ -1,143 +0,0 @@
-import org.apache.hadoop.ozone.web.request.OzoneQuota;
- * VolumeArgs is used to package all volume
- * related arguments in the call to underlying
- * file system.
-public class VolumeArgs extends UserArgs {
- private String adminName;
- private final String volumeName;
- * Returns Quota Information.
- * Returns volume name.
- * Constructs volume Args.
- * @param volumeName - volume Name
- * @param request - Http Request
- * @param info - URI info
- * @param headers - http headers
- * @param groups - list of groups allowed to access the volume
- @SuppressWarnings("parameternumber")
- public VolumeArgs(String userName, String volumeName, String requestID,
- String hostName, Request request, UriInfo info,
- HttpHeaders headers, String[] groups) {
- super(userName, requestID, hostName, request, info, headers);
- super.setGroups(groups);
- public VolumeArgs(String volumeName, UserArgs userArgs) {
- this(userArgs.getUserName(), volumeName, userArgs.getRequestID(),
- userArgs.getHostName(), userArgs.getRequest(), userArgs.getUri(),
- userArgs.getHeaders(), userArgs.getGroups());
- * Creates VolumeArgs from another VolumeArgs.
- public VolumeArgs(VolumeArgs volArgs) {
- this(volArgs.getVolumeName(), volArgs);
- * Sets Quota information.
- * @param quota - Quota Sting
- * @throws IllegalArgumentException
- public void setQuota(String quota) throws IllegalArgumentException {
- this.quota = OzoneQuota.parseQuota(quota);
- * Sets quota information.
- * @param quota - OzoneQuota
- * Gets admin Name.
- * @return - Admin Name
- public String getAdminName() {
- return adminName;
- * Sets Admin Name.
- * @param adminName - Admin Name
- public void setAdminName(String adminName) {
- this.adminName = adminName;
- * Returns UserName/VolumeName.
- return super.getResourceName() + "/" + getVolumeName();
- * REST handler value classes.
@@ -1,214 +0,0 @@
-package org.apache.hadoop.ozone.web.request;
-import com.fasterxml.jackson.annotation.JsonIgnore;
- * represents an OzoneQuota Object that can be applied to
- * a storage volume.
-public class OzoneQuota {
- private Units unit;
- private int size;
- /** Quota Units.*/
- public enum Units {UNDEFINED, BYTES, MB, GB, TB}
- * Returns size.
- public int getSize() {
- * Returns Units.
- * @return Unit in MB, GB or TB
- public Units getUnit() {
- return unit;
- * Constructs a default Quota object.
- public OzoneQuota() {
- this.size = 0;
- this.unit = Units.UNDEFINED;
- * Constructor for Ozone Quota.
- * @param size - Integer Size
- * @param unit MB, GB or TB
- public OzoneQuota(int size, Units unit) {
- this.unit = unit;
- * Formats a quota as a string.
- * @param quota the quota to format
- * @return string representation of quota
- public static String formatQuota(OzoneQuota quota) {
- return String.valueOf(quota.size) + quota.unit;
- * Parses a user provided string and returns the
- * Quota Object.
- * @param quotaString Quota String
- * @return OzoneQuota object
- public static OzoneQuota parseQuota(String quotaString)
- throws IllegalArgumentException {
- if ((quotaString == null) || (quotaString.isEmpty())) {
- "Quota string cannot be null or empty.");
- if (isRemove(quotaString)) {
- throw new IllegalArgumentException("Remove is invalid in this context.");
- String uppercase = quotaString.toUpperCase().replaceAll("\\s+", "");
- String size = "";
- int nSize;
- Units currUnit = Units.MB;
- Boolean found = false;
- if (uppercase.endsWith(Header.OZONE_QUOTA_MB)) {
- size = uppercase
- .substring(0, uppercase.length() - Header.OZONE_QUOTA_MB.length());
- currUnit = Units.MB;
- found = true;
- if (uppercase.endsWith(Header.OZONE_QUOTA_GB)) {
- .substring(0, uppercase.length() - Header.OZONE_QUOTA_GB.length());
- currUnit = Units.GB;
- if (uppercase.endsWith(Header.OZONE_QUOTA_TB)) {
- .substring(0, uppercase.length() - Header.OZONE_QUOTA_TB.length());
- currUnit = Units.TB;
- if (uppercase.endsWith(Header.OZONE_QUOTA_BYTES)) {
- .substring(0, uppercase.length() - Header.OZONE_QUOTA_BYTES.length());
- currUnit = Units.BYTES;
- if (!found) {
- "Quota unit not recognized. Supported values are BYTES, MB, GB and " +
- "TB.");
- nSize = Integer.parseInt(size);
- if (nSize < 0) {
- throw new IllegalArgumentException("Quota cannot be negative.");
- return new OzoneQuota(nSize, currUnit);
- * Checks if Quota String is just as remove operation.
- * @param quotaString User provided quota String
- * @return True if it is Remove, false otherwise
- public static boolean isRemove(String quotaString) {
- return (quotaString != null) &&
- (quotaString.compareToIgnoreCase(Header.OZONE_QUOTA_REMOVE) == 0);
- * Returns size in Bytes or -1 if there is no Quota.
- @JsonIgnore
- public long sizeInBytes() {
- switch (this.unit) {
- case BYTES:
- return this.getSize();
- case MB:
- return this.getSize() * OzoneConsts.MB;
- case GB:
- return this.getSize() * OzoneConsts.GB;
- case TB:
- return this.getSize() * OzoneConsts.TB;
- case UNDEFINED:
- * Returns OzoneQuota corresponding to size in bytes.
- * @param sizeInBytes size in bytes to be converted
- public static OzoneQuota getOzoneQuota(long sizeInBytes) {
- long size;
- Units unit;
- if (sizeInBytes % OzoneConsts.TB == 0) {
- size = sizeInBytes / OzoneConsts.TB;
- unit = Units.TB;
- } else if (sizeInBytes % OzoneConsts.GB == 0) {
- size = sizeInBytes / OzoneConsts.GB;
- unit = Units.GB;
- } else if (sizeInBytes % OzoneConsts.MB == 0) {
- size = sizeInBytes / OzoneConsts.MB;
- unit = Units.MB;
- size = sizeInBytes;
- unit = Units.BYTES;
- return new OzoneQuota((int)size, unit);
- * Netty-based HTTP server implementation for Ozone.
@@ -1,335 +0,0 @@
-package org.apache.hadoop.ozone.web.response;
-import org.apache.hadoop.ozone.web.utils.JsonUtils;
-import com.fasterxml.jackson.annotation.JsonFilter;
-import com.fasterxml.jackson.annotation.PropertyAccessor;
-import com.fasterxml.jackson.databind.ObjectWriter;
-import com.fasterxml.jackson.databind.ser.FilterProvider;
-import com.fasterxml.jackson.databind.ser.impl.SimpleBeanPropertyFilter;
-import com.fasterxml.jackson.databind.ser.impl.SimpleFilterProvider;
- * BucketInfo class, this is used as response class to send
- * Json info about a bucket back to a client.
- static final String BUCKET_INFO = "BUCKET_INFO_FILTER";
- private static final ObjectWriter WRITER;
- ObjectMapper mapper = new ObjectMapper();
- String[] ignorableFieldNames = {"bytesUsed", "keyCount"};
- FilterProvider filters = new SimpleFilterProvider().addFilter(BUCKET_INFO,
- SimpleBeanPropertyFilter.serializeAllExcept(ignorableFieldNames));
- mapper.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY);
- mapper.addMixIn(Object.class, MixIn.class);
- mapper.setFilterProvider(filters);
- WRITER = mapper.writerWithDefaultPrettyPrinter();
- private long bytesUsed;
- private long keyCount;
- private String encryptionKeyName;
- acls = new ArrayList<>();
- * @param jsonString - Json String
- * @return - BucketInfo
- * Returns a List of ACL on the Bucket.
- * @return List of Acls
- * @param acls - Acls list
- * @param storageType - Storage Type
- * @param bucketName - Name of the bucket
- * @param creationTime - Date String
- public void setEncryptionKeyName(String encryptionKeyName) {
- this.encryptionKeyName = encryptionKeyName;
- return encryptionKeyName;
- * After stripping out bytesUsed and keyCount
- * Returns the Object as a Json String.
- * The reason why both toJSONString exists and toDBString exists
- * is because toJSONString supports an external facing contract with
- * REST clients. However server internally would want to add more
- * fields to this class. The distinction helps in serializing all
- * fields vs. only fields that are part of REST protocol.
- public String toDBString() throws IOException {
- return JsonUtils.toJsonString(this);
- * Sets the Volume Name of the bucket.
- * Get the number of bytes used by this bucket.
- public long getBytesUsed() {
- return bytesUsed;
- * Set bytes Used.
- * @param bytesUsed - bytesUsed
- public void setBytesUsed(long bytesUsed) {
- this.bytesUsed = bytesUsed;
- * Get Key Count inside this bucket.
- * @return - KeyCount
- public long getKeyCount() {
- return keyCount;
- * Set Key Count inside this bucket.
- * @param keyCount - Sets the Key Count
- public void setKeyCount(long keyCount) {
- this.keyCount = keyCount;
- * This class allows us to create custom filters
- * for the Json serialization.
- @JsonFilter(BUCKET_INFO)
- static class MixIn {
@@ -1,311 +0,0 @@
- * Represents an Ozone key Object.
- static final String OBJECT_INFO = "OBJECT_INFO_FILTER";
- String[] ignorableFieldNames = {"dataFileName"};
- FilterProvider filters = new SimpleFilterProvider()
- .addFilter(OBJECT_INFO, SimpleBeanPropertyFilter
- .serializeAllExcept(ignorableFieldNames));
- mapper.setVisibility(PropertyAccessor.FIELD,
- JsonAutoDetect.Visibility.ANY);
- @JsonFilter(OBJECT_INFO)
- class MixIn {
- private String dataFileName;
- * @param createdOn - Date String
- * @param modifiedOn - Date String
- * Full path to where the actual data for this key is stored.
- public String getDataFileName() {
- return dataFileName;
- * Sets up where the file path is stored.
- * @param dataFileName - Data File Name
- public void setDataFileName(String dataFileName) {
- this.dataFileName = dataFileName;
- * Gets the Keyname of this object.
- * @param keyName - String
- * Sets the MD5 of this file.
- * @param md5hash - Md5 of this file
- * Sets the size of the Data part of this key.
- * @param size - Size in long
- * @return - returns the version of this key.
- * Parse a string to retuen BucketInfo Object.
@@ -1,80 +0,0 @@
- * Represents an Ozone key Object with detail information of location.
- * @param keyLocations - details of key location
- public void setKeyLocations(List<KeyLocation> keyLocations) {
- this.keyLocations = keyLocations;
- .append(keyLocations, that.getKeyLocations())
@@ -1,82 +0,0 @@
- private final long containerID;
- private final long localID;
- private final long length;
- private final long offset;
@@ -1,154 +0,0 @@
-import java.util.Collections;
- * List Bucket is the response for the ListBucket Query.
-public class ListBuckets {
- static final String BUCKET_LIST = "BUCKET_LIST_FILTER";
- new ObjectMapper().readerFor(ListBuckets.class);
- .addFilter(BUCKET_LIST, SimpleBeanPropertyFilter
- private List<BucketInfo> buckets;
- * Constructor for ListBuckets.
- * @param buckets - List of buckets owned by this user
- public ListBuckets(List<BucketInfo> buckets) {
- this.buckets = buckets;
- public ListBuckets() {
- this.buckets = new LinkedList<BucketInfo>();
- * Parses a String to return ListBuckets object.
- * @param data - Json String
- * @return - ListBuckets
- public static ListBuckets parse(String data) throws IOException {
- * Returns a list of Buckets.
- * @return Bucket list
- public List<BucketInfo> getBuckets() {
- return buckets;
- * Sets the list of buckets owned by this user.
- * @param buckets - List of Buckets
- public void setBuckets(List<BucketInfo> buckets) {
- * Sorts the buckets based on bucketName.
- * This is useful when we return the list of buckets
- public void sort() {
- Collections.sort(buckets);
- * Add a new bucket to the list of buckets.
- * @param bucketInfo - bucket Info
- public void addBucket(BucketInfo bucketInfo){
- this.buckets.add(bucketInfo);
- @JsonFilter(BUCKET_LIST)
@@ -1,209 +0,0 @@
-import org.apache.hadoop.ozone.web.handlers.BucketArgs;
-import org.apache.hadoop.ozone.web.handlers.ListArgs;
- * This class the represents the list of keys (Objects) in a bucket.
-public class ListKeys {
- static final String OBJECT_LIST = "OBJECT_LIST_FILTER";
- new ObjectMapper().readerFor(ListKeys.class);
- .addFilter(OBJECT_LIST, SimpleBeanPropertyFilter
- private long maxKeys;
- private boolean truncated;
- private List<KeyInfo> keyList;
- * Default constructor needed for json serialization.
- public ListKeys() {
- this.keyList = new LinkedList<>();
- * Constructor for ListKeys.
- * @param args ListArgs
- * @param truncated is truncated
- public ListKeys(ListArgs args, boolean truncated) {
- Preconditions.checkState(args.getArgs() instanceof BucketArgs);
- this.name = ((BucketArgs) args.getArgs()).getBucketName();
- this.prefix = args.getPrefix();
- this.maxKeys = args.getMaxKeys();
- this.truncated = truncated;
- * Converts a Json string to POJO.
- * @param jsonString - json string.
- * @return ListObject
- * @throws IOException - Json conversion error.
- public static ListKeys parse(String jsonString) throws IOException {
- * Returns a list of Objects.
- * @return List of KeyInfo Objects.
- public List<KeyInfo> getKeyList() {
- return keyList;
- * Sets the list of Objects.
- * @param objectList - List of Keys
- public void setKeyList(List<KeyInfo> objectList) {
- this.keyList = objectList;
- * Gets the Max Key Count.
- public long getMaxKeys() {
- * Gets Prefix.
- * Gets truncated Status.
- * @return Boolean
- public boolean isTruncated() {
- return truncated;
- * Sets the value of truncated.
- * @param value - Boolean
- public void setTruncated(boolean value) {
- this.truncated = value;
- * Returns a JSON string of this object. After stripping out bytesUsed and
- * keyCount.
- * @throws IOException - On json Errors.
- * @throws IOException - on json errors.
- * Sorts the keys based on name and version. This is useful when we return the
- * list of keys.
- Collections.sort(keyList);
- * Add a new key to the list of keys.
- * @param keyInfo - key Info
- public void addKey(KeyInfo keyInfo){
- this.keyList.add(keyInfo);
- * This class allows us to create custom filters for the Json serialization.
- @JsonFilter(OBJECT_LIST)
@@ -1,152 +0,0 @@
- * List Volume Class is the class that is returned in JSON format to
- * users when they call ListVolumes.
-public class ListVolumes {
- private List<VolumeInfo> volumes;
- static final String VOLUME_LIST = "VOLUME_LIST_FILTER";
- new ObjectMapper().readerFor(ListVolumes.class);
- String[] ignorableFieldNames = {"bytesUsed", "bucketCount"};
- .addFilter(VOLUME_LIST, SimpleBeanPropertyFilter
- * Used for json filtering.
- @JsonFilter(VOLUME_LIST)
- * Constructs ListVolume objects.
- public ListVolumes() {
- this.volumes = new LinkedList<VolumeInfo>();
- * Gets the list of volumes.
- * @return List of VolumeInfo Objects
- public List<VolumeInfo> getVolumes() {
- return volumes;
- * Sets volume info.
- * @param volumes - List of Volumes
- public void setVolumes(List<VolumeInfo> volumes) {
- this.volumes = volumes;
- * After stripping out bytesUsed and bucketCount
- * When we serialize a volumeInfo to our database
- * we will use all fields. However the toJsonString
- * will strip out bytesUsed and bucketCount from the
- * volume Info
- * @return Json String
- * Parses a String to return ListVolumes object.
- * @return - ListVolumes
- public static ListVolumes parse(String data) throws IOException {
- * Adds a new volume info to the List.
- * @param info - VolumeInfo
- public void addVolume(VolumeInfo info) {
- this.volumes.add(info);
- * Sorts the volume names based on volume name.
- * This is useful when we return the list of volume names
- Collections.sort(volumes);
@@ -1,308 +0,0 @@
- * VolumeInfo Class is the Java class that represents
- * Json when VolumeInfo Call is made.
- static final String VOLUME_INFO = "VOLUME_INFO_FILTER";
- .addFilter(VOLUME_INFO, SimpleBeanPropertyFilter
- * Custom Json Filter Class.
- @JsonFilter(VOLUME_INFO)
- private long bucketCount;
- public VolumeInfo(String volumeName, String createdOn, String createdBy) {
- * @param createdBy - UserName
- * @return - Date String
- * @return - OwnerInfo
- * @param owner - OwnerInfo
- * @param quota - Quota Info
- * @return - Volume Name
- * Gets the number of bytesUsed by this volume.
- * @return long - Bytes used
- * Sets number of bytesUsed by this volume.
- * @param bytesUsed - Number of bytesUsed
- * Total number of buckets under this volume.
- * @return - bucketCount
- public long getBucketCount() {
- return bucketCount;
- * Sets the buckets count.
- * @param bucketCount - Bucket Count
- public void setBucketCount(long bucketCount) {
- this.bucketCount = bucketCount;
@@ -1,60 +0,0 @@
- * @param name - name of the User
-package org.apache.hadoop.ozone.web;
-import org.apache.hadoop.ozone.web.response.BucketInfo;
-import org.junit.Test;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
- * Test Ozone Bucket Info operation.
-public class TestBucketInfo {
- @Test
- public void testBucketInfoJson() throws IOException {
- BucketInfo bucketInfo = new BucketInfo("volumeName", "bucketName");
- String bucketInfoString = bucketInfo.toJsonString();
- BucketInfo newBucketInfo = BucketInfo.parse(bucketInfoString);
- assertEquals(bucketInfo, newBucketInfo);
- public void testBucketInfoDBString() throws IOException {
- String bucketInfoString = bucketInfo.toDBString();
- public void testBucketInfoAddAcls() throws IOException {
- List<OzoneAcl> aclList = new ArrayList<>();
- aclList.add(OzoneAcl.parseAcl("user:bilbo:r"));
- aclList.add(OzoneAcl.parseAcl("user:samwise:rw"));
- newBucketInfo.setAcls(aclList);
- assertNotNull(newBucketInfo.getAcls());
- assertEquals(2, newBucketInfo.getAcls().size());
- public void testBucketInfoVersionAndType() throws IOException {
- bucketInfo.setVersioning(OzoneConsts.Versioning.ENABLED);
- bucketInfo.setStorageType(StorageType.DISK);
@@ -1,115 +0,0 @@
- * 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>
- * 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.
-import java.util.Set;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
- * Test Ozone Volume Quota.
-public class TestQuota {
- public void testParseQuota() {
- HashMap<String, Boolean> testMatrix;
- testMatrix = new HashMap<String, Boolean>();
- testMatrix.put("10TB", Boolean.TRUE);
- testMatrix.put("1 TB", Boolean.TRUE);
- testMatrix.put("0MB", Boolean.TRUE);
- testMatrix.put("0 TB", Boolean.TRUE);
- testMatrix.put(" 1000MB ", Boolean.TRUE);
- testMatrix.put(" 1000MBMB ", Boolean.FALSE);
- testMatrix.put(" 1000MB00 ", Boolean.FALSE);
- testMatrix.put("1000ZMB", Boolean.FALSE);
- testMatrix.put("MB1000", Boolean.FALSE);
- testMatrix.put("9999", Boolean.FALSE);
- testMatrix.put("1", Boolean.FALSE);
- testMatrix.put("remove", Boolean.FALSE);
- testMatrix.put("1UNDEFINED", Boolean.FALSE);
- testMatrix.put(null, Boolean.FALSE);
- testMatrix.put("", Boolean.FALSE);
- testMatrix.put("-1000MB", Boolean.FALSE);
- testMatrix.put("1024 bytes", Boolean.TRUE);
- testMatrix.put("1bytes", Boolean.TRUE);
- testMatrix.put("0bytes", Boolean.TRUE);
- testMatrix.put("10000 BYTES", Boolean.TRUE);
- testMatrix.put("BYTESbytes", Boolean.FALSE);
- testMatrix.put("bytes", Boolean.FALSE);
- Set<String> keys = testMatrix.keySet();
- for (String key : keys) {
- if (testMatrix.get(key)) {
- OzoneQuota.parseQuota(key);
- // should never get here since the isValid call will throw
- fail(key);
- fail("An exception was expected but did not happen.");
- } catch (IllegalArgumentException e) {
- public void testVerifyQuota() {
- OzoneQuota qt = OzoneQuota.parseQuota("10TB");
- assertEquals(10, qt.getSize());
- assertEquals(OzoneQuota.Units.TB, qt.getUnit());
- assertEquals(10L * (1024L * 1024L * 1024L * 1024L), qt.sizeInBytes());
- qt = OzoneQuota.parseQuota("10MB");
- assertEquals(OzoneQuota.Units.MB, qt.getUnit());
- assertEquals(10L * (1024L * 1024L), qt.sizeInBytes());
- qt = OzoneQuota.parseQuota("10GB");
- assertEquals(OzoneQuota.Units.GB, qt.getUnit());
- assertEquals(10L * (1024L * 1024L * 1024L), qt.sizeInBytes());
- qt = OzoneQuota.parseQuota("10BYTES");
- assertEquals(OzoneQuota.Units.BYTES, qt.getUnit());
- assertEquals(10L, qt.sizeInBytes());
- OzoneQuota emptyQuota = new OzoneQuota();
- assertEquals(-1L, emptyQuota.sizeInBytes());
- assertEquals(0, emptyQuota.getSize());
- assertEquals(OzoneQuota.Units.UNDEFINED, emptyQuota.getUnit());
- public void testVerifyRemove() {
- assertTrue(OzoneQuota.isRemove("remove"));
- assertFalse(OzoneQuota.isRemove("not remove"));
- assertFalse(OzoneQuota.isRemove(null));
@@ -1,101 +0,0 @@
-import java.util.HashSet;
-import static org.apache.hadoop.ozone.web.utils.OzoneUtils.getRequestID;
-import static org.apache.hadoop.ozone.web.utils.OzoneUtils.verifyResourceName;
- * Test Ozone Utility operations like verifying resource name.
-public class TestUtils {
- * Tests if the bucket name handling is correct.
- public void testValidBucketNames() {
- // Init the Table with Strings and Expected Return values
- testMatrix.put("bucket-.ozone.self", Boolean.FALSE);
- testMatrix.put("bucket.-ozone.self", Boolean.FALSE);
- testMatrix.put(".bucket.ozone.self", Boolean.FALSE);
- testMatrix.put("bucket.ozone.self.", Boolean.FALSE);
- testMatrix.put("bucket..ozone.self", Boolean.FALSE);
- testMatrix.put("192.1.1.1", Boolean.FALSE);
- testMatrix.put("ab", Boolean.FALSE);
- testMatrix.put("bucket.ozone.self.this.is.a.really.long.name.that."
- + "is.more.than.sixty.three.characters.long.for.sure", Boolean.FALSE);
- testMatrix.put("bucket@$", Boolean.FALSE);
- testMatrix.put("BUCKET", Boolean.FALSE);
- testMatrix.put("bucket .ozone.self", Boolean.FALSE);
- testMatrix.put(" bucket.ozone.self", Boolean.FALSE);
- testMatrix.put("bucket.ozone.self-", Boolean.FALSE);
- testMatrix.put("-bucket.ozone.self", Boolean.FALSE);
- testMatrix.put("bucket", Boolean.TRUE);
- testMatrix.put("bucket.ozone.self", Boolean.TRUE);
- testMatrix.put("bucket-name.ozone.self", Boolean.TRUE);
- testMatrix.put("bucket.1.ozone.self", Boolean.TRUE);
- // For valid names there should be no exceptions at all
- verifyResourceName(key);
- * Just calls Request ID many times and assert we
- * got different values, ideally this should be
- * run under parallel threads. Since the function under
- * test has no external dependencies it is assumed
- * that this test is good enough.
- public void testRequestIDisRandom() {
- HashSet<String> set = new HashSet<>();
- for (int i = 0; i < 1000; i++) {
- assertTrue(set.add(getRequestID()));
@@ -1,73 +0,0 @@
-import org.apache.hadoop.ozone.web.response.VolumeInfo;
-import org.apache.hadoop.ozone.web.response.VolumeOwner;
- * Test Ozone Volume info structure.
-public class TestVolumeStructs {
- public void testVolumeInfoParse() throws IOException {
- new VolumeInfo("testvol", "Thu, Apr 9, 2015 10:23:45 GMT", "gandalf");
- VolumeOwner owner = new VolumeOwner("bilbo");
- volInfo.setOwner(owner);
- String jString = volInfo.toJsonString();
- VolumeInfo newVollInfo = VolumeInfo.parse(jString);
- String one = volInfo.toJsonString();
- String two = newVollInfo.toJsonString();
- assertEquals(volInfo.toJsonString(), newVollInfo.toJsonString());
- public void testVolumeInfoValue() throws IOException {
- String createdOn = "Thu, Apr 9, 2015 10:23:45 GMT";
- String createdBy = "gandalf";
- VolumeInfo volInfo = new VolumeInfo("testvol", createdOn, createdBy);
- assertEquals(volInfo.getCreatedBy(), createdBy);
- assertEquals(volInfo.getCreatedOn(), createdOn);
- public void testVolumeListParse() throws IOException {
- ListVolumes list = new ListVolumes();
- for (int x = 0; x < 100; x++) {
- VolumeInfo volInfo = new VolumeInfo("testvol" + Integer.toString(x),
- "Thu, Apr 9, 2015 10:23:45 GMT", "gandalf");
- list.addVolume(volInfo);
- list.sort();
- String listString = list.toJsonString();
- ListVolumes newList = ListVolumes.parse(listString);
- assertEquals(list.toJsonString(), newList.toJsonString());
@@ -47,10 +47,6 @@
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdds-container-service</artifactId>
</dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-ozone-objectstore-service</artifactId>
- </dependency>
<dependency>
<groupId>com.sun.xml.bind</groupId>
<artifactId>jaxb-impl</artifactId>
@@ -112,10 +112,6 @@ run cp "${ROOT}/hadoop-ozone/common/src/main/bin/stop-ozone.sh" "sbin/"
# fault injection tests
run cp -r "${ROOT}/hadoop-ozone/fault-injection-test/network-tests/src/test/blockade" tests
-#shaded datanode service
-run mkdir -p "./share/hadoop/ozoneplugin"
-run cp "${ROOT}/hadoop-ozone/objectstore-service/target/hadoop-ozone-objectstore-service-${HDDS_VERSION}-plugin.jar" "./share/hadoop/ozoneplugin/hadoop-ozone-datanode-plugin-${HDDS_VERSION}.jar"
# Optional documentation, could be missing
cp -r "${ROOT}/hadoop-hdds/docs/target/classes/docs" ./
@@ -19,27 +19,34 @@ Library OperatingSystem
Resource ../commonlib.robot
Test Setup Run Keyword if '${SECURITY_ENABLED}' == 'true' Kinit test user testuser testuser.keytab
Test Timeout 2 minute
+Suite Setup Generate prefix
*** Variables ***
+${prefix} generated
+*** Keywords ***
+Generate prefix
+ ${random} = Generate Random String 5 [NUMBERS]
+ Set Suite Variable ${prefix} ${random}
*** Test Cases ***
RpcClient with port
- Test ozone shell o3:// om:9862 rpcwoport
+ Test ozone shell o3:// om:9862 ${prefix}-rpcwoport
RpcClient volume acls
- Test Volume Acls o3:// om:9862 rpcwoport2
+ Test Volume Acls o3:// om:9862 ${prefix}-rpcwoport2
RpcClient bucket acls
- Test Bucket Acls o3:// om:9862 rpcwoport2
+ Test Bucket Acls o3:// om:9862 ${prefix}-rpcwoport2
RpcClient key acls
- Test Key Acls o3:// om:9862 rpcwoport2
+ Test Key Acls o3:// om:9862 ${prefix}-rpcwoport2
RpcClient without host
- Test ozone shell o3:// ${EMPTY} rpcwport
+ Test ozone shell o3:// ${EMPTY} ${prefix}-rpcwport
RpcClient without scheme
- Test ozone shell ${EMPTY} ${EMPTY} rpcwoscheme
+ Test ozone shell ${EMPTY} ${EMPTY} ${prefix}-rpcwoscheme
*** Keywords ***
@@ -48,20 +55,20 @@ Test ozone shell
${result} = Execute ozone sh volume create ${protocol}${server}/${volume} --quota 100TB
Should not contain ${result} Failed
Should contain ${result} Creating Volume: ${volume}
- ${result} = Execute ozone sh volume list ${protocol}${server}/ | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.[] | select(.volumeName=="${volume}")'
- Should contain ${result} createdOn
- ${result} = Execute ozone sh volume list | grep -Ev 'Removed|DEBUG|ERROR|INFO|TRACE|WARN' | jq -r '.[] | select(.volumeName=="${volume}")'
+ ${result} = Execute ozone sh volume list ${protocol}${server}/ | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.name=="${volume}")'
+ Should contain ${result} creationTime
+ ${result} = Execute ozone sh volume list | grep -Ev 'Removed|DEBUG|ERROR|INFO|TRACE|WARN' | jq -r '. | select(.name=="${volume}")'
# TODO: Disable updating the owner, acls should be used to give access to other user.
Execute ozone sh volume update ${protocol}${server}/${volume} --quota 10TB
# ${result} = Execute ozone sh volume info ${protocol}${server}/${volume} | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.volumeName=="${volume}") | .owner | .name'
# Should Be Equal ${result} bill
- ${result} = Execute ozone sh volume info ${protocol}${server}/${volume} | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.volumeName=="${volume}") | .quota | .size'
- Should Be Equal ${result} 10
+ ${result} = Execute ozone sh volume info ${protocol}${server}/${volume} | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.name=="${volume}") | .quota'
+ Should Be Equal ${result} 10995116277760
Execute ozone sh bucket create ${protocol}${server}/${volume}/bb1
- ${result} = Execute ozone sh bucket info ${protocol}${server}/${volume}/bb1 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.bucketName=="bb1") | .storageType'
+ ${result} = Execute ozone sh bucket info ${protocol}${server}/${volume}/bb1 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.name=="bb1") | .storageType'
Should Be Equal ${result} DISK
- ${result} = Execute ozone sh bucket list ${protocol}${server}/${volume}/ | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.[] | select(.bucketName=="bb1") | .volumeName'
+ ${result} = Execute ozone sh bucket list ${protocol}${server}/${volume}/ | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.name=="bb1") | .volumeName'
Should Be Equal ${result} ${volume}
Run Keyword Test key handling ${protocol} ${server} ${volume}
Execute ozone sh bucket delete ${protocol}${server}/${volume}/bb1
@@ -106,12 +113,12 @@ Test key handling
Execute rm -f NOTICE.txt.1
Execute ozone sh key get ${protocol}${server}/${volume}/bb1/key1 NOTICE.txt.1
Execute ls -l NOTICE.txt.1
- ${result} = Execute ozone sh key info ${protocol}${server}/${volume}/bb1/key1 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.keyName=="key1")'
- ${result} = Execute ozone sh key list ${protocol}${server}/${volume}/bb1 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.[] | select(.keyName=="key1") | .keyName'
+ ${result} = Execute ozone sh key info ${protocol}${server}/${volume}/bb1/key1 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.name=="key1")'
+ ${result} = Execute ozone sh key list ${protocol}${server}/${volume}/bb1 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.name=="key1") | .name'
Should Be Equal ${result} key1
Execute ozone sh key rename ${protocol}${server}/${volume}/bb1 key1 key2
- ${result} = Execute ozone sh key list ${protocol}${server}/${volume}/bb1 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.[].keyName'
+ ${result} = Execute ozone sh key list ${protocol}${server}/${volume}/bb1 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.name'
Should Be Equal ${result} key2
Execute ozone sh key delete ${protocol}${server}/${volume}/bb1/key2
@@ -34,14 +34,14 @@ Check volume from ozonefs
Run ozoneFS tests
Execute ozone fs -mkdir -p o3fs://bucket1.fstest/testdir/deep
- ${result} = Execute ozone sh key list o3://om/fstest/bucket1 | grep -v WARN | jq -r '.[].keyName'
+ ${result} = Execute ozone sh key list o3://om/fstest/bucket1 | grep -v WARN | jq -r '.name'
Should contain ${result} testdir/deep
Execute ozone fs -copyFromLocal NOTICE.txt o3fs://bucket1.fstest/testdir/deep/
Should contain ${result} NOTICE.txt
Execute ozone fs -put NOTICE.txt o3fs://bucket1.fstest/testdir/deep/PUTFILE.txt
Should contain ${result} PUTFILE.txt
${result} = Execute ozone fs -ls o3fs://bucket1.fstest/testdir/deep/
@@ -49,13 +49,13 @@ Run ozoneFS tests
Execute ozone fs -mv o3fs://bucket1.fstest/testdir/deep/NOTICE.txt o3fs://bucket1.fstest/testdir/deep/MOVED.TXT
Should contain ${result} MOVED.TXT
Should not contain ${result} NOTICE.txt
Execute ozone fs -mkdir -p o3fs://bucket1.fstest/testdir/deep/subdir1
Execute ozone fs -cp o3fs://bucket1.fstest/testdir/deep/MOVED.TXT o3fs://bucket1.fstest/testdir/deep/subdir1/NOTICE.txt
Should contain ${result} subdir1/NOTICE.txt
${result} = Execute ozone fs -ls o3fs://bucket1.fstest/testdir/deep/subdir1/
@@ -65,19 +65,19 @@ Run ozoneFS tests
Execute ozone fs -rm o3fs://bucket1.fstest/testdir/deep/subdir1/NOTICE.txt
${result} = Execute ozone fs -rmdir o3fs://bucket1.fstest/testdir/deep/subdir1/
Should not contain ${result} subdir1
Execute ozone fs -touch o3fs://bucket1.fstest/testdir/TOUCHFILE.txt
Should contain ${result} TOUCHFILE.txt
Execute ozone fs -rm -r o3fs://bucket1.fstest/testdir/
Should not contain ${result} testdir
Execute rm -Rf localdir1
@@ -48,10 +48,6 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
<scope>test</scope>
<artifactId>hadoop-ozone-s3gateway</artifactId>
@@ -137,15 +137,6 @@ public interface MiniOzoneCluster {
OzoneClient getRpcClient() throws IOException;
- * Returns an REST based {@link OzoneClient} to access the
- * {@link MiniOzoneCluster}.
- * @return {@link OzoneClient}
- OzoneClient getRestClient() throws IOException;
* Returns StorageContainerLocationClient to communicate with
* {@link StorageContainerManager} associated with the MiniOzoneCluster.
@@ -36,7 +36,6 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.ozone.client.OzoneClient;
import org.apache.hadoop.ozone.client.OzoneClientFactory;
import org.apache.hadoop.ozone.common.Storage.StorageState;
import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
import org.apache.hadoop.ozone.om.OMConfigKeys;
@@ -67,7 +66,6 @@ import java.util.concurrent.TimeoutException;
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_HEARTBEAT_INTERVAL;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState
.HEALTHY;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_PLUGINS_KEY;
import static org.apache.hadoop.ozone.OzoneConfigKeys
.DFS_CONTAINER_IPC_PORT;
@@ -212,18 +210,6 @@ public class MiniOzoneClusterImpl implements MiniOzoneCluster {
return OzoneClientFactory.getRpcClient(conf);
- * Creates an {@link OzoneClient} connected to this cluster's REST
- * service. Callers take ownership of the client and must close it when done.
- * @return OzoneRestClient connected to this cluster's REST service
- * @throws OzoneException if Ozone encounters an error creating the client
- public OzoneClient getRestClient() throws IOException {
- return OzoneClientFactory.getRestClient(conf);
* Returns an RPC proxy connected to this cluster's StorageContainerManager
* for accessing container location information. Callers take ownership of
@@ -619,8 +605,6 @@ public class MiniOzoneClusterImpl implements MiniOzoneCluster {
private void configureHddsDatanodes() {
conf.set(ScmConfigKeys.HDDS_REST_HTTP_ADDRESS_KEY, "0.0.0.0:0");
conf.set(HddsConfigKeys.HDDS_DATANODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
- conf.set(HDDS_DATANODE_PLUGINS_KEY,
- "org.apache.hadoop.ozone.web.OzoneHddsDatanodeService");
conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT,
randomContainerPort);
conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_RANDOM_PORT,
@@ -18,15 +18,23 @@
package org.apache.hadoop.ozone;
+import java.io.Closeable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.ratis.RatisHelper;
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.ratis.RatisHelper;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL;
+import static org.apache.hadoop.hdds.ratis.RatisHelper.newRaftClient;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL;
import org.apache.ratis.client.RaftClient;
import org.apache.ratis.protocol.RaftPeer;
import org.apache.ratis.rpc.RpcType;
@@ -35,16 +43,6 @@ import org.apache.ratis.util.TimeDuration;
-import java.io.Closeable;
-import java.util.concurrent.TimeoutException;
-import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL;
-import static org.apache.hadoop.hdds.ratis.RatisHelper.newRaftClient;
* Helpers for Ratis tests.
@@ -80,7 +78,7 @@ public interface RatisTestHelper {
public ClientProtocol newOzoneClient()
- throws OzoneException, URISyntaxException, IOException {
+ throws IOException {
return new RpcClient(conf);
@@ -0,0 +1,95 @@
+/*
+ * 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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.Scanner;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.protocol.StorageType;
+import org.apache.hadoop.ozone.client.BucketArgs;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.VolumeArgs;
+import org.apache.commons.lang3.RandomStringUtils;
+/**
+ * Utility to help to generate test data.
+public final class TestDataUtil {
+ private TestDataUtil() {
+ public static OzoneBucket createVolumeAndBucket(MiniOzoneCluster cluster,
+ String volumeName, String bucketName) throws IOException {
+ String userName = "user" + RandomStringUtils.randomNumeric(5);
+ String adminName = "admin" + RandomStringUtils.randomNumeric(5);
+ OzoneClient client = cluster.getClient();
+ VolumeArgs volumeArgs = VolumeArgs.newBuilder()
+ .setAdmin(adminName)
+ .setOwner(userName)
+ .build();
+ ObjectStore objectStore = client.getObjectStore();
+ objectStore.createVolume(volumeName, volumeArgs);
+ OzoneVolume volume = objectStore.getVolume(volumeName);
+ BucketArgs omBucketArgs = BucketArgs.newBuilder()
+ .setStorageType(StorageType.DISK)
+ volume.createBucket(bucketName, omBucketArgs);
+ return volume.getBucket(bucketName);
+ public static void createKey(OzoneBucket bucket, String keyName,
+ String content) throws IOException {
+ try (OutputStream stream = bucket
+ .createKey(keyName, content.length(), ReplicationType.STAND_ALONE,
+ ReplicationFactor.ONE, new HashMap<>())) {
+ stream.write(content.getBytes());
+ public static String getKey(OzoneBucket bucket, String keyName)
+ try (InputStream stream = bucket.readKey(keyName)) {
+ return new Scanner(stream).useDelimiter("\\A").next();
+ public static OzoneBucket createVolumeAndBucket(MiniOzoneCluster cluster)
+ String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
+ String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
+ return createVolumeAndBucket(cluster, volumeName, bucketName);
+}
@@ -1,413 +0,0 @@
-package org.apache.hadoop.ozone;
-import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
-import org.apache.hadoop.ozone.web.utils.OzoneUtils;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.Locale;
- * Helper functions to test Ozone.
-public class TestOzoneHelper {
- public CloseableHttpClient createHttpClient() {
- return HttpClients.createDefault();
- * Creates Volumes on Ozone Store.
- public void testCreateVolumes(int port) throws IOException {
- SimpleDateFormat format =
- new SimpleDateFormat("EEE, dd MMM yyyy HH:mm:ss ZZZ", Locale.US);
- CloseableHttpClient client = createHttpClient();
- String volumeName = OzoneUtils.getRequestID().toLowerCase();
- HttpPost httppost = new HttpPost(
- String.format("http://localhost:%d/%s", port, volumeName));
- httppost.addHeader(Header.OZONE_VERSION_HEADER,
- httppost.addHeader(HttpHeaders.DATE,
- format.format(new Date(Time.monotonicNow())));
- httppost.addHeader(HttpHeaders.AUTHORIZATION,
- OzoneConsts.OZONE_SIMPLE_HDFS_USER);
- httppost.addHeader(Header.OZONE_USER, OzoneConsts.OZONE_SIMPLE_HDFS_USER);
- HttpResponse response = client.execute(httppost);
- assertEquals(response.toString(), HTTP_CREATED,
- response.getStatusLine().getStatusCode());
- client.close();
- * Create Volumes with Quota.
- public void testCreateVolumesWithQuota(int port) throws IOException {
- String.format("http://localhost:%d/%s?quota=10TB", port, volumeName));
- * Create Volumes with Invalid Quota.
- public void testCreateVolumesWithInvalidQuota(int port) throws IOException {
- String.format("http://localhost:%d/%s?quota=NaN", port, volumeName));
- assertEquals(response.toString(), ErrorTable.MALFORMED_QUOTA
- .getHttpCode(),
- * To create a volume a user name must be specified using OZONE_USER header.
- * This test verifies that we get an error in case we call without a OZONE
- * user name.
- public void testCreateVolumesWithInvalidUser(int port) throws IOException {
- String.format("http://localhost:%d/%s?quota=1TB", port, volumeName));
- assertEquals(response.toString(), ErrorTable.USER_NOT_FOUND.getHttpCode(),
- * Only Admins can create volumes in Ozone. This test uses simple userauth as
- * backend and hdfs and root are admin users in the simple backend.
- * This test tries to create a volume as user bilbo.
- public void testCreateVolumesWithOutAdminRights(int port) throws IOException {
- "bilbo"); // This is not a root user in Simple Auth
- assertEquals(response.toString(), ErrorTable.ACCESS_DENIED.getHttpCode(),
- * Create a bunch of volumes in a loop.
- public void testCreateVolumesInLoop(int port) throws IOException {
- for (int x = 0; x < 1000; x++) {
- String userName = OzoneUtils.getRequestID().toLowerCase();
- httppost.addHeader(Header.OZONE_USER, userName);
- * Get volumes owned by the user.
- public void testGetVolumesByUser(int port) throws IOException {
- // We need to create a volume for this test to succeed.
- testCreateVolumes(port);
- HttpGet httpget =
- new HttpGet(String.format("http://localhost:%d/", port));
- httpget.addHeader(Header.OZONE_VERSION_HEADER,
- httpget.addHeader(HttpHeaders.DATE,
- httpget.addHeader(HttpHeaders.AUTHORIZATION,
- httpget.addHeader(Header.OZONE_USER,
- HttpResponse response = client.execute(httpget);
- assertEquals(response.toString(), HTTP_OK,
- * Admins can read volumes belonging to other users.
- public void testGetVolumesOfAnotherUser(int port) throws IOException {
- OzoneConsts.OZONE_SIMPLE_ROOT_USER);
- // User Root is getting volumes belonging to user HDFS
- httpget.addHeader(Header.OZONE_USER, OzoneConsts.OZONE_SIMPLE_HDFS_USER);
- * if you try to read volumes belonging to another user,
- * then server always ignores it.
- public void testGetVolumesOfAnotherUserShouldFail(int port)
- userName);
- // userName is NOT a root user, hence he should NOT be able to read the
- // volumes of user HDFS
- // We will get an Error called userNotFound when using Simple Auth Scheme
- public void testListKeyOnEmptyBucket(int port) throws IOException {
- String bucketName = OzoneUtils.getRequestID().toLowerCase() + "bucket";
- Header.OZONE_SIMPLE_AUTHENTICATION_SCHEME + " "
- + OzoneConsts.OZONE_SIMPLE_HDFS_USER);
- client = createHttpClient();
- httppost = new HttpPost(String
- .format("http://localhost:%d/%s/%s", port, volumeName, bucketName));
- response = client.execute(httppost);
- HttpGet httpget = new HttpGet(String
- response = client.execute(httpget);
- assertEquals(response.toString() + " " + response.getStatusLine()
- .getReasonPhrase(), HTTP_OK,
@@ -16,37 +16,30 @@
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import com.google.common.primitives.Longs;
-import org.apache.commons.lang3.RandomStringUtils;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
-import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
+import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
+import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
-import org.apache.hadoop.ozone.web.handlers.KeyArgs;
-import org.apache.hadoop.ozone.web.handlers.UserArgs;
-import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
-import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
import org.apache.hadoop.utils.MetadataKeyFilters;
import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
-import org.apache.hadoop.ozone.container.common.utils.ReferenceCountedDB;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import com.google.common.primitives.Longs;
* A helper class used by {@link TestStorageContainerManager} to generate
@@ -56,53 +49,32 @@ public class TestStorageContainerManagerHelper {
private final MiniOzoneCluster cluster;
private final Configuration conf;
- private final StorageHandler storageHandler;
public TestStorageContainerManagerHelper(MiniOzoneCluster cluster,
Configuration conf) throws IOException {
this.cluster = cluster;
this.conf = conf;
- storageHandler = new ObjectStoreHandler(conf).getStorageHandler();
public Map<String, OmKeyInfo> createKeys(int numOfKeys, int keySize)
throws Exception {
Map<String, OmKeyInfo> keyLocationMap = Maps.newHashMap();
- String volume = "volume" + RandomStringUtils.randomNumeric(5);
- String bucket = "bucket" + RandomStringUtils.randomNumeric(5);
- String userName = "user" + RandomStringUtils.randomNumeric(5);
- String adminName = "admin" + RandomStringUtils.randomNumeric(5);
- UserArgs userArgs = new UserArgs(null, OzoneUtils.getRequestID(),
- null, null, null, null);
- VolumeArgs createVolumeArgs = new VolumeArgs(volume, userArgs);
- createVolumeArgs.setUserName(userName);
- createVolumeArgs.setAdminName(adminName);
- storageHandler.createVolume(createVolumeArgs);
- BucketArgs bucketArgs = new BucketArgs(bucket, createVolumeArgs);
- bucketArgs.setStorageType(StorageType.DISK);
- storageHandler.createBucket(bucketArgs);
- // Write 20 keys in bucket.
+ OzoneBucket bucket = TestDataUtil.createVolumeAndBucket(cluster);
+ // Write 20 keys in bucketName.
Set<String> keyNames = Sets.newHashSet();
- KeyArgs keyArgs;
for (int i = 0; i < numOfKeys; i++) {
String keyName = RandomStringUtils.randomAlphabetic(5) + i;
keyNames.add(keyName);
- keyArgs = new KeyArgs(keyName, bucketArgs);
- keyArgs.setSize(keySize);
- // Just for testing list keys call, so no need to write real data.
- OutputStream stream = storageHandler.newKeyWriter(keyArgs);
- stream.write(DFSUtil.string2Bytes(
- RandomStringUtils.randomAlphabetic(5)));
- stream.close();
+ TestDataUtil
+ .createKey(bucket, keyName, RandomStringUtils.randomAlphabetic(5));
for (String key : keyNames) {
OmKeyArgs arg = new OmKeyArgs.Builder()
- .setVolumeName(volume)
- .setBucketName(bucket)
+ .setVolumeName(bucket.getVolumeName())
+ .setBucketName(bucket.getName())
.setKeyName(key)
.setRefreshPipeline(true)
.build();
@@ -58,6 +58,7 @@ import org.apache.hadoop.ozone.client.BucketArgs;
import org.apache.hadoop.ozone.client.ObjectStore;
import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClientException;
import org.apache.hadoop.ozone.client.OzoneKey;
import org.apache.hadoop.ozone.client.OzoneKeyDetails;
@@ -67,7 +68,6 @@ import org.apache.hadoop.ozone.client.OzoneVolume;
import org.apache.hadoop.ozone.client.VolumeArgs;
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.common.OzoneChecksumException;
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
import org.apache.hadoop.ozone.container.common.interfaces.Container;
@@ -240,7 +240,7 @@ public abstract class TestOzoneRpcClientAbstract {
@Test
public void testSetVolumeQuota()
- throws IOException, OzoneException {
String volumeName = UUID.randomUUID().toString();
store.createVolume(volumeName);
store.getVolume(volumeName).setQuota(
@@ -264,7 +264,7 @@ public abstract class TestOzoneRpcClientAbstract {
public void testCreateVolumeWithMetadata()
+ throws IOException, OzoneClientException {
VolumeArgs volumeArgs = VolumeArgs.newBuilder()
.addMetadata("key1", "val1")
@@ -278,7 +278,7 @@ public abstract class TestOzoneRpcClientAbstract {
public void testCreateBucketWithMetadata()
long currentTime = Time.now();
String bucketName = UUID.randomUUID().toString();
@@ -297,7 +297,7 @@ public abstract class TestOzoneRpcClientAbstract {
public void testCreateBucket()
@@ -312,7 +312,7 @@ public abstract class TestOzoneRpcClientAbstract {
public void testCreateS3Bucket()
String userName = UserGroupInformation.getCurrentUser().getUserName();
@@ -345,7 +345,7 @@ public abstract class TestOzoneRpcClientAbstract {
public void testListS3Buckets()
String userName = "ozone100";
String bucketName1 = UUID.randomUUID().toString();
String bucketName2 = UUID.randomUUID().toString();
@@ -363,7 +363,7 @@ public abstract class TestOzoneRpcClientAbstract {
public void testListS3BucketsFail()
String userName = "randomUser";
Iterator<? extends OzoneBucket> iterator = store.listS3Buckets(userName,
null);
@@ -402,7 +402,7 @@ public abstract class TestOzoneRpcClientAbstract {
public void testCreateS3BucketMapping()
String userName = "ozone";
@@ -421,7 +421,7 @@ public abstract class TestOzoneRpcClientAbstract {
public void testCreateBucketWithVersioning()
@@ -436,7 +436,7 @@ public abstract class TestOzoneRpcClientAbstract {
public void testCreateBucketWithStorageType()
@@ -451,7 +451,7 @@ public abstract class TestOzoneRpcClientAbstract {
public void testCreateBucketWithAcls()
OzoneAcl userAcl = new OzoneAcl(USER, "test",
@@ -470,7 +470,7 @@ public abstract class TestOzoneRpcClientAbstract {
public void testCreateBucketWithAllArgument()
@@ -507,7 +507,7 @@ public abstract class TestOzoneRpcClientAbstract {
public void testAddBucketAcl()
@@ -526,7 +526,7 @@ public abstract class TestOzoneRpcClientAbstract {
public void testRemoveBucketAcl()
@@ -581,7 +581,7 @@ public abstract class TestOzoneRpcClientAbstract {
public void testSetBucketVersioning()
@@ -618,7 +618,7 @@ public abstract class TestOzoneRpcClientAbstract {
public void testSetBucketStorageType()
@@ -678,7 +678,7 @@ public abstract class TestOzoneRpcClientAbstract {
public void testPutKey()
@@ -746,7 +746,7 @@ public abstract class TestOzoneRpcClientAbstract {
public void testPutKeyRatisOneNode()
@@ -781,7 +781,7 @@ public abstract class TestOzoneRpcClientAbstract {
public void testPutKeyRatisThreeNodes()
@@ -976,7 +976,7 @@ public abstract class TestOzoneRpcClientAbstract {
- public void testGetKeyDetails() throws IOException, OzoneException {
+ public void testGetKeyDetails() throws IOException, OzoneClientException {
@@ -48,7 +48,6 @@ import org.apache.hadoop.ozone.om.OzoneManager;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
-import org.apache.hadoop.ozone.ozShell.TestOzoneShell;
import org.apache.hadoop.ozone.protocol.commands.RetriableDatanodeEventWatcher;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
@@ -97,7 +96,7 @@ public class TestBlockDeletion {
GenericTestUtils.setLogLevel(SCMBlockDeletingService.LOG, Level.DEBUG);
String path =
- GenericTestUtils.getTempPath(TestOzoneShell.class.getSimpleName());
+ GenericTestUtils.getTempPath(TestBlockDeletion.class.getSimpleName());
File baseDir = new File(path);
baseDir.mkdirs();
@@ -17,8 +17,15 @@
package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
@@ -26,32 +33,25 @@ import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE;
import org.junit.Assert;
import org.junit.Test;
* Test to behaviour of the datanode when recieve close container command.
public class TestCloseContainerHandler {
- public void test() throws IOException, TimeoutException, InterruptedException,
- OzoneException {
+ public void test()
+ throws IOException, TimeoutException, InterruptedException {
//setup a cluster (1G free space is enough for a unit test)
OzoneConfiguration conf = new OzoneConfiguration();
@@ -1,208 +0,0 @@
- * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
-package org.apache.hadoop.ozone.om;
-import org.apache.hadoop.conf.StorageUnit;
-import org.apache.hadoop.metrics2.MetricsRecordBuilder;
-import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.scm.ScmConfigKeys;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.rules.ExpectedException;
-import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
-import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
- * Test key write/read where a key can span multiple containers.
-public class TestMultipleContainerReadWrite {
- private static MiniOzoneCluster cluster = null;
- private static StorageHandler storageHandler;
- private static UserArgs userArgs;
- private static OzoneConfiguration conf;
- @Rule
- public ExpectedException exception = ExpectedException.none();
- * Create a MiniDFSCluster for testing.
- * Ozone is made active by setting OZONE_ENABLED = true
- @BeforeClass
- public static void init() throws Exception {
- conf = new OzoneConfiguration();
- conf.setStorageSize(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE, 1,
- StorageUnit.MB);
- conf.setInt(ScmConfigKeys.OZONE_SCM_PIPELINE_OWNER_CONTAINER_COUNT, 5);
- cluster = MiniOzoneCluster.newBuilder(conf).build();
- cluster.waitForClusterToBeReady();
- userArgs = new UserArgs(null, OzoneUtils.getRequestID(),
- * Shutdown MiniDFSCluster.
- @AfterClass
- public static void shutdown() {
- if (cluster != null) {
- cluster.shutdown();
- public void testWriteRead() throws Exception {
- String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
- String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
- String keyName = "key" + RandomStringUtils.randomNumeric(5);
- VolumeArgs createVolumeArgs = new VolumeArgs(volumeName, userArgs);
- BucketArgs bucketArgs = new BucketArgs(bucketName, createVolumeArgs);
- String dataString = RandomStringUtils.randomAscii(3 * (int)OzoneConsts.MB);
- KeyArgs keyArgs = new KeyArgs(volumeName, bucketName, keyName, userArgs);
- keyArgs.setSize(3 * (int)OzoneConsts.MB);
- keyArgs.setUserName(userName);
- try (OutputStream outputStream = storageHandler.newKeyWriter(keyArgs)) {
- outputStream.write(dataString.getBytes());
- byte[] data = new byte[dataString.length()];
- try (InputStream inputStream = storageHandler.newKeyReader(keyArgs)) {
- inputStream.read(data, 0, data.length);
- assertEquals(dataString, new String(data));
- // checking whether container meta data has the chunk file persisted.
- MetricsRecordBuilder containerMetrics = getMetrics(
- "StorageContainerMetrics");
- assertCounter("numWriteChunk", 3L, containerMetrics);
- assertCounter("numReadChunk", 3L, containerMetrics);
- // Disable this test, because this tests assumes writing beyond a specific
- // size is not allowed. Which is not true for now. Keeping this test in case
- // we add this restrict in the future.
- @Ignore
- public void testErrorWrite() throws Exception {
- String dataString1 = RandomStringUtils.randomAscii(100);
- String dataString2 = RandomStringUtils.randomAscii(500);
- keyArgs.setSize(500);
- // first write will write succeed
- outputStream.write(dataString1.getBytes());
- // second write
- exception.expect(IOException.class);
- exception.expectMessage(
- "Can not write 500 bytes with only 400 byte space");
- outputStream.write(dataString2.getBytes());
- public void testPartialRead() throws Exception {
- String dataString = RandomStringUtils.randomAscii(500);
- byte[] data = new byte[600];
- int readLen = inputStream.read(data, 0, 340);
- assertEquals(340, readLen);
- assertEquals(dataString.substring(0, 340),
- new String(data).substring(0, 340));
- readLen = inputStream.read(data, 340, 260);
- assertEquals(160, readLen);
- assertEquals(dataString, new String(data).substring(0, 500));
- readLen = inputStream.read(data, 500, 1);
- assertEquals(-1, readLen);
@@ -19,26 +19,23 @@ package org.apache.hadoop.ozone.om;
import java.util.UUID;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneTestUtils;
+import org.apache.hadoop.ozone.TestDataUtil;
import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.apache.hadoop.ozone.security.acl.IOzoneObj;
import org.apache.hadoop.ozone.security.acl.RequestContext;
import org.apache.commons.lang3.RandomStringUtils;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_AUTHORIZER_CLASS;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_ENABLED;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ADMINISTRATORS;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ADMINISTRATORS_WILDCARD;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS;
import org.junit.AfterClass;
import static org.junit.Assert.assertTrue;
@@ -52,9 +49,8 @@ import org.junit.rules.ExpectedException;
public class TestOmAcls {
+ private static boolean aclAllow = true;
private static MiniOzoneCluster cluster = null;
private static OMMetrics omMetrics;
private static OzoneConfiguration conf;
private static String clusterId;
@@ -80,15 +76,13 @@ public class TestOmAcls {
conf.setInt(OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS, 2);
conf.setClass(OZONE_ACL_AUTHORIZER_CLASS, OzoneAccessAuthorizerTest.class,
IAccessAuthorizer.class);
+ conf.setStrings(OZONE_ADMINISTRATORS, OZONE_ADMINISTRATORS_WILDCARD);
cluster = MiniOzoneCluster.newBuilder(conf)
.setClusterId(clusterId)
.setScmId(scmId)
.setOmId(omId)
cluster.waitForClusterToBeReady();
omMetrics = cluster.getOzoneManager().getMetrics();
logCapturer =
GenericTestUtils.LogCapturer.captureLogs(OzoneManager.getLogger());
@@ -104,65 +98,54 @@ public class TestOmAcls {
* Tests the OM Initialization.
- public void testOMAclsPermissionDenied() throws Exception {
- String user0 = "testListVolumes-user-0";
- String adminUser = "testListVolumes-admin";
- final VolumeArgs createVolumeArgs;
- int i = 100;
- String user0VolName = "Vol-" + user0 + "-" + i;
- createVolumeArgs = new VolumeArgs(user0VolName, userArgs);
- createVolumeArgs.setUserName(user0);
- createVolumeArgs.setAdminName(adminUser);
- createVolumeArgs.setQuota(new OzoneQuota(i, OzoneQuota.Units.GB));
- logCapturer.clearOutput();
- OzoneTestUtils.expectOmException(ResultCodes.PERMISSION_DENIED,
- () -> storageHandler.createVolume(createVolumeArgs));
- assertTrue(logCapturer.getOutput().contains("Only admin users are " +
- "authorized to create Ozone"));
+ public void testBucketCreationPermissionDenied() throws Exception {
- BucketArgs bucketArgs = new BucketArgs("bucket1", createVolumeArgs);
+ TestOmAcls.aclAllow = true;
+ String volumeName = RandomStringUtils.randomAlphabetic(5).toLowerCase();
+ String bucketName = RandomStringUtils.randomAlphabetic(5).toLowerCase();
+ cluster.getClient().getObjectStore().createVolume(volumeName);
+ OzoneVolume volume =
+ cluster.getClient().getObjectStore().getVolume(volumeName);
+ TestOmAcls.aclAllow = false;
OzoneTestUtils.expectOmException(ResultCodes.PERMISSION_DENIED,
- () -> storageHandler.createBucket(bucketArgs));
- assertTrue(logCapturer.getOutput().contains("Only admin users are" +
- " authorized to create Ozone"));
+ () -> volume.createBucket(bucketName));
+ assertTrue(logCapturer.getOutput()
+ .contains("doesn't have CREATE permission to access volume"));
public void testFailureInKeyOp() throws Exception {
final VolumeArgs createVolumeArgs;
- createVolumeArgs = new VolumeArgs(userName, userArgs);
- createVolumeArgs.setQuota(new OzoneQuota(100, OzoneQuota.Units.GB));
logCapturer.clearOutput();
- // write a key without specifying size at all
- String keyName = "testKey";
- KeyArgs keyArgs = new KeyArgs(keyName, bucketArgs);
- () -> storageHandler.newKeyWriter(keyArgs));
+ () -> TestDataUtil.createKey(bucket, "testKey", "testcontent"));
assertTrue(logCapturer.getOutput().contains("doesn't have WRITE " +
"permission to access key"));
- * Test implementation to negative case.
-class OzoneAccessAuthorizerTest implements IAccessAuthorizer {
+ /**
+ * Test implementation to negative case.
+ static class OzoneAccessAuthorizerTest implements IAccessAuthorizer {
- public boolean checkAccess(IOzoneObj ozoneObject, RequestContext context) {
+ @Override
+ public boolean checkAccess(IOzoneObj ozoneObject, RequestContext context) {
+ return TestOmAcls.aclAllow;
@@ -16,50 +16,39 @@
package org.apache.hadoop.ozone.om;
+import java.util.ArrayList;
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
-import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.Assert;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
import org.junit.BeforeClass;
import org.junit.Rule;
-import org.junit.Assert;
import org.junit.rules.ExpectedException;
* This class tests the versioning of blocks from OM side.
public class TestOmBlockVersioning {
private static OzoneManager ozoneManager;
@Rule
public ExpectedException exception = ExpectedException.none();
@@ -76,9 +65,6 @@ public class TestOmBlockVersioning {
conf = new OzoneConfiguration();
cluster = MiniOzoneCluster.newBuilder(conf).build();
ozoneManager = cluster.getOzoneManager();
@@ -101,14 +87,7 @@ public class TestOmBlockVersioning {
String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
String keyName = "key" + RandomStringUtils.randomNumeric(5);
+ TestDataUtil.createVolumeAndBucket(cluster, volumeName, bucketName);
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
.setVolumeName(volumeName)
@@ -200,14 +179,8 @@ public class TestOmBlockVersioning {
+ OzoneBucket bucket =
OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
@@ -218,48 +191,30 @@ public class TestOmBlockVersioning {
String dataString = RandomStringUtils.randomAlphabetic(100);
- // this write will create 1st version with one block
- try (OutputStream stream = storageHandler.newKeyWriter(keyArgs)) {
- stream.write(dataString.getBytes());
- try (InputStream in = storageHandler.newKeyReader(keyArgs)) {
- in.read(data);
+ TestDataUtil.createKey(bucket, keyName, dataString);
+ assertEquals(dataString, TestDataUtil.getKey(bucket, keyName));
OmKeyInfo keyInfo = ozoneManager.lookupKey(omKeyArgs);
- assertEquals(dataString, DFSUtil.bytes2String(data));
assertEquals(0, keyInfo.getLatestVersionLocations().getVersion());
assertEquals(1,
keyInfo.getLatestVersionLocations().getLocationList().size());
// this write will create 2nd version, 2nd version will contain block from
// version 1, and add a new block
- dataString = RandomStringUtils.randomAlphabetic(10);
- data = new byte[dataString.length()];
keyInfo = ozoneManager.lookupKey(omKeyArgs);
assertEquals(1, keyInfo.getLatestVersionLocations().getVersion());
assertEquals(2,
dataString = RandomStringUtils.randomAlphabetic(200);
assertEquals(2, keyInfo.getLatestVersionLocations().getVersion());
assertEquals(3,
@@ -16,14 +16,14 @@
+import java.util.UUID;
import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS;
@@ -31,18 +31,11 @@ import org.junit.Rule;
-import java.util.UUID;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS;
* Test Ozone Manager Init.
public class TestOmInit {
@@ -72,9 +65,6 @@ public class TestOmInit {
@@ -1,1449 +0,0 @@
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.stream.Stream;
-import org.apache.commons.codec.binary.StringUtils;
-import org.apache.hadoop.hdds.HddsConfigKeys;
-import org.apache.hadoop.hdds.scm.ScmInfo;
-import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
-import org.apache.hadoop.ozone.OzoneTestUtils;
-import org.apache.hadoop.ozone.common.BlockGroup;
-import org.apache.hadoop.ozone.om.codec.OmKeyInfoCodec;
-import org.apache.hadoop.ozone.om.exceptions.OMException;
-import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
-import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
-import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
-import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
-import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
-import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
-import org.apache.hadoop.ozone.om.helpers.OzoneAclUtil;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DBUpdatesRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
-import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLIdentityType;
-import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType;
-import org.apache.hadoop.ozone.util.OzoneVersionInfo;
-import org.apache.hadoop.ozone.web.response.KeyInfo;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.utils.db.DBUpdatesWrapper;
-import org.apache.hadoop.utils.db.RDBStore;
-import org.apache.hadoop.utils.db.Table;
-import org.apache.hadoop.utils.db.Table.KeyValue;
-import org.apache.hadoop.utils.db.TableIterator;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY;
-import static org.apache.hadoop.ozone.OzoneAcl.AclScope.ACCESS;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_ENABLED;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ADMINISTRATORS;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ADMINISTRATORS_WILDCARD;
-import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
-import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_FOUND;
-import static org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType.ALL;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.rules.Timeout;
-import org.rocksdb.RocksDB;
- * Test Ozone Manager operation in distributed handler scenario.
-public class TestOzoneManager {
- private MiniOzoneCluster cluster = null;
- private StorageHandler storageHandler;
- private UserArgs userArgs;
- private OMMetrics omMetrics;
- private OzoneConfiguration conf;
- private String clusterId;
- private String scmId;
- private String omId;
- public Timeout timeout = new Timeout(60000);
- @Before
- public void init() throws Exception {
- clusterId = UUID.randomUUID().toString();
- scmId = UUID.randomUUID().toString();
- omId = UUID.randomUUID().toString();
- conf.setBoolean(OZONE_ACL_ENABLED, true);
- conf.setInt(OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS, 2);
- conf.set(OZONE_ADMINISTRATORS, OZONE_ADMINISTRATORS_WILDCARD);
- cluster = MiniOzoneCluster.newBuilder(conf)
- .setClusterId(clusterId)
- .setScmId(scmId)
- .setOmId(omId)
- omMetrics = cluster.getOzoneManager().getMetrics();
- @After
- public void shutdown() {
- // Create a volume and test its attribute after creating them
- public void testCreateVolume() throws IOException, OzoneException {
- long volumeCreateFailCount = omMetrics.getNumVolumeCreateFails();
- VolumeArgs getVolumeArgs = new VolumeArgs(volumeName, userArgs);
- VolumeInfo retVolumeinfo = storageHandler.getVolumeInfo(getVolumeArgs);
- Assert.assertTrue(retVolumeinfo.getVolumeName().equals(volumeName));
- Assert.assertTrue(retVolumeinfo.getOwner().getName().equals(userName));
- Assert.assertEquals(volumeCreateFailCount,
- omMetrics.getNumVolumeCreateFails());
- // Create a volume and modify the volume owner and then test its attributes
- public void testChangeVolumeOwner() throws IOException, OzoneException {
- long volumeInfoFailCount = omMetrics.getNumVolumeInfoFails();
- String newUserName = "user" + RandomStringUtils.randomNumeric(5);
- createVolumeArgs.setUserName(newUserName);
- storageHandler.setVolumeOwner(createVolumeArgs);
- VolumeInfo retVolumeInfo = storageHandler.getVolumeInfo(getVolumeArgs);
- Assert.assertTrue(retVolumeInfo.getVolumeName().equals(volumeName));
- Assert.assertFalse(retVolumeInfo.getOwner().getName().equals(userName));
- Assert.assertTrue(retVolumeInfo.getOwner().getName().equals(newUserName));
- Assert.assertEquals(volumeInfoFailCount,
- omMetrics.getNumVolumeInfoFails());
- public void testChangeVolumeQuota() throws IOException, OzoneException {
- long numVolumeCreateFail = omMetrics.getNumVolumeCreateFails();
- long numVolumeInfoFail = omMetrics.getNumVolumeInfoFails();
- Random rand = new Random();
- // Create a new volume with a quota
- OzoneQuota createQuota =
- new OzoneQuota(rand.nextInt(100), OzoneQuota.Units.GB);
- createVolumeArgs.setQuota(createQuota);
- Assert.assertEquals(createQuota.sizeInBytes(),
- retVolumeInfo.getQuota().sizeInBytes());
- // Set a new quota and test it
- OzoneQuota setQuota =
- createVolumeArgs.setQuota(setQuota);
- storageHandler.setVolumeQuota(createVolumeArgs, false);
- getVolumeArgs = new VolumeArgs(volumeName, userArgs);
- retVolumeInfo = storageHandler.getVolumeInfo(getVolumeArgs);
- Assert.assertEquals(setQuota.sizeInBytes(),
- // Remove the quota and test it again
- storageHandler.setVolumeQuota(createVolumeArgs, true);
- Assert.assertEquals(OzoneConsts.MAX_QUOTA_IN_BYTES,
- Assert.assertEquals(numVolumeCreateFail,
- Assert.assertEquals(numVolumeInfoFail,
- // Create a volume and then delete it and then check for deletion
- public void testDeleteVolume() throws IOException, OzoneException {
- String volumeName1 = volumeName + "_A";
- String volumeName2 = volumeName + "_AA";
- VolumeArgs volumeArgs = null;
- VolumeInfo volumeInfo = null;
- // Create 2 empty volumes with same prefix.
- volumeArgs = new VolumeArgs(volumeName1, userArgs);
- volumeArgs.setUserName(userName);
- volumeArgs.setAdminName(adminName);
- storageHandler.createVolume(volumeArgs);
- volumeArgs = new VolumeArgs(volumeName2, userArgs);
- volumeInfo = storageHandler.getVolumeInfo(volumeArgs);
- Assert.assertTrue(volumeInfo.getVolumeName().equals(volumeName1));
- Assert.assertTrue(volumeInfo.getOwner().getName().equals(userName));
- // Volume with _A should be able to delete as it is empty.
- storageHandler.deleteVolume(volumeArgs);
- // Make sure volume with _AA suffix still exists.
- Assert.assertTrue(volumeInfo.getVolumeName().equals(volumeName2));
- // Make sure volume with _A suffix is successfully deleted.
- storageHandler.getVolumeInfo(volumeArgs);
- Assert.fail("Volume is not deleted");
- } catch (OMException ex) {
- Assert.assertEquals(ResultCodes.VOLUME_NOT_FOUND, ex.getResult());
- //delete the _AA volume, too
- storageHandler.deleteVolume(new VolumeArgs(volumeName2, userArgs));
- //Make sure there is no volume information for the specific user
- OMMetadataManager metadataManager =
- cluster.getOzoneManager().getMetadataManager();
- String userKey = metadataManager.getUserKey(userName);
- VolumeList volumes = metadataManager.getUserTable().get(userKey);
- //that was the last volume of the user, shouldn't be any record here
- Assert.assertNull(volumes);
- // Create a volume and a bucket inside the volume,
- // then delete it and then check for deletion failure
- public void testFailedDeleteVolume() throws IOException, OzoneException {
- long numVolumeCreateFails = omMetrics.getNumVolumeCreateFails();
- Assert.assertTrue(retVolumeInfo.getOwner().getName().equals(userName));
- Assert.assertEquals(numVolumeCreateFails,
- BucketArgs bucketArgs = new BucketArgs(volumeName, bucketName, userArgs);
- storageHandler.deleteVolume(createVolumeArgs);
- Assert.fail("Expecting deletion should fail "
- + "because volume is not empty");
- Assert.assertEquals(ResultCodes.VOLUME_NOT_EMPTY, ex.getResult());
- // Create a volume and test Volume access for a different user
- public void testAccessVolume() throws IOException, OzoneException {
- String userName = UserGroupInformation.getCurrentUser().getUserName();
- String[] groupName =
- {"group" + RandomStringUtils.randomNumeric(5)};
- createVolumeArgs.setGroups(groupName);
- OzoneAcl userAcl = new OzoneAcl(ACLIdentityType.USER, userName,
- ACLType.READ, ACCESS);
- Assert.assertTrue(storageHandler.checkVolumeAccess(volumeName, userAcl));
- OzoneAcl group = new OzoneAcl(ACLIdentityType.GROUP, groupName[0],
- Assert.assertTrue(storageHandler.checkVolumeAccess(volumeName, group));
- // Create a different user and access should fail
- String falseUserName = "user" + RandomStringUtils.randomNumeric(5);
- OzoneAcl falseUserAcl =
- new OzoneAcl(ACLIdentityType.USER, falseUserName,
- ACLType.ALL, ACCESS);
- Assert.assertFalse(storageHandler
- .checkVolumeAccess(volumeName, falseUserAcl));
- // Checking access with user name and Group Type should fail
- OzoneAcl falseGroupAcl = new OzoneAcl(ACLIdentityType.GROUP, userName,
- .checkVolumeAccess(volumeName, falseGroupAcl));
- // Access for acl type world should also fail
- OzoneAcl worldAcl =
- new OzoneAcl(ACLIdentityType.WORLD, "", ACLType.READ, ACCESS);
- Assert.assertFalse(storageHandler.checkVolumeAccess(volumeName, worldAcl));
- Assert.assertEquals(0, omMetrics.getNumVolumeCheckAccessFails());
- Assert.assertEquals(0, omMetrics.getNumVolumeCreateFails());
- public void testCreateBucket() throws IOException, OzoneException {
- long numBucketCreateFail = omMetrics.getNumBucketCreateFails();
- long numBucketInfoFail = omMetrics.getNumBucketInfoFails();
- VolumeArgs volumeArgs = new VolumeArgs(volumeName, userArgs);
- BucketArgs getBucketArgs = new BucketArgs(volumeName, bucketName,
- userArgs);
- BucketInfo bucketInfo = storageHandler.getBucketInfo(getBucketArgs);
- Assert.assertTrue(bucketInfo.getVolumeName().equals(volumeName));
- Assert.assertTrue(bucketInfo.getBucketName().equals(bucketName));
- Assert.assertEquals(numBucketCreateFail,
- omMetrics.getNumBucketCreateFails());
- Assert.assertEquals(numBucketInfoFail,
- omMetrics.getNumBucketInfoFails());
- public void testDeleteBucket() throws Exception {
- storageHandler.deleteBucket(bucketArgs);
- OzoneTestUtils.expectOmException(ResultCodes.BUCKET_NOT_FOUND,
- () -> storageHandler.getBucketInfo(getBucketArgs));
- public void testDeleteNonExistingBucket() throws Exception {
- BucketArgs newBucketArgs = new BucketArgs(
- volumeName, bucketName + "_invalid", userArgs);
- () -> storageHandler.deleteBucket(newBucketArgs));
- public void testDeleteNonEmptyBucket() throws Exception {
- String dataString = RandomStringUtils.randomAscii(100);
- keyArgs.setSize(100);
- OzoneTestUtils.expectOmException(ResultCodes.BUCKET_NOT_EMPTY,
- () -> storageHandler.deleteBucket(bucketArgs));
- * Basic test of both putKey and getKey from OM, as one can not be tested
- * without the other.
- * @throws OzoneException
- public void testGetKeyWriterReader() throws IOException, OzoneException {
- long numKeyAllocates = omMetrics.getNumKeyAllocates();
- long numKeyLookups = omMetrics.getNumKeyLookups();
- Assert.assertEquals(1 + numKeyAllocates, omMetrics.getNumKeyAllocates());
- Assert.assertEquals(dataString, DFSUtil.bytes2String(data));
- Assert.assertEquals(1 + numKeyLookups, omMetrics.getNumKeyLookups());
- * Test write the same key twice, the second write should fail, as currently
- * key overwrite is not supported.
- public void testKeyOverwrite() throws IOException, OzoneException {
- long numKeyAllocateFails = omMetrics.getNumKeyAllocateFails();
- // We allow the key overwrite to be successful. Please note : Till
- // HDFS-11922 is fixed this causes a data block leak on the data node side.
- // That is this overwrite only overwrites the keys on OM. We need to
- // garbage collect those blocks from datanode.
- KeyArgs keyArgs2 = new KeyArgs(volumeName, bucketName, keyName, userArgs);
- keyArgs2.setUserName(userName);
- storageHandler.newKeyWriter(keyArgs2);
- Assert
- .assertEquals(numKeyAllocateFails, omMetrics.getNumKeyAllocateFails());
- * Test get a non-exiting key.
- public void testGetNonExistKey() throws Exception {
- long numKeyLookupFails = omMetrics.getNumKeyLookupFails();
- // try to get the key, should fail as it hasn't been created
- OzoneTestUtils.expectOmException(KEY_NOT_FOUND,
- () -> storageHandler.newKeyReader(keyArgs));
- Assert.assertEquals(1 + numKeyLookupFails,
- omMetrics.getNumKeyLookupFails());
- * Test delete keys for om.
- public void testDeleteKey() throws Exception {
- long numKeyDeletes = omMetrics.getNumKeyDeletes();
- long numKeyDeleteFails = omMetrics.getNumKeyDeletesFails();
- storageHandler.deleteKey(keyArgs);
- Assert.assertEquals(1 + numKeyDeletes, omMetrics.getNumKeyDeletes());
- // Make sure the deleted key has been moved to the deleted table.
- OMMetadataManager manager = cluster.getOzoneManager().
- getMetadataManager();
- try (TableIterator<String, ? extends KeyValue<String, OmKeyInfo>> iter =
- manager.getDeletedTable().iterator()) {
- iter.seekToFirst();
- Table.KeyValue kv = iter.next();
- Assert.assertNotNull(kv);
- // Delete the key again to test deleting non-existing key.
- () -> storageHandler.deleteKey(keyArgs));
- Assert.assertEquals(1 + numKeyDeleteFails,
- omMetrics.getNumKeyDeletesFails());
- * Test rename key for om.
- public void testRenameKey() throws IOException, OzoneException {
- long numKeyRenames = omMetrics.getNumKeyRenames();
- long numKeyRenameFails = omMetrics.getNumKeyRenameFails();
- int testRenameFails = 0;
- int testRenames = 0;
- OMException omException = null;
- String toKeyName = "key" + RandomStringUtils.randomNumeric(5);
- // Rename from non-existent key should fail
- testRenames++;
- storageHandler.renameKey(keyArgs, toKeyName);
- } catch (OMException e) {
- testRenameFails++;
- omException = e;
- Assert.assertEquals(KEY_NOT_FOUND, omException.getResult());
- // Write the contents of the key to be renamed
- // Rename the key
- toKeyName = "key" + RandomStringUtils.randomNumeric(5);
- Assert.assertEquals(numKeyRenames + testRenames,
- omMetrics.getNumKeyRenames());
- Assert.assertEquals(numKeyRenameFails + testRenameFails,
- omMetrics.getNumKeyRenameFails());
- // Try to get the key, should fail as it has been renamed
- storageHandler.newKeyReader(keyArgs);
- // Verify the contents of the renamed key
- keyArgs = new KeyArgs(toKeyName, bucketArgs);
- InputStream in = storageHandler.newKeyReader(keyArgs);
- byte[] b = new byte[dataString.getBytes().length];
- in.read(b);
- Assert.assertEquals(new String(b), dataString);
- // Rewrite the renamed key. Rename to key which already exists should fail.
- dataString = RandomStringUtils.randomAscii(100);
- Assert.assertEquals(ResultCodes.KEY_ALREADY_EXISTS,
- omException.getResult());
- // Rename to empty string should fail
- toKeyName = "";
- Assert.assertEquals(ResultCodes.INVALID_KEY_NAME, omException.getResult());
- // Rename from empty string should fail
- keyArgs = new KeyArgs("", bucketArgs);
- public void testListBuckets() throws IOException, OzoneException {
- ListBuckets result = null;
- ListArgs listBucketArgs = null;
- // Create volume - volA.
- final String volAname = "volA";
- VolumeArgs volAArgs = new VolumeArgs(volAname, userArgs);
- volAArgs.setUserName("userA");
- volAArgs.setAdminName("adminA");
- storageHandler.createVolume(volAArgs);
- // Create 20 buckets in volA for tests.
- for (int i=0; i<10; i++) {
- // Create "/volA/aBucket_0" to "/volA/aBucket_9" buckets in volA volume.
- BucketArgs aBuckets = new BucketArgs(volAname,
- "aBucket_" + i, userArgs);
- if(i % 3 == 0) {
- aBuckets.setStorageType(StorageType.ARCHIVE);
- aBuckets.setStorageType(StorageType.DISK);
- storageHandler.createBucket(aBuckets);
- // Create "/volA/bBucket_0" to "/volA/bBucket_9" buckets in volA volume.
- BucketArgs bBuckets = new BucketArgs(volAname,
- "bBucket_" + i, userArgs);
- bBuckets.setStorageType(StorageType.RAM_DISK);
- bBuckets.setStorageType(StorageType.SSD);
- storageHandler.createBucket(bBuckets);
- VolumeArgs volArgs = new VolumeArgs(volAname, userArgs);
- // List all buckets in volA.
- listBucketArgs = new ListArgs(volArgs, null, 100, null);
- result = storageHandler.listBuckets(listBucketArgs);
- Assert.assertEquals(20, result.getBuckets().size());
- List<BucketInfo> archiveBuckets = result.getBuckets().stream()
- .filter(item -> item.getStorageType() == StorageType.ARCHIVE)
- Assert.assertEquals(4, archiveBuckets.size());
- // List buckets with prefix "aBucket".
- listBucketArgs = new ListArgs(volArgs, "aBucket", 100, null);
- Assert.assertEquals(10, result.getBuckets().size());
- Assert.assertTrue(result.getBuckets().stream()
- .allMatch(entry -> entry.getBucketName().startsWith("aBucket")));
- // List a certain number of buckets.
- listBucketArgs = new ListArgs(volArgs, null, 3, null);
- Assert.assertEquals(3, result.getBuckets().size());
- Assert.assertEquals("aBucket_0",
- result.getBuckets().get(0).getBucketName());
- Assert.assertEquals("aBucket_1",
- result.getBuckets().get(1).getBucketName());
- Assert.assertEquals("aBucket_2",
- result.getBuckets().get(2).getBucketName());
- // List a certain number of buckets from the startKey.
- listBucketArgs = new ListArgs(volArgs, null, 2, "bBucket_3");
- Assert.assertEquals(2, result.getBuckets().size());
- Assert.assertEquals("bBucket_4",
- Assert.assertEquals("bBucket_5",
- // Provide an invalid bucket name as start key.
- listBucketArgs = new ListArgs(volArgs, null, 100, "unknown_bucket_name");
- ListBuckets buckets = storageHandler.listBuckets(listBucketArgs);
- Assert.assertEquals(buckets.getBuckets().size(), 0);
- // Use all arguments.
- listBucketArgs = new ListArgs(volArgs, "b", 5, "bBucket_7");
- Assert.assertEquals("bBucket_8",
- Assert.assertEquals("bBucket_9",
- // Provide an invalid maxKeys argument.
- listBucketArgs = new ListArgs(volArgs, null, -1, null);
- storageHandler.listBuckets(listBucketArgs);
- Assert.fail("Expecting an error when the given"
- + " maxKeys argument is invalid.");
- } catch (Exception e) {
- Assert.assertTrue(e.getMessage()
- .contains(String.format("the value must be in range (0, %d]",
- OzoneConsts.MAX_LISTBUCKETS_SIZE)));
- // Provide an invalid volume name.
- VolumeArgs invalidVolArgs = new VolumeArgs("invalid_name", userArgs);
- listBucketArgs = new ListArgs(invalidVolArgs, null, 100, null);
- Assert.fail("Expecting an error when the given volume name is invalid.");
- Assert.assertEquals(VOLUME_NOT_FOUND, e.getResult());
- * Test list keys.
- public void testListKeys() throws Exception {
- ListKeys result = null;
- ListArgs listKeyArgs = null;
- int numKeys = 20;
- String keyName = "Key";
- KeyArgs keyArgs = null;
- for (int i = 0; i < numKeys; i++) {
- if (i % 2 == 0) {
- // Create /volume/bucket/aKey[0,2,4,...,18] in bucket.
- keyArgs = new KeyArgs("a" + keyName + i, bucketArgs);
- // Create /volume/bucket/bKey[1,3,5,...,19] in bucket.
- keyArgs = new KeyArgs("b" + keyName + i, bucketArgs);
- keyArgs.setSize(4096);
- // List all keys in bucket.
- bucketArgs = new BucketArgs(volumeName, bucketName, userArgs);
- listKeyArgs = new ListArgs(bucketArgs, null, 100, null);
- result = storageHandler.listKeys(listKeyArgs);
- Assert.assertEquals(numKeys, result.getKeyList().size());
- // List keys with prefix "aKey".
- listKeyArgs = new ListArgs(bucketArgs, "aKey", 100, null);
- Assert.assertEquals(numKeys / 2, result.getKeyList().size());
- Assert.assertTrue(result.getKeyList().stream()
- .allMatch(entry -> entry.getKeyName().startsWith("aKey")));
- // List a certain number of keys.
- listKeyArgs = new ListArgs(bucketArgs, null, 3, null);
- Assert.assertEquals(3, result.getKeyList().size());
- Assert.assertEquals("aKey0",
- result.getKeyList().get(0).getKeyName());
- Assert.assertEquals("aKey10",
- result.getKeyList().get(1).getKeyName());
- Assert.assertEquals("aKey12",
- result.getKeyList().get(2).getKeyName());
- // List a certain number of keys from the startKey.
- listKeyArgs = new ListArgs(bucketArgs, null, 2, "bKey1");
- Assert.assertEquals(2, result.getKeyList().size());
- Assert.assertEquals("bKey11",
- Assert.assertEquals("bKey13",
- // Provide an invalid key name as start key.
- listKeyArgs = new ListArgs(bucketArgs, null, 100, "invalid_start_key");
- ListKeys keys = storageHandler.listKeys(listKeyArgs);
- Assert.assertEquals(keys.getKeyList().size(), 0);
- listKeyArgs = new ListArgs(bucketArgs, null, -1, null);
- storageHandler.listBuckets(listKeyArgs);
- GenericTestUtils.assertExceptionContains(
- String.format("the value must be in range (0, %d]",
- OzoneConsts.MAX_LISTKEYS_SIZE), e);
- OzoneTestUtils.expectOmException(ResultCodes.BUCKET_NOT_FOUND, () -> {
- // Provide an invalid bucket name.
- BucketArgs bucket = new BucketArgs("invalid_bucket", createVolumeArgs);
- ListArgs ks = new ListArgs(bucket, null, numKeys, null);
- storageHandler.listKeys(ks);
- public void testListVolumes() throws IOException, OzoneException {
- UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
- String user0 = ugi.getUserName();
- String user1 = "testListVolumes-user-1";
- ListArgs listVolumeArgs;
- ListVolumes volumes;
- // Create 10 volumes by user0 and user1
- String[] user0vols = new String[10];
- String[] user1vols = new String[10];
- for (int i =0; i<10; i++) {
- VolumeArgs createVolumeArgs;
- user0vols[i] = user0VolName;
- String user1VolName = "Vol-" + user1 + "-" + i;
- user1vols[i] = user1VolName;
- createVolumeArgs = new VolumeArgs(user1VolName, userArgs);
- createVolumeArgs.setUserName(user1);
- // Test list all volumes - Removed Support for this operation for time
- // being. TODO: we will need to bring this back if needed.
- UserArgs userArgs0 = new UserArgs(user0, OzoneUtils.getRequestID(),
- //listVolumeArgs = new ListArgs(userArgs0,"Vol-testListVolumes", 100, null);
- // listVolumeArgs.setRootScan(true);
- // volumes = storageHandler.listVolumes(listVolumeArgs);
- // Assert.assertEquals(20, volumes.getVolumes().size());
- // Test list all volumes belongs to an user
- listVolumeArgs = new ListArgs(userArgs0, null, 100, null);
- listVolumeArgs.setRootScan(false);
- volumes = storageHandler.listVolumes(listVolumeArgs);
- Assert.assertEquals(10, volumes.getVolumes().size());
- // Test prefix
- listVolumeArgs = new ListArgs(userArgs0,
- "Vol-" + user0 + "-3", 100, null);
- Assert.assertEquals(1, volumes.getVolumes().size());
- Assert.assertEquals(user0vols[3],
- volumes.getVolumes().get(0).getVolumeName());
- Assert.assertEquals(user0,
- volumes.getVolumes().get(0).getOwner().getName());
- // Test list volumes by user
- UserArgs userArgs1 = new UserArgs(user1, OzoneUtils.getRequestID(),
- listVolumeArgs = new ListArgs(userArgs1, null, 100, null);
- Assert.assertEquals(0, volumes.getVolumes().size());
- // Make sure all available fields are returned
- final String user0vol4 = "Vol-" + user0 + "-4";
- final String user0vol5 = "Vol-" + user0 + "-5";
- listVolumeArgs = new ListArgs(userArgs0, null, 1, user0vol4);
- Assert.assertEquals(user0vol5,
- Assert.assertEquals(5,
- volumes.getVolumes().get(0).getQuota().getSize());
- Assert.assertEquals(OzoneQuota.Units.GB,
- volumes.getVolumes().get(0).getQuota().getUnit());
- // User doesn't have volumes
- UserArgs userArgsX = new UserArgs("unknwonUser", OzoneUtils.getRequestID(),
- listVolumeArgs = new ListArgs(userArgsX, null, 100, null);
- * Test get key information.
- public void testGetKeyInfo() throws IOException,
- OzoneException, ParseException {
- long currentTime = Time.now();
- KeyInfo keyInfo = storageHandler.getKeyInfo(keyArgs);
- // Compare the time in second unit since the date string reparsed to
- // millisecond will lose precision.
- Assert.assertTrue(
- (HddsClientUtils.formatDateTime(keyInfo.getCreatedOn()) / 1000) >= (
- currentTime / 1000));
- (HddsClientUtils.formatDateTime(keyInfo.getModifiedOn()) / 1000) >= (
- Assert.assertEquals(keyName, keyInfo.getKeyName());
- // with out data written, the size would be 0
- Assert.assertEquals(0, keyInfo.getSize());
- * Test that the write can proceed without having to set the right size.
- public void testWriteSize() throws IOException, OzoneException {
- // write a key with a size, but write above it.
- String keyName1 = "testKey1";
- KeyArgs keyArgs1 = new KeyArgs(keyName1, bucketArgs);
- keyArgs1.setSize(30);
- try (OutputStream stream = storageHandler.newKeyWriter(keyArgs1)) {
- byte[] data1 = new byte[dataString.length()];
- try (InputStream in = storageHandler.newKeyReader(keyArgs1)) {
- in.read(data1);
- Assert.assertEquals(dataString, DFSUtil.bytes2String(data1));
- * Tests the RPC call for getting scmId and clusterId from SCM.
- public void testGetScmInfo() throws IOException {
- ScmInfo info = cluster.getOzoneManager().getScmInfo();
- Assert.assertEquals(clusterId, info.getClusterId());
- Assert.assertEquals(scmId, info.getScmId());
- //Disabling this test
- @Ignore("Disabling this test until Open Key is fixed.")
- public void testExpiredOpenKey() throws Exception {
-// BackgroundService openKeyCleanUpService = ((BlockManagerImpl)cluster
-// .getOzoneManager().getBlockManager()).getOpenKeyCleanupService();
- // open some keys.
- KeyArgs keyArgs1 = new KeyArgs("testKey1", bucketArgs);
- KeyArgs keyArgs2 = new KeyArgs("testKey2", bucketArgs);
- KeyArgs keyArgs3 = new KeyArgs("testKey3", bucketArgs);
- KeyArgs keyArgs4 = new KeyArgs("testKey4", bucketArgs);
- List<BlockGroup> openKeys;
- storageHandler.newKeyWriter(keyArgs1);
- storageHandler.newKeyWriter(keyArgs3);
- storageHandler.newKeyWriter(keyArgs4);
- Set<String> expected = Stream.of(
- "testKey1", "testKey2", "testKey3", "testKey4")
- .collect(Collectors.toSet());
- // Now all k1-k4 should be in open state, so ExpiredOpenKeys should not
- // contain these values.
- openKeys = cluster.getOzoneManager()
- .getMetadataManager().getExpiredOpenKeys();
- for (BlockGroup bg : openKeys) {
- String[] subs = bg.getGroupID().split("/");
- String keyName = subs[subs.length - 1];
- Assert.assertFalse(expected.contains(keyName));
- Thread.sleep(2000);
- // Now all k1-k4 should be in ExpiredOpenKeys
- if (expected.contains(keyName)) {
- expected.remove(keyName);
- Assert.assertEquals(0, expected.size());
- KeyArgs keyArgs5 = new KeyArgs("testKey5", bucketArgs);
- storageHandler.newKeyWriter(keyArgs5);
- //openKeyCleanUpService.triggerBackgroundTaskForTesting();
- // now all k1-k4 should have been removed by the clean-up task, only k5
- // should be present in ExpiredOpenKeys.
- openKeys =
- cluster.getOzoneManager().getMetadataManager().getExpiredOpenKeys();
- System.out.println(openKeys);
- boolean key5found = false;
- Set<String> removed = Stream.of(
- Assert.assertFalse(removed.contains(keyName));
- if (keyName.equals("testKey5")) {
- key5found = true;
- Assert.assertTrue(key5found);
- * Tests the OM Initialization.
- public void testOmInitialization() throws IOException {
- // Read the version file info from OM version file
- OMStorage omStorage = cluster.getOzoneManager().getOmStorage();
- SCMStorageConfig scmStorageConfig = new SCMStorageConfig(conf);
- // asserts whether cluster Id and SCM ID are properly set in SCM Version
- // file.
- Assert.assertEquals(clusterId, scmStorageConfig.getClusterID());
- Assert.assertEquals(scmId, scmStorageConfig.getScmId());
- // asserts whether OM Id is properly set in OM Version file.
- Assert.assertEquals(omId, omStorage.getOmId());
- // asserts whether the SCM info is correct in OM Version file.
- Assert.assertEquals(clusterId, omStorage.getClusterID());
- Assert.assertEquals(scmId, omStorage.getScmId());
- * Tests the OM Initialization Failure.
- public void testOmInitializationFailure() throws Exception {
- OzoneConfiguration config = new OzoneConfiguration();
- final String path =
- GenericTestUtils.getTempPath(UUID.randomUUID().toString());
- Path metaDirPath = Paths.get(path, "om-meta");
- config.set(HddsConfigKeys.OZONE_METADATA_DIRS, metaDirPath.toString());
- config.setBoolean(OzoneConfigKeys.OZONE_ENABLED, true);
- config.set(OZONE_OM_ADDRESS_KEY, "127.0.0.1:0");
- config.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "127.0.0.1:0");
- config.set(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY,
- conf.get(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY));
- OzoneTestUtils.expectOmException(ResultCodes.OM_NOT_INITIALIZED, () -> {
- OzoneManager.createOm(config);
- OzoneTestUtils
- .expectOmException(ResultCodes.SCM_VERSION_MISMATCH_ERROR, () -> {
- OMStorage omStore = new OMStorage(config);
- omStore.setClusterId("testClusterId");
- omStore.setScmId("testScmId");
- // writes the version file properties
- omStore.initialize();
- public void testGetServiceList() throws IOException {
- long numGetServiceListCalls = omMetrics.getNumGetServiceLists();
- List<ServiceInfo> services = cluster.getOzoneManager().getServiceList();
- Assert.assertEquals(numGetServiceListCalls + 1,
- omMetrics.getNumGetServiceLists());
- ServiceInfo omInfo = services.stream().filter(
- a -> a.getNodeType().equals(HddsProtos.NodeType.OM))
- .collect(Collectors.toList()).get(0);
- InetSocketAddress omAddress = new InetSocketAddress(omInfo.getHostname(),
- omInfo.getPort(ServicePort.Type.RPC));
- Assert.assertEquals(NetUtils.createSocketAddr(
- conf.get(OZONE_OM_ADDRESS_KEY)), omAddress);
- ServiceInfo scmInfo = services.stream().filter(
- a -> a.getNodeType().equals(HddsProtos.NodeType.SCM))
- InetSocketAddress scmAddress = new InetSocketAddress(scmInfo.getHostname(),
- scmInfo.getPort(ServicePort.Type.RPC));
- conf.get(OZONE_SCM_CLIENT_ADDRESS_KEY)), scmAddress);
- public void testVersion() {
- String expectedVersion = OzoneVersionInfo.OZONE_VERSION_INFO.getVersion();
- String actualVersion = cluster.getOzoneManager().getSoftwareVersion();
- Assert.assertEquals(expectedVersion, actualVersion);
- * Test if OM RocksDB keyMayExist API works for keys that are present.
- * Test added in this module since we need access to custom codec dependent
- * objects like OMKeyInfo.
- * @throws Exception if OM or RocksDB operations fail.
- public void testDBKeyMayExist() throws Exception {
- RDBStore rdbStore = (RDBStore) cluster.getOzoneManager()
- .getMetadataManager().getStore();
- RocksDB db = rdbStore.getDb();
- OmKeyInfo keyInfo = getNewOmKeyInfo();
- OmKeyInfoCodec omKeyInfoCodec = new OmKeyInfoCodec();
- db.put(StringUtils.getBytesUtf16("OMKey1"),
- omKeyInfoCodec.toPersistedFormat(keyInfo));
- StringBuilder sb = new StringBuilder();
- Assert.assertTrue(db.keyMayExist(StringUtils.getBytesUtf16("OMKey1"),
- sb));
- Assert.assertTrue(sb.length() > 0);
- public void testGetOMDBUpdates() throws IOException {
- DBUpdatesRequest dbUpdatesRequest =
- DBUpdatesRequest.newBuilder().setSequenceNumber(0).build();
- DBUpdatesWrapper dbUpdates =
- cluster.getOzoneManager().getDBUpdates(dbUpdatesRequest);
- Assert.assertTrue(dbUpdates.getData().isEmpty());
- //Write data to OM.
- Assert.assertNotNull(keyInfo);
- dbUpdates =
- Assert.assertFalse(dbUpdates.getData().isEmpty());
- private OmKeyInfo getNewOmKeyInfo() throws IOException {
- OmVolumeArgs volumeArgs = OmVolumeArgs.newBuilder()
- .setVolume("vol1")
- .setAdminName("bilbo")
- .setOwnerName("bilbo")
- cluster.getOzoneManager().createVolume(volumeArgs);
- OmBucketInfo bucketInfo = OmBucketInfo.newBuilder()
- .setVolumeName("vol1")
- .setBucketName("bucket1")
- cluster.getOzoneManager().createBucket(bucketInfo);
- OmKeyArgs keyArgs = new OmKeyArgs.Builder()
- .setFactor(HddsProtos.ReplicationFactor.ONE)
- .setDataSize(0)
- .setType(HddsProtos.ReplicationType.STAND_ALONE)
- .setAcls(OzoneAclUtil.getAclList(ugi.getUserName(), ugi.getGroups(),
- ALL, ALL))
- .setKeyName(UUID.randomUUID().toString())
- .setDataSize(16 * 1024 * 1024 * 10)
- OpenKeySession keySession = cluster.getOzoneManager().getKeyManager()
- .openKey(keyArgs);
- return keySession.getKeyInfo();
@@ -18,7 +18,10 @@
import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
@@ -29,25 +32,20 @@ import org.apache.hadoop.ozone.client.OzoneClient;
import org.apache.hadoop.ozone.client.OzoneVolume;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ADMINISTRATORS;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ADMINISTRATORS_WILDCARD;
+import org.junit.After;
import static org.junit.Assert.fail;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
* Test some client operations after cluster starts. And perform restart and
@@ -55,7 +53,6 @@ import static org.junit.Assert.fail;
public class TestOzoneManagerRestart {
private MiniOzoneCluster cluster = null;
private OzoneConfiguration conf;
private String clusterId;
private String scmId;
@@ -86,8 +83,7 @@ public class TestOzoneManagerRestart {
@@ -17,13 +17,10 @@
package org.apache.hadoop.ozone.ozShell;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.PrintStream;
import java.util.Arrays;
-import java.util.Collection;
import org.apache.hadoop.fs.FileUtil;
@@ -31,19 +28,19 @@ import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.ozone.HddsDatanodeService;
-import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import com.google.common.base.Strings;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION;
import org.junit.After;
+import static org.junit.Assert.fail;
import org.junit.Before;
import org.junit.rules.Timeout;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
import picocli.CommandLine;
@@ -53,13 +50,9 @@ import picocli.CommandLine.ParameterException;
import picocli.CommandLine.ParseResult;
import picocli.CommandLine.RunLast;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION;
* This test class specified for testing Ozone datanode shell command.
-@RunWith(value = Parameterized.class)
public class TestOzoneDatanodeShell {
private static final Logger LOG =
@@ -81,17 +74,6 @@ public class TestOzoneDatanodeShell {
private static final PrintStream OLD_OUT = System.out;
private static final PrintStream OLD_ERR = System.err;
- @Parameterized.Parameters
- public static Collection<Object[]> clientProtocol() {
- Object[][] params = new Object[][]{
- {RpcClient.class},
- {RestClient.class}};
- return Arrays.asList(params);
- @Parameterized.Parameter
- @SuppressWarnings("visibilitymodifier")
- public Class clientProtocol;
* Create a MiniDFSCluster for testing with using distributed Ozone
* handler type.
@@ -103,7 +85,7 @@ public class TestOzoneDatanodeShell {
String path = GenericTestUtils.getTempPath(
- TestOzoneShell.class.getSimpleName());
+ TestOzoneDatanodeShell.class.getSimpleName());
baseDir = new File(path);
@@ -1,1266 +0,0 @@
-package org.apache.hadoop.ozone.ozShell;
-import java.io.ByteArrayOutputStream;
-import java.io.EOFException;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.PrintStream;
-import java.nio.file.Files;
-import java.util.Objects;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdds.cli.MissingSubcommandException;
-import org.apache.hadoop.hdds.tracing.StringCodec;
-import org.apache.hadoop.ozone.client.OzoneBucket;
-import org.apache.hadoop.ozone.client.OzoneKey;
-import org.apache.hadoop.ozone.client.OzoneVolume;
-import org.apache.hadoop.ozone.web.ozShell.OzoneShell;
-import org.apache.hadoop.ozone.web.ozShell.Shell;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SECURITY_ENABLED_KEY;
-import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_FOUND;
-import static org.slf4j.event.Level.TRACE;
-import picocli.CommandLine;
-import picocli.CommandLine.ExecutionException;
-import picocli.CommandLine.IExceptionHandler2;
-import picocli.CommandLine.ParameterException;
-import picocli.CommandLine.ParseResult;
-import picocli.CommandLine.RunLast;
- * This test class specified for testing Ozone shell command.
-public class TestOzoneShell {
- LoggerFactory.getLogger(TestOzoneShell.class);
- * Set the timeout for every test.
- public Timeout testTimeout = new Timeout(300000);
- private static String url;
- private static File baseDir;
- private static OzoneConfiguration conf = null;
- private static ClientProtocol client = null;
- private static Shell shell = null;
- private final ByteArrayOutputStream out = new ByteArrayOutputStream();
- private final ByteArrayOutputStream err = new ByteArrayOutputStream();
- private static final PrintStream OLD_OUT = System.out;
- private static final PrintStream OLD_ERR = System.err;
- * Create a MiniDFSCluster for testing with using distributed Ozone
- * handler type.
- * @throws Exception
- String path = GenericTestUtils.getTempPath(
- baseDir = new File(path);
- baseDir.mkdirs();
- shell = new OzoneShell();
- .setNumDatanodes(3)
- conf.setInt(OZONE_REPLICATION, ReplicationFactor.THREE.getValue());
- conf.setQuietMode(false);
- client = new RpcClient(conf);
- * shutdown MiniDFSCluster.
- if (baseDir != null) {
- FileUtil.fullyDelete(baseDir, true);
- public void setup() {
- System.setOut(new PrintStream(out));
- System.setErr(new PrintStream(err));
- url = "o3://" + getOmAddress();
- public void reset() {
- // reset stream after each unit test
- out.reset();
- err.reset();
- // restore system streams
- System.setOut(OLD_OUT);
- System.setErr(OLD_ERR);
- public void testCreateVolume() throws Exception {
- LOG.info("Running testCreateVolume");
- testCreateVolume(volumeName, "");
- volumeName = "volume" + RandomStringUtils.randomNumeric(5);
- testCreateVolume("/////" + volumeName, "");
- testCreateVolume("/////", "Volume name is required");
- testCreateVolume("/////vol/123",
- "Invalid volume name. Delimiters (/) not allowed in volume name");
- private void testCreateVolume(String volumeName, String errorMsg)
- throws Exception {
- String userName = "bilbo";
- String[] args = new String[] {"volume", "create", url + "/" + volumeName,
- "--user", userName, "--root"};
- if (Strings.isNullOrEmpty(errorMsg)) {
- execute(shell, args);
- executeWithError(shell, args, errorMsg);
- return;
- String truncatedVolumeName =
- volumeName.substring(volumeName.lastIndexOf('/') + 1);
- OzoneVolume volumeInfo = client.getVolumeDetails(truncatedVolumeName);
- assertEquals(truncatedVolumeName, volumeInfo.getName());
- assertEquals(userName, volumeInfo.getOwner());
- private void execute(Shell ozoneShell, String[] args) {
- LOG.info("Executing shell command with args {}", Arrays.asList(args));
- CommandLine cmd = ozoneShell.getCmd();
- IExceptionHandler2<List<Object>> exceptionHandler =
- new IExceptionHandler2<List<Object>>() {
- public List<Object> handleParseException(ParameterException ex,
- String[] args) {
- throw ex;
- public List<Object> handleExecutionException(ExecutionException ex,
- ParseResult parseResult) {
- cmd.parseWithHandlers(new RunLast(),
- exceptionHandler, args);
- * Test to create volume without specifying --user or -u.
- public void testCreateVolumeWithoutUser() throws Exception {
- "--root"};
- assertEquals(UserGroupInformation.getCurrentUser().getUserName(),
- volumeInfo.getOwner());
- public void testDeleteVolume() throws Exception {
- LOG.info("Running testDeleteVolume");
- VolumeArgs volumeArgs = VolumeArgs.newBuilder()
- .setOwner("bilbo")
- .setQuota("100TB")
- client.createVolume(volumeName, volumeArgs);
- OzoneVolume volume = client.getVolumeDetails(volumeName);
- assertNotNull(volume);
- String[] args = new String[] {"volume", "delete", url + "/" + volumeName};
- String output = out.toString();
- assertTrue(output.contains("Volume " + volumeName + " is deleted"));
- // verify if volume has been deleted
- client.getVolumeDetails(volumeName);
- fail("Get volume call should have thrown.");
- volumeArgs = VolumeArgs.newBuilder()
- volume = client.getVolumeDetails(volumeName);
- //volumeName prefixed with /
- String volumeNameWithSlashPrefix = "/" + volumeName;
- args = new String[] {"volume", "delete",
- url + "/" + volumeNameWithSlashPrefix};
- output = out.toString();
- public void testInfoVolume() throws Exception {
- LOG.info("Running testInfoVolume");
- //volumeName supplied as-is
- String[] args = new String[] {"volume", "info", url + "/" + volumeName};
- assertTrue(output.contains(volumeName));
- assertTrue(output.contains("createdOn")
- && output.contains(OzoneConsts.OZONE_TIME_ZONE));
- args = new String[] {"volume", "info",
- // test infoVolume with invalid volume name
- url + "/" + volumeName + "/invalid-name"};
- executeWithError(shell, args, "Invalid volume name. " +
- "Delimiters (/) not allowed in volume name");
- // get info for non-exist volume
- args = new String[] {"volume", "info", url + "/invalid-volume"};
- executeWithError(shell, args, VOLUME_NOT_FOUND);
- public void testShellIncompleteCommand() throws Exception {
- LOG.info("Running testShellIncompleteCommand");
- String expectedError = "Incomplete command";
- String[] args = new String[] {}; //executing 'ozone sh'
- executeWithError(shell, args, expectedError);
- args = new String[] {"volume"}; //executing 'ozone sh volume'
- executeWithError(shell, args, MissingSubcommandException.class,
- expectedError);
- args = new String[] {"bucket"}; //executing 'ozone sh bucket'
- args = new String[] {"key"}; //executing 'ozone sh key'
- public void testUpdateVolume() throws Exception {
- LOG.info("Running testUpdateVolume");
- OzoneVolume vol = client.getVolumeDetails(volumeName);
- assertEquals(userName, vol.getOwner());
- assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(), vol.getQuota());
- String[] args = new String[] {"volume", "update", url + "/" + volumeName,
- "--quota", "500MB"};
- vol = client.getVolumeDetails(volumeName);
- assertEquals(OzoneQuota.parseQuota("500MB").sizeInBytes(), vol.getQuota());
- String newUser = "new-user";
- args = new String[] {"volume", "update", url + "/" + volumeName,
- "--user", newUser};
- assertEquals(newUser, vol.getOwner());
- //volume with / prefix
- String volumeWithPrefix = "/" + volumeName;
- String newUser2 = "new-user2";
- args = new String[] {"volume", "update", url + "/" + volumeWithPrefix,
- "--user", newUser2};
- assertEquals(newUser2, vol.getOwner());
- // test error conditions
- args = new String[] {"volume", "update", url + "/invalid-volume",
- executeWithError(shell, args, ResultCodes.VOLUME_NOT_FOUND);
- * Execute command, assert exception message and returns true if error
- * was thrown.
- private void executeWithError(Shell ozoneShell, String[] args,
- OMException.ResultCodes code) {
- execute(ozoneShell, args);
- fail("Exception is expected from command execution " + Arrays
- .asList(args));
- } catch (Exception ex) {
- Assert.assertEquals(OMException.class, ex.getCause().getClass());
- Assert.assertEquals(code, ((OMException) ex.getCause()).getResult());
- String expectedError) {
- if (Strings.isNullOrEmpty(expectedError)) {
- if (!Strings.isNullOrEmpty(expectedError)) {
- Throwable exceptionToCheck = ex;
- if (exceptionToCheck.getCause() != null) {
- exceptionToCheck = exceptionToCheck.getCause();
- String.format(
- "Error of shell code doesn't contain the " +
- "exception [%s] in [%s]",
- expectedError, exceptionToCheck.getMessage()),
- exceptionToCheck.getMessage().contains(expectedError));
- * Execute command, assert exception message and exception class
- * and returns true if error was thrown.
- Class expectedException, String expectedError) {
- "expectedException [%s] in [%s]",
- assertTrue(ex.getClass().getCanonicalName()
- .equals(expectedException.getCanonicalName()));
- * Execute command, assert exception cause message and returns true if error
- Class expectedCause) {
- if (Objects.isNull(expectedCause)) {
- LOG.error("Exception: ", ex);
- assertTrue(ex.getCause().getClass().getCanonicalName()
- .equals(expectedCause.getCanonicalName()));
- public void testListVolume() throws Exception {
- LOG.info("Running testListVolume");
- String protocol = "rpcclient";
- String commandOutput, commandError;
- List<VolumeInfo> volumes;
- final int volCount = 20;
- final String user1 = "test-user-a-" + protocol;
- final String user2 = "test-user-b-" + protocol;
- // Create 20 volumes, 10 for user1 and another 10 for user2.
- for (int x = 0; x < volCount; x++) {
- String volumeName;
- String userName;
- if (x % 2 == 0) {
- // create volume [test-vol0, test-vol2, ..., test-vol18] for user1
- userName = user1;
- volumeName = "test-vol-" + protocol + x;
- // create volume [test-vol1, test-vol3, ..., test-vol19] for user2
- userName = user2;
- .setOwner(userName)
- assertNotNull(vol);
- String[] args = new String[] {"volume", "list", url + "/abcde", "--user",
- user1, "--length", "100"};
- executeWithError(shell, args, "Invalid URI");
- // test -length option
- args = new String[] {"volume", "list", url + "/", "--user",
- commandOutput = out.toString();
- volumes = (List<VolumeInfo>) JsonUtils
- .toJsonList(commandOutput, VolumeInfo.class);
- assertEquals(10, volumes.size());
- for (VolumeInfo volume : volumes) {
- assertEquals(volume.getOwner().getName(), user1);
- assertTrue(volume.getCreatedOn().contains(OzoneConsts.OZONE_TIME_ZONE));
- user1, "--length", "2"};
- assertEquals(2, volumes.size());
- // test --prefix option
- args =
- new String[] {"volume", "list", url + "/", "--user", user1, "--length",
- "100", "--prefix", "test-vol-" + protocol + "1"};
- assertEquals(5, volumes.size());
- // return volume names should be [test-vol10, test-vol12, ..., test-vol18]
- for (int i = 0; i < volumes.size(); i++) {
- assertEquals(volumes.get(i).getVolumeName(),
- "test-vol-" + protocol + ((i + 5) * 2));
- assertEquals(volumes.get(i).getOwner().getName(), user1);
- // test -start option
- new String[] {"volume", "list", url + "/", "--user", user2, "--length",
- "100", "--start", "test-vol-" + protocol + "15"};
- assertEquals(volumes.get(0).getVolumeName(), "test-vol-" + protocol + "17");
- assertEquals(volumes.get(1).getVolumeName(), "test-vol-" + protocol + "19");
- assertEquals(volumes.get(0).getOwner().getName(), user2);
- assertEquals(volumes.get(1).getOwner().getName(), user2);
- user2, "--length", "-1"};
- executeWithError(shell, args, "the length should be a positive number");
- user2, "--length", "invalid-length"};
- executeWithError(shell, args, "Invalid value for option " +
- "'--length': 'invalid-length' is not an int");
- public void testCreateBucket() throws Exception {
- LOG.info("Running testCreateBucket");
- OzoneVolume vol = creatVolume();
- String[] args = new String[] {"bucket", "create",
- url + "/" + vol.getName() + "/" + bucketName};
- OzoneBucket bucketInfo = vol.getBucket(bucketName);
- assertEquals(vol.getName(),
- bucketInfo.getVolumeName());
- assertEquals(bucketName, bucketInfo.getName());
- // test create a bucket in a non-exist volume
- args = new String[] {"bucket", "create",
- url + "/invalid-volume/" + bucketName};
- // test createBucket with invalid bucket name
- url + "/" + vol.getName() + "/" + bucketName + "/invalid-name"};
- executeWithError(shell, args,
- "Invalid bucket name. Delimiters (/) not allowed in bucket name");
- LOG.info("Running testDeleteBucket");
- vol.createBucket(bucketName);
- assertNotNull(bucketInfo);
- String[] args = new String[] {"bucket", "delete",
- // verify if bucket has been deleted in volume
- vol.getBucket(bucketName);
- fail("Get bucket should have thrown.");
- Assert.assertEquals(BUCKET_NOT_FOUND, e.getResult());
- // test delete bucket in a non-exist volume
- args = new String[] {"bucket", "delete",
- url + "/invalid-volume" + "/" + bucketName};
- // test delete non-exist bucket
- url + "/" + vol.getName() + "/invalid-bucket"};
- executeWithError(shell, args, BUCKET_NOT_FOUND);
- public void testInfoBucket() throws Exception {
- LOG.info("Running testInfoBucket");
- String[] args = new String[] {"bucket", "info",
- assertTrue(output.contains(bucketName));
- // test infoBucket with invalid bucket name
- args = new String[] {"bucket", "info",
- // test get info from a non-exist bucket
- url + "/" + vol.getName() + "/invalid-bucket" + bucketName};
- ResultCodes.BUCKET_NOT_FOUND);
- public void testListBucket() throws Exception {
- LOG.info("Running testListBucket");
- List<BucketInfo> buckets;
- String commandOutput;
- int bucketCount = 11;
- List<String> bucketNames = new ArrayList<>();
- // create bucket from test-bucket0 to test-bucket10
- for (int i = 0; i < bucketCount; i++) {
- String name = "test-bucket" + i;
- bucketNames.add(name);
- vol.createBucket(name);
- OzoneBucket bucket = vol.getBucket(name);
- assertNotNull(bucket);
- // test listBucket with invalid volume name
- String[] args = new String[] {"bucket", "list",
- url + "/" + vol.getName() + "/invalid-name"};
- args = new String[] {"bucket", "list",
- url + "/" + vol.getName(), "--length", "100"};
- buckets = (List<BucketInfo>) JsonUtils.toJsonList(commandOutput,
- BucketInfo.class);
- assertEquals(11, buckets.size());
- // sort bucket names since the return buckets isn't in created order
- Collections.sort(bucketNames);
- // return bucket names should be [test-bucket0, test-bucket1,
- // test-bucket10, test-bucket2, ,..., test-bucket9]
- for (int i = 0; i < buckets.size(); i++) {
- assertEquals(buckets.get(i).getBucketName(), bucketNames.get(i));
- assertEquals(buckets.get(i).getVolumeName(), vol.getName());
- assertTrue(buckets.get(i).getCreatedOn()
- .contains(OzoneConsts.OZONE_TIME_ZONE));
- args = new String[] {"bucket", "list", url + "/" + vol.getName(),
- "--length", "3"};
- assertEquals(3, buckets.size());
- // return bucket names should be [test-bucket0,
- // test-bucket1, test-bucket10]
- assertEquals(buckets.get(0).getBucketName(), "test-bucket0");
- assertEquals(buckets.get(1).getBucketName(), "test-bucket1");
- assertEquals(buckets.get(2).getBucketName(), "test-bucket10");
- "--length", "100", "--prefix", "test-bucket1"};
- assertEquals(2, buckets.size());
- // return bucket names should be [test-bucket1, test-bucket10]
- assertEquals(buckets.get(0).getBucketName(), "test-bucket1");
- assertEquals(buckets.get(1).getBucketName(), "test-bucket10");
- "--length", "100", "--start", "test-bucket7"};
- assertEquals(buckets.get(0).getBucketName(), "test-bucket8");
- assertEquals(buckets.get(1).getBucketName(), "test-bucket9");
- "--length", "-1"};
- public void testPutKey() throws Exception {
- LOG.info("Running testPutKey");
- OzoneBucket bucket = creatBucket();
- String volumeName = bucket.getVolumeName();
- String bucketName = bucket.getName();
- String[] args = new String[] {"key", "put",
- url + "/" + volumeName + "/" + bucketName + "/" + keyName,
- createTmpFile()};
- OzoneKey keyInfo = bucket.getKey(keyName);
- assertEquals(keyName, keyInfo.getName());
- // test put key in a non-exist bucket
- args = new String[] {"key", "put",
- url + "/" + volumeName + "/invalid-bucket/" + keyName,
- public void testGetKey() throws Exception {
- GenericTestUtils.LogCapturer logs = GenericTestUtils.LogCapturer
- .captureLogs(StringCodec.LOG);
- GenericTestUtils.setLogLevel(StringCodec.LOG, TRACE);
- LOG.info("Running testGetKey");
- String dataStr = "test-data";
- OzoneOutputStream keyOutputStream =
- bucket.createKey(keyName, dataStr.length());
- keyOutputStream.write(dataStr.getBytes());
- keyOutputStream.close();
- assertFalse("put key without malformed tracing",
- logs.getOutput().contains("MalformedTracerStateString"));
- logs.clearOutput();
- String tmpPath = baseDir.getAbsolutePath() + "/testfile-"
- + UUID.randomUUID().toString();
- String[] args = new String[] {"key", "get",
- tmpPath};
- assertFalse("get key without malformed tracing",
- byte[] dataBytes = new byte[dataStr.length()];
- try (FileInputStream randFile = new FileInputStream(new File(tmpPath))) {
- randFile.read(dataBytes);
- assertEquals(dataStr, DFSUtil.bytes2String(dataBytes));
- tmpPath = baseDir.getAbsolutePath() + File.separatorChar + keyName;
- args = new String[] {"key", "get",
- baseDir.getAbsolutePath()};
- dataBytes = new byte[dataStr.length()];
- LOG.info("Running testDeleteKey");
- String[] args = new String[] {"key", "delete",
- url + "/" + volumeName + "/" + bucketName + "/" + keyName};
- // verify if key has been deleted in the bucket
- assertKeyNotExists(bucket, keyName);
- // test delete key in a non-exist bucket
- args = new String[] {"key", "delete",
- url + "/" + volumeName + "/invalid-bucket/" + keyName};
- // test delete a non-exist key in bucket
- url + "/" + volumeName + "/" + bucketName + "/invalid-key"};
- executeWithError(shell, args, KEY_NOT_FOUND);
- public void testRenameKey() throws Exception {
- LOG.info("Running testRenameKey");
- OzoneKey oldKey = createTestKey(bucket);
- String oldName = oldKey.getName();
- String newName = oldName + ".new";
- String[] args = new String[]{
- "key", "rename",
- String.format("%s/%s/%s",
- url, oldKey.getVolumeName(), oldKey.getBucketName()),
- oldName,
- newName
- OzoneKey newKey = bucket.getKey(newName);
- assertEquals(oldKey.getCreationTime(), newKey.getCreationTime());
- assertEquals(oldKey.getDataSize(), newKey.getDataSize());
- assertKeyNotExists(bucket, oldName);
- public void testInfoKeyDetails() throws Exception {
- LOG.info("Running testInfoKey");
- String[] args = new String[] {"key", "info",
- // verify the response output
- assertTrue(output.contains(keyName));
- assertTrue(
- output.contains("createdOn") && output.contains("modifiedOn") && output
- output.contains("containerID") && output.contains("localID") && output
- .contains("length") && output.contains("offset"));
- // reset stream
- // get the info of a non-exist key
- args = new String[] {"key", "info",
- // get the non-exist key info should be failed
- public void testInfoDirKey() throws Exception {
- LOG.info("Running testInfoKey for Dir Key");
- String dirKeyName = "test/";
- String keyNameOnly = "test";
- bucket.createKey(dirKeyName, dataStr.length());
- url + "/" + volumeName + "/" + bucketName + "/" + dirKeyName};
- assertTrue(output.contains(dirKeyName));
- assertTrue(output.contains("createdOn") &&
- output.contains("modifiedOn") &&
- output.contains(OzoneConsts.OZONE_TIME_ZONE));
- url + "/" + volumeName + "/" + bucketName + "/" + keyNameOnly};
- public void testListKey() throws Exception {
- LOG.info("Running testListKey");
- List<KeyInfo> keys;
- int keyCount = 11;
- String keyName;
- List<String> keyNames = new ArrayList<>();
- for (int i = 0; i < keyCount; i++) {
- keyName = "test-key" + i;
- keyNames.add(keyName);
- // test listKey with invalid bucket name
- String[] args = new String[] {"key", "list",
- url + "/" + volumeName + "/" + bucketName + "/invalid-name"};
- executeWithError(shell, args, "Invalid bucket name. " +
- "Delimiters (/) not allowed in bucket name");
- args = new String[] {"key", "list",
- url + "/" + volumeName + "/" + bucketName, "--length", "100"};
- keys = (List<KeyInfo>) JsonUtils.toJsonList(commandOutput,
- KeyInfo.class);
- assertEquals(11, keys.size());
- // sort key names since the return keys isn't in created order
- Collections.sort(keyNames);
- // return key names should be [test-key0, test-key1,
- // test-key10, test-key2, ,..., test-key9]
- for (int i = 0; i < keys.size(); i++) {
- assertEquals(keys.get(i).getKeyName(), keyNames.get(i));
- // verify the creation/modification time of key
- assertTrue(keys.get(i).getCreatedOn()
- assertTrue(keys.get(i).getModifiedOn()
- String msgText = "Listing first 3 entries of the result. " +
- "Use --length (-l) to override max returned keys.";
- new String[] {"key", "list", url + "/" + volumeName + "/" + bucketName,
- assertTrue("Expecting output to start with " + msgText,
- commandOutput.contains(msgText));
- commandOutput = commandOutput.replace(msgText, "");
- assertEquals(3, keys.size());
- // return key names should be [test-key0, test-key1, test-key10]
- assertEquals(keys.get(0).getKeyName(), "test-key0");
- assertEquals(keys.get(1).getKeyName(), "test-key1");
- assertEquals(keys.get(2).getKeyName(), "test-key10");
- "--length", "100", "--prefix", "test-key1"};
- assertEquals(2, keys.size());
- // return key names should be [test-key1, test-key10]
- assertEquals(keys.get(0).getKeyName(), "test-key1");
- assertEquals(keys.get(1).getKeyName(), "test-key10");
- "--length", "100", "--start", "test-key7"};
- assertEquals(keys.get(0).getKeyName(), "test-key8");
- assertEquals(keys.get(1).getKeyName(), "test-key9");
- private OzoneVolume creatVolume() throws OzoneException, IOException {
- String volumeName = RandomStringUtils.randomNumeric(5) + "volume";
- Assert.assertEquals("PartialGroupNameException",
- ex.getCause().getClass().getSimpleName());
- private OzoneBucket creatBucket() throws OzoneException, IOException {
- String bucketName = RandomStringUtils.randomNumeric(5) + "bucket";
- private OzoneKey createTestKey(OzoneBucket bucket) throws IOException {
- String key = "key" + RandomStringUtils.randomNumeric(5);
- String value = "value";
- bucket.createKey(key, value.length());
- keyOutputStream.write(value.getBytes());
- return bucket.getKey(key);
- public void testTokenCommands() throws Exception {
- String omAdd = "--set=" + OZONE_OM_ADDRESS_KEY + "=" + getOmAddress();
- List<String[]> shellCommands = new ArrayList<>(4);
- // Case 1: Execution will fail when security is disabled.
- shellCommands.add(new String[]{omAdd, "token", "get"});
- shellCommands.add(new String[]{omAdd, "token", "renew"});
- shellCommands.add(new String[]{omAdd, "token", "cancel"});
- shellCommands.add(new String[]{omAdd, "token", "print"});
- shellCommands.forEach(cmd -> execute(cmd, "Error:Token operations " +
- "work only"));
- String security = "-D=" + OZONE_SECURITY_ENABLED_KEY + "=true";
- // Case 2: Execution of get token will fail when security is enabled but
- // OzoneManager is not setup correctly.
- execute(new String[]{omAdd, security,
- "token", "get"}, "Error: Get delegation token operation failed.");
- // Clear all commands.
- shellCommands.clear();
- // Case 3: Execution of renew/cancel/print token will fail as token file
- // doesn't exist.
- shellCommands.add(new String[]{omAdd, security, "token", "renew"});
- shellCommands.add(new String[]{omAdd, security, "token", "cancel"});
- shellCommands.add(new String[]{omAdd, security, "token", "print"});
- shellCommands.forEach(cmd -> execute(cmd, "token " +
- "operation failed as token file:"));
- // Create corrupt token file.
- File testPath = GenericTestUtils.getTestDir();
- Files.createDirectories(testPath.toPath());
- Path tokenFile = Paths.get(testPath.toString(), "token.txt");
- String question = RandomStringUtils.random(100);
- Files.write(tokenFile, question.getBytes());
- String file = "-t=" + tokenFile.toString();
- // Case 4: Execution of renew/cancel/print token will fail if token file
- // is corrupt.
- shellCommands.add(new String[]{omAdd, security, "token", "renew", file});
- shellCommands.add(new String[]{omAdd, security, "token",
- "cancel", file});
- shellCommands.add(new String[]{omAdd, security, "token", "print", file});
- shellCommands.forEach(cmd -> executeWithError(shell, cmd,
- EOFException.class));
- private void execute(String[] cmd, String msg) {
- execute(shell, cmd);
- String output = err.toString();
- assertTrue(output.contains(msg));
- * Create a temporary file used for putting key.
- * @return the created file's path string
- private String createTmpFile() throws Exception {
- // write a new file that used for putting key
- File tmpFile = new File(baseDir,
- "/testfile-" + UUID.randomUUID().toString());
- FileOutputStream randFile = new FileOutputStream(tmpFile);
- Random r = new Random();
- for (int x = 0; x < 10; x++) {
- char c = (char) (r.nextInt(26) + 'a');
- randFile.write(c);
- randFile.close();
- return tmpFile.getAbsolutePath();
- private String getOmAddress() {
- List<ServiceInfo> services;
- services = cluster.getOzoneManager().getServiceList();
- } catch (IOException e) {
- fail("Could not get service list from OM");
- return services.stream()
- .filter(a -> HddsProtos.NodeType.OM.equals(a.getNodeType()))
- .findFirst()
- .map(s -> s.getServiceAddress(ServicePort.Type.RPC))
- .orElseThrow(IllegalStateException::new);
- private static void assertKeyNotExists(OzoneBucket bucket, String keyName)
- bucket.getKey(keyName);
- fail(String.format("Key %s should not exist, but it does", keyName));
- Assert.assertEquals(KEY_NOT_FOUND, e.getResult());
@@ -1,221 +0,0 @@
-import static com.google.common.base.Charsets.UTF_8;
-import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
-import static org.apache.hadoop.ozone.OzoneConsts.CHUNK_SIZE;
-import static org.junit.Assert.*;
-import org.apache.commons.io.IOUtils;
- * End-to-end testing of Ozone REST operations.
-public class TestOzoneRestWithMiniCluster {
- private static MiniOzoneCluster cluster;
- private static ClientProtocol client;
- private static ReplicationFactor replicationFactor = ReplicationFactor.ONE;
- private static ReplicationType replicationType = ReplicationType.RATIS;
- public static void shutdown() throws InterruptedException, IOException {
- public void testCreateAndGetVolume() throws Exception {
- createAndGetVolume();
- public void testCreateAndGetBucket() throws Exception {
- OzoneVolume volume = createAndGetVolume();
- createAndGetBucket(volume);
- public void testPutAndGetKey() throws Exception {
- String keyName = nextId("key");
- String keyData = nextId("data");
- OzoneBucket bucket = createAndGetBucket(volume);
- putKey(bucket, keyName, keyData);
- private void putKey(OzoneBucket bucket, String keyName, String keyData)
- try (
- OzoneOutputStream ozoneOutputStream = bucket
- .createKey(keyName, 0, replicationType, replicationFactor,
- InputStream inputStream = IOUtils.toInputStream(keyData, UTF_8)) {
- IOUtils.copy(inputStream, ozoneOutputStream);
- InputStream inputStream = IOUtils.toInputStream(keyData, UTF_8);
- OzoneInputStream ozoneInputStream = bucket.readKey(keyName)) {
- IOUtils.contentEquals(ozoneInputStream, inputStream);
- public void testPutAndGetEmptyKey() throws Exception {
- String keyData = "";
- public void testPutAndGetMultiChunkKey() throws Exception {
- int keyDataLen = 3 * CHUNK_SIZE;
- String keyData = buildKeyData(keyDataLen);
- public void testPutAndGetMultiChunkKeyLastChunkPartial() throws Exception {
- int keyDataLen = (int)(2.5 * CHUNK_SIZE);
- public void testReplaceKey() throws Exception {
- // Replace key with data consisting of fewer chunks.
- keyDataLen = (int)(1.5 * CHUNK_SIZE);
- keyData = buildKeyData(keyDataLen);
- // Replace key with data consisting of more chunks.
- keyDataLen = (int)(3.5 * CHUNK_SIZE);
- private OzoneVolume createAndGetVolume() throws IOException {
- String volumeName = nextId("volume");
- .setAdmin("hdfs")
- assertEquals(volumeName, volume.getName());
- assertEquals("bilbo", volume.getOwner());
- assertNotNull(volume.getQuota());
- assertEquals(OzoneQuota.parseQuota("100TB").sizeInBytes(),
- volume.getQuota());
- private OzoneBucket createAndGetBucket(OzoneVolume vol) throws IOException {
- String bucketName = nextId("bucket");
- OzoneBucket bucket = vol.getBucket(bucketName);
- assertEquals(bucketName, bucket.getName());
- * Creates sample key data of the specified length. The data is a string of
- * printable ASCII characters. This makes it easy to debug through visual
- * inspection of the chunk files if a test fails.
- * @param keyDataLen desired length of key data
- * @return string of printable ASCII characters of the specified length
- private static String buildKeyData(int keyDataLen) {
- return new String(dataset(keyDataLen, 33, 93), UTF_8);
- * Generates identifiers unique enough for use in tests, so that individual
- * tests don't collide on each others' data in the shared mini-cluster.
- * @param idPrefix prefix to put in front of ID
- * @return unique ID generated by appending a suffix to the given prefix
- private static String nextId(String idPrefix) {
- return (idPrefix + RandomStringUtils.random(5, true, true)).toLowerCase();
@@ -1,187 +0,0 @@
-import org.apache.hadoop.ozone.TestOzoneHelper;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
- * Test ozone volume in the distributed storage handler scenario.
-public class TestOzoneVolumes extends TestOzoneHelper {
- private static final org.slf4j.Logger LOG =
- LoggerFactory.getLogger(TestOzoneVolumes.class);
- private static int port = 0;
- OzoneConfiguration conf = new OzoneConfiguration();
- Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
- port = cluster.getHddsDatanodes().get(0)
- .getDatanodeDetails()
- .getPort(DatanodeDetails.Port.Name.REST).getValue();
- public void testCreateVolumes() throws IOException {
- super.testCreateVolumes(port);
- Assert.assertEquals(0, cluster.getOzoneManager()
- .getMetrics().getNumVolumeCreateFails());
- public void testCreateVolumesWithQuota() throws IOException {
- super.testCreateVolumesWithQuota(port);
- public void testCreateVolumesWithInvalidQuota() throws IOException {
- super.testCreateVolumesWithInvalidQuota(port);
- public void testCreateVolumesWithInvalidUser() throws IOException {
- super.testCreateVolumesWithInvalidUser(port);
- public void testCreateVolumesWithOutAdminRights() throws IOException {
- super.testCreateVolumesWithOutAdminRights(port);
- public void testCreateVolumesInLoop() throws IOException {
- super.testCreateVolumesInLoop(port);
- @Ignore("Test is ignored for time being, to be enabled after security.")
- public void testGetVolumesByUser() throws IOException {
- testGetVolumesByUser(port);
- public void testGetVolumesOfAnotherUser() throws IOException {
- super.testGetVolumesOfAnotherUser(port);
- public void testGetVolumesOfAnotherUserShouldFail() throws IOException {
- super.testGetVolumesOfAnotherUserShouldFail(port);
- * Test Ozone Access through REST protocol.
-public class TestOzoneWebAccess {
- private static int port;
- .getDatanodeDetails().getPort(
- DatanodeDetails.Port.Name.REST).getValue();
- * shutdown MiniOzoneCluster.
- * Send a vaild Ozone Request.
- public void testOzoneRequest() throws IOException {
- CloseableHttpClient client = HttpClients.createDefault();
- String volumeName = getRequestID().toLowerCase(Locale.US);
- format.format(new Date(Time.now())));
@@ -1,349 +0,0 @@
-package org.apache.hadoop.ozone.web.client;
-import org.apache.hadoop.ozone.client.BucketArgs;
-import java.util.Iterator;
-import static org.junit.Assume.assumeFalse;
- * Test Ozone Bucket Lifecycle.
-public class TestBuckets {
- Object[][] params = new Object[][] {
- public static Class clientProtocol;
- public static void init()
- throws IOException, URISyntaxException, OzoneException, TimeoutException,
- InterruptedException {
- public void setup() throws Exception {
- if (clientProtocol.equals(RestClient.class)) {
- client = new RestClient(conf);
- runTestCreateBucket(client);
- static void runTestCreateBucket(ClientProtocol protocol)
- protocol.createVolume(volumeName, volumeArgs);
- OzoneVolume vol = protocol.getVolumeDetails(volumeName);
- String[] acls = {"user:frodo:rw", "user:samwise:rw"};
- // create 10 buckets under same volume
- String bucketName = OzoneUtils.getRequestID().toLowerCase();
- List<OzoneAcl> aclList =
- Arrays.stream(acls).map(acl -> OzoneAcl.parseAcl(acl))
- BucketArgs bucketArgs = BucketArgs.newBuilder()
- .setAcls(aclList)
- vol.createBucket(bucketName, bucketArgs);
- assertEquals(bucket.getName(), bucketName);
- // verify the bucket creation time
- assertTrue((bucket.getCreationTime() / 1000) >= (currentTime / 1000));
- protocol.close();
- assertEquals(vol.getName(), volumeName);
- assertEquals(vol.getAdmin(), "hdfs");
- assertEquals(vol.getOwner(), "bilbo");
- assertEquals(vol.getQuota(), OzoneQuota.parseQuota("100TB").sizeInBytes());
- // Test create a bucket with invalid bucket name,
- // not use Rule here because the test method is static.
- String invalidBucketName = "#" + OzoneUtils.getRequestID().toLowerCase();
- vol.createBucket(invalidBucketName);
- fail("Except the bucket creation to be failed because the"
- + " bucket name starts with an invalid char #");
- assertTrue(e.getMessage()
- .contains("Bucket or Volume name has an unsupported character : #"));
- public void testAddBucketAcls() throws Exception {
- assumeFalse("Rest Client does not support ACL",
- clientProtocol.equals(RestClient.class));
- runTestAddBucketAcls(client);
- static void runTestAddBucketAcls(ClientProtocol protocol)
- throws OzoneException, IOException, ParseException {
- int numAcls = bucket.getAcls().size();
- for (OzoneAcl ozoneAcl : aclList) {
- Assert.assertTrue(bucket.addAcls(ozoneAcl));
- OzoneBucket updatedBucket = vol.getBucket(bucketName);
- assertEquals(updatedBucket.getAcls().size(), 2 + numAcls);
- // verify if the creation time is missing after update operation
- (updatedBucket.getCreationTime()) / 1000 >= 0);
- public void testRemoveBucketAcls() throws Exception {
- runTestRemoveBucketAcls(client);
- static void runTestRemoveBucketAcls(ClientProtocol protocol)
- assertEquals(bucket.getAcls().size(), 2 + numAcls);
- Assert.assertTrue(bucket.removeAcls(ozoneAcl));
- // We removed all acls
- assertEquals(updatedBucket.getAcls().size(), numAcls);
- (updatedBucket.getCreationTime() / 1000) >= 0);
- public void testDeleteBucket() throws OzoneException, IOException {
- runTestDeleteBucket(client);
- static void runTestDeleteBucket(ClientProtocol protocol)
- throws OzoneException, IOException {
- vol.deleteBucket(bucketName);
- fail("Fetching deleted bucket, Should not reach here.");
- // must throw
- assertNotNull(ex);
- runTestListBucket(client);
- static void runTestListBucket(ClientProtocol protocol)
- String bucketName = "listbucket-test-" + x;
- Iterator<? extends OzoneBucket> bucketIterator = vol.listBuckets(null);
- int count = 0;
- while (bucketIterator.hasNext()) {
- assertTrue((bucketIterator.next().getCreationTime()
- / 1000) >= (currentTime / 1000));
- count++;
- assertEquals(count, 10);
- bucketIterator = vol.listBuckets(null, "listbucket-test-4");
- assertEquals(getSize(bucketIterator), 5);
- bucketIterator = vol.listBuckets(null, "listbucket-test-3");
- assertEquals(getSize(bucketIterator), 6);
- private static int getSize(Iterator<? extends OzoneBucket> bucketIterator) {
- bucketIterator.next();
- return count;
@@ -1,108 +0,0 @@
-import org.apache.hadoop.ozone.RatisTestHelper;
-/** The same as {@link TestBuckets} except that this test is Ratis enabled. */
-@Ignore("Disabling Ratis tests for pipeline work.")
-public class TestBucketsRatis {
- private static RatisTestHelper.RatisTestSuite suite;
- suite = new RatisTestHelper.RatisTestSuite();
- conf = suite.getConf();
- if (suite != null) {
- suite.close();
- TestBuckets.runTestCreateBucket(client);
- TestBuckets.runTestAddBucketAcls(client);
- TestBuckets.runTestRemoveBucketAcls(client);
- TestBuckets.runTestDeleteBucket(client);
- TestBuckets.runTestListBucket(client);
@@ -1,734 +0,0 @@
-import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.commons.collections.IteratorUtils;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.RandomUtils;
-import org.apache.commons.lang3.tuple.ImmutablePair;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.ozone.container.common.helpers.BlockData;
-import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
-import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
-import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
-import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
-import org.apache.hadoop.ozone.om.OzoneManager;
-import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
-import org.apache.hadoop.util.StringUtils;
-import static org.apache.hadoop.hdds
- .HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys
- .OZONE_SCM_STALENODE_INTERVAL;
- * Test Ozone Key Lifecycle.
-public class TestKeys {
- private static MiniOzoneCluster ozoneCluster = null;
- private static String path;
- private static long currentTime;
- private static ReplicationType replicationType = ReplicationType.STAND_ALONE;
- // Set short block deleting service interval to speed up deletions.
- conf.setTimeDuration(OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL,
- 1000, TimeUnit.MILLISECONDS);
- conf.setTimeDuration(HDDS_CONTAINER_REPORT_INTERVAL, 1, TimeUnit.SECONDS);
- conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 30, TimeUnit.SECONDS);
- path = GenericTestUtils.getTempPath(TestKeys.class.getSimpleName());
- ozoneCluster = MiniOzoneCluster.newBuilder(conf)
- .setNumDatanodes(1)
- .setHbInterval(1000)
- .setHbProcessorInterval(1000)
- ozoneCluster.waitForClusterToBeReady();
- currentTime = Time.now();
- if (ozoneCluster != null) {
- ozoneCluster.shutdown();
- * Creates a file with Random Data.
- * @return File.
- static File createRandomDataFile(String dir, String fileName, long size)
- File tmpDir = new File(dir);
- FileUtils.forceMkdir(tmpDir);
- File tmpFile = new File(tmpDir, fileName);
- try (FileOutputStream randFile = new FileOutputStream(tmpFile)) {
- for (int x = 0; x < size; x++) {
- fail(e.getMessage());
- return tmpFile;
- * This function generates multi part key which are delimited by a certain
- * delimiter. Different parts of key are random string of random length
- * between 0 - 4. Number of parts of the keys are between 0 and 5.
- * @param delimiter delimiter used to delimit parts of string
- * @return Key composed of multiple parts delimited by "/"
- static String getMultiPartKey(String delimiter) {
- int numParts = RandomUtils.nextInt(0, 5) + 1;
- String[] nameParts = new String[numParts];
- for (int i = 0; i < numParts; i++) {
- int stringLength = numParts == 1 ? 5 : RandomUtils.nextInt(0, 5);
- nameParts[i] = RandomStringUtils.randomAlphanumeric(stringLength);
- return StringUtils.join(delimiter, nameParts);
- static class PutHelper {
- private final ClientProtocol client;
- private final String dir;
- private final String keyName;
- private OzoneVolume vol;
- private OzoneBucket bucket;
- private File file;
- PutHelper(ClientProtocol client, String dir) {
- this(client, dir, OzoneUtils.getRequestID().toLowerCase());
- PutHelper(ClientProtocol client, String dir, String key) {
- this.client = client;
- this.dir = dir;
- this.keyName = key;
- public OzoneVolume getVol() {
- return vol;
- public OzoneBucket getBucket() {
- public File getFile() {
- return file;
- * This function is reused in all other tests.
- * @return Returns the name of the new key that was created.
- private String putKey() throws Exception {
- bucket = vol.getBucket(bucketName);
- String fileName = OzoneUtils.getRequestID().toLowerCase();
- file = createRandomDataFile(dir, fileName, 1024);
- InputStream fileInputStream = new FileInputStream(file)) {
- IOUtils.copy(fileInputStream, ozoneOutputStream);
- // Test non-delimited keys
- runTestPutKey(new PutHelper(client, path));
- // Test key delimited by a random delimiter
- String delimiter = RandomStringUtils.randomAscii(1);
- runTestPutKey(new PutHelper(client, path,
- getMultiPartKey(delimiter)));
- @SuppressWarnings("emptyblock")
- static void runTestPutKey(PutHelper helper) throws Exception {
- final ClientProtocol helperClient = helper.client;
- helper.putKey();
- assertNotNull(helper.getBucket());
- assertNotNull(helper.getFile());
- List<OzoneKey> keyList = helperClient
- .listKeys(helper.getVol().getName(), helper.getBucket().getName(), null,
- null, 10);
- Assert.assertEquals(1, keyList.size());
- // test list key using a more efficient call
- String newkeyName = OzoneUtils.getRequestID().toLowerCase();
- OzoneOutputStream ozoneOutputStream = helperClient
- .createKey(helper.getVol().getName(), helper.getBucket().getName(),
- newkeyName, 0, replicationType, replicationFactor, new HashMap<>());
- ozoneOutputStream.close();
- keyList = helperClient
- Assert.assertEquals(2, keyList.size());
- // test new put key with invalid volume/bucket name
- OzoneTestUtils.expectOmException(ResultCodes.VOLUME_NOT_FOUND, () -> {
- try (OzoneOutputStream oos = helperClient
- .createKey("invalid-volume", helper.getBucket().getName(), newkeyName,
- 0, replicationType, replicationFactor, new HashMap<>())) {
- .createKey(helper.getVol().getName(), "invalid-bucket", newkeyName, 0,
- replicationType, replicationFactor, new HashMap<>())) {
- private static void restartDatanode(MiniOzoneCluster cluster, int datanodeIdx)
- cluster.restartHddsDatanode(datanodeIdx, true);
- public void testPutAndGetKeyWithDnRestart() throws Exception {
- runTestPutAndGetKeyWithDnRestart(
- new PutHelper(client, path), ozoneCluster);
- new PutHelper(client, path,
- getMultiPartKey(delimiter)), ozoneCluster);
- static void runTestPutAndGetKeyWithDnRestart(
- PutHelper helper, MiniOzoneCluster cluster) throws Exception {
- String keyName = helper.putKey();
- // restart the datanode
- restartDatanode(cluster, 0);
- // verify getKey after the datanode restart
- String newFileName = helper.dir + "/"
- + OzoneUtils.getRequestID().toLowerCase();
- Path newPath = Paths.get(newFileName);
- FileOutputStream newOutputStream = new FileOutputStream(
- newPath.toString());
- OzoneInputStream ozoneInputStream = helper.client
- .getKey(helper.getVol().getName(), helper.getBucket().getName(),
- keyName)) {
- IOUtils.copy(ozoneInputStream, newOutputStream);
- FileInputStream original = new FileInputStream(helper.getFile());
- FileInputStream downloaded = new FileInputStream(newPath.toFile())) {
- String originalHash = DigestUtils.sha256Hex(original);
- String downloadedHash = DigestUtils.sha256Hex(downloaded);
- assertEquals(
- "Sha256 does not match between original file and downloaded file.",
- originalHash, downloadedHash);
- runTestPutAndGetKey(new PutHelper(client, path));
- runTestPutAndGetKey(new PutHelper(client, path,
- static void runTestPutAndGetKey(PutHelper helper) throws Exception {
- final String newFileName1 = helper.dir + "/"
- final String newFileName2 = helper.dir + "/"
- Path newPath1 = Paths.get(newFileName1);
- Path newPath2 = Paths.get(newFileName2);
- newPath1.toString());
- OzoneInputStream ozoneInputStream = helper.getBucket()
- .readKey(keyName)) {
- // test get key using a more efficient call
- newPath2.toString());
- try (FileInputStream original = new FileInputStream(helper.getFile());
- FileInputStream downloaded1 = new FileInputStream(newPath1.toFile());
- FileInputStream downloaded2 = new FileInputStream(newPath1.toFile())) {
- String downloadedHash1 = DigestUtils.sha256Hex(downloaded1);
- String downloadedHash2 = DigestUtils.sha256Hex(downloaded2);
- originalHash, downloadedHash1);
- originalHash, downloadedHash2);
- // test new get key with invalid volume/bucket name
- OzoneTestUtils.expectOmException(ResultCodes.KEY_NOT_FOUND,
- () -> helperClient.getKey(
- "invalid-volume", helper.getBucket().getName(), keyName));
- helper.getVol().getName(), "invalid-bucket", keyName));
- public void testPutAndDeleteKey() throws Exception {
- runTestPutAndDeleteKey(new PutHelper(client, path));
- runTestPutAndDeleteKey(new PutHelper(client, path,
- static void runTestPutAndDeleteKey(PutHelper helper) throws Exception {
- helper.getBucket().deleteKey(keyName);
- OzoneTestUtils.expectOmException(ResultCodes.KEY_NOT_FOUND, () -> {
- helper.getBucket().getKey(keyName);
- public void testPutAndListKey() throws Exception {
- runTestPutAndListKey(new PutHelper(client, path));
- runTestPutAndListKey(new PutHelper(client, path,
- static void runTestPutAndListKey(PutHelper helper) throws Exception {
- ClientProtocol helperClient = helper.client;
- // add keys [list-key0, list-key1, ..., list-key9]
- String newkeyName = "list-key" + x;
- OzoneOutputStream ozoneOutputStream = helper.getBucket()
- .createKey(newkeyName, 0, replicationType, replicationFactor,
- InputStream fileInputStream = new FileInputStream(helper.getFile())) {
- List<OzoneKey> keyList1 =
- IteratorUtils.toList(helper.getBucket().listKeys(null, null));
- List<OzoneKey> keyList2 = helperClient
- null, 100);
- Assert.assertEquals(11, keyList1.size());
- Assert.assertEquals(11, keyList2.size());
- // Verify the key creation/modification time. Here we compare the time in
- // second unit since the date string reparsed to millisecond will
- // lose precision.
- for (OzoneKey key : keyList1) {
- assertTrue((key.getCreationTime() / 1000) >= (currentTime / 1000));
- assertTrue((key.getModificationTime() / 1000) >= (currentTime / 1000));
- for (OzoneKey key : keyList2) {
- // test maxLength parameter of list keys
- keyList2 = helperClient
- null, 1);
- Assert.assertEquals(1, keyList2.size());
- // test startKey parameter of list keys
- keyList1 = IteratorUtils
- .toList(helper.getBucket().listKeys("list-key", "list-key4"));
- .listKeys(helper.getVol().getName(), helper.getBucket().getName(),
- "list-key", "list-key4", 100);
- Assert.assertEquals(5, keyList1.size());
- Assert.assertEquals(5, keyList2.size());
- // test prefix parameter of list keys
- keyList1 =
- IteratorUtils.toList(helper.getBucket().listKeys("list-key2", null));
- "list-key2", null, 100);
- keyList1.size() == 1 && keyList1.get(0).getName().equals("list-key2"));
- keyList2.size() == 1 && keyList2.get(0).getName().equals("list-key2"));
- // test new list keys with invalid volume/bucket name
- helperClient.listKeys("invalid-volume", helper.getBucket().getName(),
- null, null, 100);
- helperClient.listKeys(helper.getVol().getName(), "invalid-bucket", null,
- public void testGetKeyInfo() throws Exception {
- runTestGetKeyInfo(new PutHelper(client, path));
- runTestGetKeyInfo(new PutHelper(client, path,
- static void runTestGetKeyInfo(PutHelper helper) throws Exception {
- OzoneKey keyInfo = helper.getBucket().getKey(keyName);
- assertNotNull(keyInfo);
- .assertTrue((keyInfo.getCreationTime() / 1000) >= (currentTime / 1000));
- (keyInfo.getModificationTime() / 1000) >= (currentTime / 1000));
- // Volume, bucket, keys info that helps for test create/delete keys.
- private static class BucketKeys {
- private Map<Pair<String, String>, List<String>> buckets;
- BucketKeys() {
- buckets = Maps.newHashMap();
- void addKey(String volume, String bucket, String key) {
- // check if this bucket exists
- for (Map.Entry<Pair<String, String>, List<String>> entry :
- buckets.entrySet()) {
- if (entry.getKey().getValue().equals(bucket)) {
- entry.getValue().add(key);
- // bucket not exist
- Pair<String, String> newBucket = new ImmutablePair(volume, bucket);
- List<String> keyList = Lists.newArrayList();
- keyList.add(key);
- buckets.put(newBucket, keyList);
- Set<Pair<String, String>> getAllBuckets() {
- return buckets.keySet();
- List<String> getBucketKeys(String bucketName) {
- for (Map.Entry<Pair<String, String>, List<String>> entry : buckets
- .entrySet()) {
- if (entry.getKey().getValue().equals(bucketName)) {
- return entry.getValue();
- return Lists.newArrayList();
- int totalNumOfKeys() {
- count += entry.getValue().size();
- private int countOmKeys(OzoneManager om) throws IOException {
- int totalCount = 0;
- List<OmVolumeArgs> volumes =
- om.listAllVolumes(null, null, Integer.MAX_VALUE);
- for (OmVolumeArgs volume : volumes) {
- List<OmBucketInfo> buckets =
- om.listBuckets(volume.getVolume(), null, null, Integer.MAX_VALUE);
- for (OmBucketInfo bucket : buckets) {
- List<OmKeyInfo> keys = om.listKeys(bucket.getVolumeName(),
- bucket.getBucketName(), null, null, Integer.MAX_VALUE);
- totalCount += keys.size();
- return totalCount;
- @Ignore("Until delete background service is fixed.")
- OzoneManager ozoneManager = ozoneCluster.getOzoneManager();
- // To avoid interference from other test cases,
- // we collect number of existing keys at the beginning
- int numOfExistedKeys = countOmKeys(ozoneManager);
- // Keep tracking bucket keys info while creating them
- PutHelper helper = new PutHelper(client, path);
- BucketKeys bucketKeys = new BucketKeys();
- for (int i = 0; i < 20; i++) {
- bucketKeys.addKey(helper.getVol().getName(), helper.getBucket().getName(),
- keyName);
- // There should be 20 keys in the buckets we just created.
- Assert.assertEquals(20, bucketKeys.totalNumOfKeys());
- int numOfCreatedKeys = 0;
- OzoneContainer cm = ozoneCluster.getHddsDatanodes().get(0)
- .getDatanodeStateMachine().getContainer();
- // Expected to delete chunk file list.
- List<File> expectedChunkFiles = Lists.newArrayList();
- // Iterate over all buckets, and list all keys in each bucket,
- // count the total number of created keys.
- Set<Pair<String, String>> buckets = bucketKeys.getAllBuckets();
- for (Pair<String, String> buk : buckets) {
- List<OmKeyInfo> createdKeys =
- ozoneManager.listKeys(buk.getKey(), buk.getValue(), null, null, 20);
- // Memorize chunks that has been created,
- // so we can verify actual deletions at DN side later.
- for (OmKeyInfo keyInfo : createdKeys) {
- List<OmKeyLocationInfo> locations =
- keyInfo.getLatestVersionLocations().getLocationList();
- OzoneTestUtils.closeContainers(keyInfo.getKeyLocationVersions(),
- ozoneCluster.getStorageContainerManager());
- for (OmKeyLocationInfo location : locations) {
- KeyValueHandler keyValueHandler = (KeyValueHandler) cm
- .getDispatcher().getHandler(ContainerProtos.ContainerType
- .KeyValueContainer);
- KeyValueContainer container = (KeyValueContainer) cm.getContainerSet()
- .getContainer(location.getBlockID().getContainerID());
- BlockData blockInfo = keyValueHandler.getBlockManager()
- .getBlock(container, location.getBlockID());
- KeyValueContainerData containerData =
- (KeyValueContainerData) container.getContainerData();
- File dataDir = new File(containerData.getChunksPath());
- for (ContainerProtos.ChunkInfo chunkInfo : blockInfo.getChunks()) {
- File chunkFile = dataDir.toPath()
- .resolve(chunkInfo.getChunkName()).toFile();
- System.out.println("Chunk File created: "
- + chunkFile.getAbsolutePath());
- Assert.assertTrue(chunkFile.exists());
- expectedChunkFiles.add(chunkFile);
- numOfCreatedKeys += createdKeys.size();
- // Ensure all keys are created.
- Assert.assertEquals(20, numOfCreatedKeys);
- // Ensure all keys are visible from OM.
- // Total number should be numOfCreated + numOfExisted
- Assert.assertEquals(20 + numOfExistedKeys, countOmKeys(ozoneManager));
- // Delete 10 keys
- int delCount = 20;
- Set<Pair<String, String>> allBuckets = bucketKeys.getAllBuckets();
- for (Pair<String, String> bucketInfo : allBuckets) {
- List<String> bks = bucketKeys.getBucketKeys(bucketInfo.getValue());
- for (String keyName : bks) {
- if (delCount > 0) {
- OmKeyArgs arg =
- new OmKeyArgs.Builder().setVolumeName(bucketInfo.getKey())
- .setBucketName(bucketInfo.getValue()).setKeyName(keyName)
- ozoneManager.deleteKey(arg);
- delCount--;
- // It should be pretty quick that keys are removed from OM namespace,
- // because actual deletion happens in async mode.
- GenericTestUtils.waitFor(() -> {
- int num = countOmKeys(ozoneManager);
- return num == (numOfExistedKeys);
- }, 1000, 10000);
- // It might take a while until all blocks are actually deleted,
- // verify all chunk files created earlier are removed from disk.
- GenericTestUtils.waitFor(
- () -> expectedChunkFiles.stream().allMatch(file -> !file.exists()),
- 1000, 60000);
@@ -1,126 +0,0 @@
-import static org.apache.hadoop.ozone.web.client
- .TestKeys.PutHelper;
- .TestKeys.getMultiPartKey;
- .TestKeys.runTestGetKeyInfo;
- .TestKeys.runTestPutAndDeleteKey;
- .TestKeys.runTestPutAndGetKey;
- .TestKeys.runTestPutAndGetKeyWithDnRestart;
- .TestKeys.runTestPutAndListKey;
- .TestKeys.runTestPutKey;
-/** The same as {@link TestKeys} except that this test is Ratis enabled. */
-public class TestKeysRatis {
- static private String path;
- path = GenericTestUtils.getTempPath(TestKeysRatis.class.getSimpleName());
- ozoneCluster = suite.getCluster();
- client = suite.newOzoneClient();
- String delimiter = RandomStringUtils.randomAlphanumeric(1);
- new PutHelper(client, path, getMultiPartKey(delimiter)),
- ozoneCluster);
@@ -1,304 +0,0 @@
-import io.netty.bootstrap.Bootstrap;
-import io.netty.channel.Channel;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.ChannelOption;
-import io.netty.channel.ChannelPipeline;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.SimpleChannelInboundHandler;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.channel.socket.nio.NioSocketChannel;
-import io.netty.handler.codec.http.DefaultFullHttpRequest;
-import io.netty.handler.codec.http.FullHttpRequest;
-import io.netty.handler.codec.http.HttpClientCodec;
-import io.netty.handler.codec.http.HttpContent;
-import io.netty.handler.codec.http.HttpContentDecompressor;
-import io.netty.handler.codec.http.HttpMethod;
-import io.netty.handler.codec.http.HttpObject;
-import io.netty.handler.codec.http.HttpResponse;
-import io.netty.handler.codec.http.HttpResponseStatus;
-import io.netty.handler.codec.http.HttpVersion;
-import io.netty.handler.codec.http.LastHttpContent;
-import io.netty.handler.logging.LogLevel;
-import io.netty.handler.logging.LoggingHandler;
-import org.apache.http.client.methods.CloseableHttpResponse;
-import java.net.HttpURLConnection;
-import static io.netty.util.CharsetUtil.UTF_8;
- * Unit tests for Ozone client connection reuse with Apache HttpClient and Netty
- * based HttpClient.
-public class TestOzoneClient {
- private static Logger log = Logger.getLogger(TestOzoneClient.class);
- private static int testVolumeCount = 5;
- private static String endpoint = null;
- Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.ALL);
- int port = cluster.getHddsDatanodes().get(0)
- endpoint = String.format("http://localhost:%d", port);
- @Test(timeout = 5000)
- public void testNewConnectionPerRequest()
- throws IOException, URISyntaxException {
- for (int i = 0; i < testVolumeCount; i++) {
- try (CloseableHttpClient httpClient =
- HttpClients.createDefault()) {
- createVolume(getRandomVolumeName(i), httpClient);
- * Object handler should be able to serve multiple requests from
- * a single http client. This allows the client side to reuse
- * http connections in a connection pool instead of creating a new
- * connection per request which consumes resource heavily.
- public void testReuseWithApacheHttpClient()
- PoolingHttpClientConnectionManager cm =
- cm.setMaxTotal(200);
- cm.setDefaultMaxPerRoute(20);
- HttpClients.custom().setConnectionManager(cm).build()) {
- @Test(timeout = 10000)
- public void testReuseWithNettyHttpClient()
- throws IOException, InterruptedException, URISyntaxException {
- URI uri = new URI(endpoint);
- String host = uri.getHost() == null? "127.0.0.1" : uri.getHost();
- int port = uri.getPort();
- EventLoopGroup workerGroup = new NioEventLoopGroup();
- Bootstrap b = new Bootstrap();
- b.group(workerGroup)
- .channel(NioSocketChannel.class)
- .option(ChannelOption.SO_KEEPALIVE, true)
- .option(ChannelOption.SO_REUSEADDR, true)
- .handler(new ChannelInitializer<SocketChannel>() {
- * This method will be called once the {@link Channel} was
- * registered. After the method returns this instance
- * will be removed from the {@link ChannelPipeline}
- * of the {@link Channel}.
- * @param ch the {@link Channel} which was registered.
- * @throws Exception is thrown if an error occurs.
- * In that case the {@link Channel} will be closed.
- public void initChannel(SocketChannel ch) {
- ChannelPipeline p = ch.pipeline();
- // Comment the following line if you don't want client http trace
- p.addLast("log", new LoggingHandler(LogLevel.INFO));
- p.addLast(new HttpClientCodec());
- p.addLast(new HttpContentDecompressor());
- p.addLast(new NettyHttpClientHandler());
- Channel ch = b.connect(host, port).sync().channel();
- String volumeName = getRandomVolumeName(i);
- sendNettyCreateVolumeRequest(ch, volumeName);
- Thread.sleep(1000);
- e.printStackTrace();
- ch.close();
- // Wait for the server to close the connection.
- ch.closeFuture().sync();
- log.error("Error received in client setup", ex);
- }finally {
- workerGroup.shutdownGracefully();
- class NettyHttpClientHandler extends
- SimpleChannelInboundHandler<HttpObject> {
- public void channelRead0(ChannelHandlerContext ctx, HttpObject msg) {
- if (msg instanceof HttpResponse) {
- HttpResponse response = (HttpResponse) msg;
- log.info("STATUS: " + response.getStatus());
- log.info("VERSION: " + response.getProtocolVersion());
- Assert.assertEquals(HttpResponseStatus.CREATED.code(),
- response.getStatus().code());
- if (msg instanceof HttpContent) {
- HttpContent content = (HttpContent) msg;
- log.info(content.content().toString(UTF_8));
- if (content instanceof LastHttpContent) {
- log.info("END OF CONTENT");
- public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
- log.error("Exception upon channel read", cause);
- ctx.close();
- private String getRandomVolumeName(int index) {
- UUID id = UUID.randomUUID();
- return "test-volume-" + index + "-" + id;
- // Prepare the HTTP request and send it over the netty channel.
- private void sendNettyCreateVolumeRequest(Channel channel, String volumeName)
- throws URISyntaxException, IOException {
- URIBuilder builder = new URIBuilder(endpoint);
- builder.setPath("/" + volumeName);
- URI uri = builder.build();
- String host = uri.getHost() == null ? "127.0.0.1" : uri.getHost();
- FullHttpRequest request = new DefaultFullHttpRequest(
- HttpVersion.HTTP_1_1, HttpMethod.POST, uri.getRawPath());
- request.headers().set(HttpHeaders.HOST, host);
- request.headers().add(HttpHeaders.CONTENT_TYPE, "application/json");
- request.headers().set(Header.OZONE_VERSION_HEADER,
- request.headers().set(HttpHeaders.DATE,
- request.headers().set(Header.OZONE_USER,
- UserGroupInformation.getCurrentUser().getUserName());
- request.headers().set(HttpHeaders.AUTHORIZATION,
- // Send the HTTP request via netty channel.
- channel.writeAndFlush(request);
- // It is caller's responsibility to close the client.
- private void createVolume(String volumeName, CloseableHttpClient httpClient)
- HttpPost create1 =
- getCreateVolumeRequest(volumeName);
- HttpEntity entity = null;
- CloseableHttpResponse response1 =
- httpClient.execute(create1);
- Assert.assertEquals(HttpURLConnection.HTTP_CREATED,
- response1.getStatusLine().getStatusCode());
- entity = response1.getEntity();
- EntityUtils.consumeQuietly(entity);
- private HttpPost getCreateVolumeRequest(String volumeName)
- HttpPost httpPost = new HttpPost(builder.build().toString());
- httpPost.addHeader(Header.OZONE_VERSION_HEADER,
- httpPost.addHeader(HttpHeaders.DATE,
- httpPost.addHeader(Header.OZONE_USER,
- return httpPost;
@@ -1,381 +0,0 @@
- * Test Ozone Volumes Lifecycle.
-public class TestVolume {
- {RpcClient.class}};
- String path = GenericTestUtils
- .getTempPath(TestVolume.class.getSimpleName());
- FileUtils.deleteDirectory(new File(path));
- runTestCreateVolume(client);
- static void runTestCreateVolume(ClientProtocol clientProtocol)
- clientProtocol.createVolume(volumeName, volumeArgs);
- OzoneVolume vol = clientProtocol.getVolumeDetails(volumeName);
- // verify the key creation time
- assertTrue((vol.getCreationTime()
- // Test create a volume with invalid volume name,
- String invalidVolumeName = "#" + OzoneUtils.getRequestID().toLowerCase();
- clientProtocol.createVolume(invalidVolumeName);
- //TODO: RestClient and RpcClient should use HddsClientUtils to verify name
- fail("Except the volume creation be failed because the"
- + " volume name starts with an invalid char #");*/
- assertTrue(e.getMessage().contains("Bucket or Volume name"
- + " has an unsupported character : #"));
- public void testCreateDuplicateVolume() throws Exception {
- runTestCreateDuplicateVolume(client);
- static void runTestCreateDuplicateVolume(ClientProtocol clientProtocol)
- clientProtocol.createVolume("testvol");
- OzoneTestUtils.expectOmException(ResultCodes.VOLUME_ALREADY_EXISTS,
- () -> clientProtocol.createVolume("testvol"));
- public void testDeleteVolume() throws OzoneException, IOException {
- runTestDeleteVolume(client);
- static void runTestDeleteVolume(ClientProtocol clientProtocol)
- clientProtocol.createVolume(volumeName);
- clientProtocol.deleteVolume(volumeName);
- public void testChangeOwnerOnVolume() throws Exception {
- runTestChangeOwnerOnVolume(client);
- static void runTestChangeOwnerOnVolume(ClientProtocol clientProtocol)
- throws OzoneException, ParseException, IOException {
- clientProtocol.getVolumeDetails(volumeName);
- clientProtocol.setVolumeOwner(volumeName, "frodo");
- OzoneVolume newVol = clientProtocol.getVolumeDetails(volumeName);
- assertEquals(newVol.getOwner(), "frodo");
- // verify if the creation time is missing after setting owner operation
- assertTrue(newVol.getCreationTime() > 0);
- public void testChangeQuotaOnVolume() throws Exception {
- runTestChangeQuotaOnVolume(client);
- static void runTestChangeQuotaOnVolume(ClientProtocol clientProtocol)
- clientProtocol.setVolumeQuota(volumeName, OzoneQuota.parseQuota("1000MB"));
- assertEquals(newVol.getQuota(),
- OzoneQuota.parseQuota("1000MB").sizeInBytes());
- // verify if the creation time is missing after setting quota operation
- // Listing all volumes in the cluster feature has to be fixed after HDDS-357.
- // TODO: fix this
- public void testListVolume() throws OzoneException, IOException {
- runTestListVolume(client);
- static void runTestListVolume(ClientProtocol clientProtocol)
- List<OzoneVolume> ovols = clientProtocol.listVolumes(
- UserGroupInformation.getCurrentUser().getUserName(), null, null, 100);
- assertTrue(ovols.size() >= 10);
- // TODO: remove @Ignore below once the problem has been resolved.
- @Ignore("Takes 3m to run, disable for now.")
- public void testListVolumePagination() throws OzoneException, IOException {
- runTestListVolumePagination(client);
- static void runTestListVolumePagination(ClientProtocol clientProtocol)
- final int volCount = 2000;
- final int step = 100;
- String prevKey = null;
- int pagecount = 0;
- while (count < volCount) {
- List<OzoneVolume> ovols = clientProtocol.listVolumes(null, prevKey, step);
- count += ovols.size();
- prevKey = ovols.get(ovols.size() - 1).getName();
- pagecount++;
- assertEquals(volCount / step, pagecount);
- public void testListAllVolumes() throws OzoneException, IOException {
- runTestListAllVolumes(client);
- static void runTestListAllVolumes(ClientProtocol clientProtocol)
- final int volCount = 200;
- final int step = 10;
- String userName =
- "frodo" + RandomStringUtils.randomAlphabetic(5).toLowerCase();
- String volumeName =
- "vol" + RandomStringUtils.randomAlphabetic(5).toLowerCase();
- if (ovols.size() > 0) {
- // becasue we are querying an existing ozone store, there will
- // be volumes created by other tests too. So we should get more page counts.
- public void testListVolumes() throws Exception {
- runTestListVolumes(client);
- static void runTestListVolumes(ClientProtocol clientProtocol)
- final String user1 = "test-user-a";
- final String user2 = "test-user-b";
- volumeName = "test-vol" + x;
- // list all the volumes belong to user1
- List<OzoneVolume> volumeList =
- clientProtocol.listVolumes(user1, null, null, 100);
- assertEquals(10, volumeList.size());
- // verify the owner name and creation time of volume
- for (OzoneVolume vol : volumeList) {
- assertTrue(vol.getOwner().equals(user1));
- // test max key parameter of listing volumes
- volumeList = clientProtocol.listVolumes(user1, null, null, 2);
- assertEquals(2, volumeList.size());
- // test prefix parameter of listing volumes
- volumeList = clientProtocol.listVolumes(user1, "test-vol10", null, 10);
- assertTrue(volumeList.size() == 1
- && volumeList.get(0).getName().equals("test-vol10"));
- volumeList = clientProtocol.listVolumes(user1, "test-vol1", null, 10);
- assertEquals(5, volumeList.size());
- // test start key parameter of listing volumes
- volumeList = clientProtocol.listVolumes(user2, null, "test-vol15", 10);
@@ -1,155 +0,0 @@
-import org.junit.*;
-/** The same as {@link TestVolume} except that this test is Ratis enabled. */
-public class TestVolumeRatis {
- // This enables Ratis in the cluster.
- conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY, true);
- cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(3).build();
- final int port = cluster.getHddsDatanodes().get(0)
- TestVolume.runTestCreateVolume(client);
- TestVolume.runTestCreateDuplicateVolume(client);
- TestVolume.runTestDeleteVolume(client);
- TestVolume.runTestChangeOwnerOnVolume(client);
- TestVolume.runTestChangeQuotaOnVolume(client);
- @Ignore("listVolumes not implemented in DistributedStorageHandler")
- TestVolume.runTestListVolume(client);
- @Ignore("See TestVolume.testListVolumePagination()")
- TestVolume.runTestListVolumePagination(client);
- @Ignore("See TestVolume.testListAllVolumes()")
- public void testListAllVolumes() throws Exception {
- TestVolume.runTestListAllVolumes(client);
- @Ignore("Disabling Ratis tests for pipeline work.")
- TestVolume.runTestListVolumes(client);
- * REST client tests.
- * Rest Client Tests.
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed under the Apache License, Version 2.0 (the "License");
- you may not use this file except in compliance with the License.
- You may obtain a copy of the License at
- http://www.apache.org/licenses/LICENSE-2.0
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License. See accompanying LICENSE file.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
- xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
- xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
-https://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
- <parent>
- <artifactId>hadoop-ozone</artifactId>
- <version>0.5.0-SNAPSHOT</version>
- </parent>
- <description>Apache Hadoop Ozone Object Store REST Service</description>
- <name>Apache Hadoop Ozone Object Store REST Service</name>
- <packaging>jar</packaging>
- <dependencies>
- <artifactId>hadoop-ozone-common</artifactId>
- <artifactId>hadoop-ozone-client</artifactId>
- <artifactId>hadoop-common</artifactId>
- <scope>provided</scope>
- <artifactId>hadoop-hdfs</artifactId>
- <artifactId>hadoop-hdfs-client</artifactId>
- <exclusions>
- <exclusion>
- <groupId>com.squareup.okhttp</groupId>
- <artifactId>okhttp</artifactId>
- </exclusion>
- </exclusions>
- <groupId>junit</groupId>
- <artifactId>junit</artifactId>
- <scope>test</scope>
- <groupId>io.swagger</groupId>
- <artifactId>swagger-annotations</artifactId>
- <version>1.5.9</version>
- <groupId>org.mockito</groupId>
- <artifactId>mockito-core</artifactId>
- <version>2.2.0</version>
- </dependencies>
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-shade-plugin</artifactId>
- <version>3.1.1</version>
- <configuration>
- <shadedArtifactAttached>true</shadedArtifactAttached>
- <shadedClassifierName>plugin</shadedClassifierName>
- <artifactSet>
- <includes>
- <!-- artifacts with provided scope will be excluded anyway -->
- <include>*:*:*:*</include>
- </includes>
- </artifactSet>
- <filters>
- <filter>
- <artifact>*:*</artifact>
- <excludes>
- <exclude>META-INF/*.SF</exclude>
- <exclude>META-INF/*.DSA</exclude>
- <exclude>META-INF/*.RSA</exclude>
- </excludes>
- </filter>
- </filters>
- </configuration>
- <executions>
- <execution>
- <phase>package</phase>
- <goals>
- <goal>shade</goal>
- </goals>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
-</project>
@@ -1,160 +0,0 @@
-package org.apache.hadoop.hdfs.server.datanode;
-import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
-import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
-import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
-import org.apache.hadoop.hdds.tracing.TracingUtil;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ipc.Client;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
-import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolClientSideTranslatorPB;
-import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB;
-import org.apache.hadoop.ozone.web.ObjectStoreApplication;
-import org.apache.hadoop.ozone.web.handlers.ServiceFilter;
-import org.apache.hadoop.ozone.web.netty.ObjectStoreJerseyContainer;
-import org.apache.hadoop.ozone.web.storage.DistributedStorageHandler;
-import com.sun.jersey.api.container.ContainerFactory;
-import com.sun.jersey.api.core.ApplicationAdapter;
-import static com.sun.jersey.api.core.ResourceConfig.FEATURE_TRACE;
-import static com.sun.jersey.api.core.ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS;
-import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
-import static org.apache.hadoop.ozone.OmUtils.getOmAddress;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_TRACE_ENABLED_DEFAULT;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_TRACE_ENABLED_KEY;
-import org.apache.ratis.protocol.ClientId;
- * Implements object store handling within the DataNode process. This class is
- * responsible for initializing and maintaining the RPC clients and servers and
- * the web application required for the object store implementation.
-public final class ObjectStoreHandler implements Closeable {
- LoggerFactory.getLogger(ObjectStoreHandler.class);
- private final ObjectStoreJerseyContainer objectStoreJerseyContainer;
- private final OzoneManagerProtocol ozoneManagerClient;
- private final StorageContainerLocationProtocol
- storageContainerLocationClient;
- private ClientId clientId = ClientId.randomId();
- * Creates a new ObjectStoreHandler.
- * @param conf configuration
- * @throws IOException if there is an I/O error
- public ObjectStoreHandler(Configuration conf) throws IOException {
- boolean ozoneTrace = conf.getBoolean(OZONE_TRACE_ENABLED_KEY,
- OZONE_TRACE_ENABLED_DEFAULT);
- // Initialize Jersey container for object store web application.
- RPC.setProtocolEngine(conf, StorageContainerLocationProtocolPB.class,
- ProtobufRpcEngine.class);
- long scmVersion =
- RPC.getProtocolVersion(StorageContainerLocationProtocolPB.class);
- InetSocketAddress scmAddress =
- getScmAddressForClients(conf);
- this.storageContainerLocationClient =
- TracingUtil.createProxy(
- new StorageContainerLocationProtocolClientSideTranslatorPB(
- RPC.getProxy(StorageContainerLocationProtocolPB.class,
- scmVersion,
- scmAddress, UserGroupInformation.getCurrentUser(), conf,
- NetUtils.getDefaultSocketFactory(conf),
- Client.getRpcTimeout(conf))),
- StorageContainerLocationProtocol.class, conf);
- RPC.setProtocolEngine(conf, OzoneManagerProtocolPB.class,
- long omVersion =
- RPC.getProtocolVersion(OzoneManagerProtocolPB.class);
- InetSocketAddress omAddress = getOmAddress(conf);
- this.ozoneManagerClient =
- new OzoneManagerProtocolClientSideTranslatorPB(
- RPC.getProxy(OzoneManagerProtocolPB.class, omVersion,
- omAddress, UserGroupInformation.getCurrentUser(), conf,
- Client.getRpcTimeout(conf)), clientId.toString()),
- OzoneManagerProtocol.class, conf);
- storageHandler = new DistributedStorageHandler(
- new OzoneConfiguration(conf),
- TracingUtil.createProxy(storageContainerLocationClient,
- StorageContainerLocationProtocol.class, conf),
- this.ozoneManagerClient);
- ApplicationAdapter aa =
- new ApplicationAdapter(new ObjectStoreApplication());
- Map<String, Object> settingsMap = new HashMap<>();
- settingsMap.put(PROPERTY_CONTAINER_REQUEST_FILTERS,
- ServiceFilter.class.getCanonicalName());
- settingsMap.put(FEATURE_TRACE, ozoneTrace);
- aa.setPropertiesAndFeatures(settingsMap);
- this.objectStoreJerseyContainer = ContainerFactory.createContainer(
- ObjectStoreJerseyContainer.class, aa);
- this.objectStoreJerseyContainer.setStorageHandler(storageHandler);
- * Returns the initialized web application container.
- * @return initialized web application container
- public ObjectStoreJerseyContainer getObjectStoreJerseyContainer() {
- return this.objectStoreJerseyContainer;
- * Returns the storage handler.
- * @return returns the storage handler
- public StorageHandler getStorageHandler() {
- return this.storageHandler;
- public void close() {
- LOG.info("Closing ObjectStoreHandler.");
- storageHandler.close();
- IOUtils.cleanupWithLogger(LOG, storageContainerLocationClient);
- IOUtils.cleanupWithLogger(LOG, ozoneManagerClient);
- * Object store related service inside the datanode.
@@ -1,222 +0,0 @@
-import javax.ws.rs.core.MediaType;
-import java.util.TimeZone;
-import org.apache.hadoop.ozone.client.io.LengthInputStream;
- * Set of Utility functions used in ozone.
-public final class OzoneRestUtils {
- private static final Logger LOG = LoggerFactory.getLogger(
- OzoneRestUtils.class);
- private OzoneRestUtils() {
- // Never constructed
- * Date format that used in ozone. Here the format is thread safe to use.
- private static final ThreadLocal<SimpleDateFormat> DATE_FORMAT =
- new ThreadLocal<SimpleDateFormat>() {
- protected SimpleDateFormat initialValue() {
- SimpleDateFormat format = new SimpleDateFormat(
- OzoneConsts.OZONE_DATE_FORMAT, Locale.US);
- format.setTimeZone(TimeZone.getTimeZone(OzoneConsts.OZONE_TIME_ZONE));
- return format;
- * verifies that bucket name / volume name is a valid DNS name.
- * @param resName Bucket or volume Name to be validated
- public static void verifyResourceName(String resName)
- HddsClientUtils.verifyResourceName(resName);
- * Returns a random Request ID.
- * Request ID is returned to the client as well as flows through the system
- * facilitating debugging on why a certain request failed.
- * @return String random request ID
- public static String getRequestID() {
- return UUID.randomUUID().toString();
- * Basic validate routine to make sure that all the
- * required headers are in place.
- * @param request - http request
- * @param reqId - request id
- * @param resource - Resource Name
- * @param hostname - Hostname
- public static void validate(Request request, HttpHeaders headers,
- String reqId, String resource, String hostname)
- throws OzoneException {
- List<String> ozHeader =
- headers.getRequestHeader(Header.OZONE_VERSION_HEADER);
- if (ozHeader == null) {
- throw ErrorTable
- .newError(ErrorTable.MISSING_VERSION, reqId, resource, hostname);
- List<String> date = headers.getRequestHeader(HttpHeaders.DATE);
- if (date == null) {
- .newError(ErrorTable.MISSING_DATE, reqId, resource, hostname);
- TODO :
- Ignore the results for time being. Eventually we can validate if the
- request Date time is too skewed and reject if it is so.
- parseDate(date.get(0), reqId, resource, hostname);
- * Parses the Date String coming from the Users.
- * @param dateString - Date String
- * @param reqID - Ozone Request ID
- * @param hostname - HostName
- * @return - Date
- * @throws OzoneException - in case of parsing error
- public static synchronized Date parseDate(String dateString, String reqID,
- String resource, String hostname)
- return DATE_FORMAT.get().parse(dateString);
- } catch (ParseException ex) {
- OzoneException exp =
- ErrorTable.newError(ErrorTable.BAD_DATE, reqID, resource, hostname);
- exp.setMessage(ex.getMessage());
- throw exp;
- * Returns a response with appropriate OZONE headers and payload.
- * @param args - UserArgs or Inherited class
- * @param statusCode - HttpStatus code
- * @param payload - Content Body
- * @return JAX-RS Response
- public static Response getResponse(UserArgs args, int statusCode,
- String payload) {
- String date = DATE_FORMAT.get().format(new Date(Time.now()));
- return Response.ok(payload)
- .header(Header.OZONE_SERVER_NAME, args.getHostName())
- .header(Header.OZONE_REQUEST_ID, args.getRequestID())
- .header(HttpHeaders.DATE, date).status(statusCode).build();
- * @param stream InputStream
- LengthInputStream stream) {
- return Response.ok(stream, MediaType.APPLICATION_OCTET_STREAM)
- .header(HttpHeaders.DATE, date).status(statusCode)
- .header(HttpHeaders.CONTENT_LENGTH, stream.getLength())
- * Convert time in millisecond to a human readable format required in ozone.
- * @return a human readable string for the input time
- public static String formatTime(long millis) {
- return DATE_FORMAT.get().format(millis);
- * Convert time in ozone date format to millisecond.
- * @return time in milliseconds
- public static long formatDate(String date) throws ParseException {
- Preconditions.checkNotNull(date, "Date string should not be null.");
- return DATE_FORMAT.get().parse(date).getTime();
- * Ozone related generic classes.
@@ -1,59 +0,0 @@
-import org.apache.hadoop.ozone.client.rest.OzoneExceptionMapper;
-import org.apache.hadoop.ozone.web.handlers.BucketHandler;
-import org.apache.hadoop.ozone.web.handlers.KeyHandler;
-import org.apache.hadoop.ozone.web.handlers.VolumeHandler;
-import org.apache.hadoop.ozone.web.messages.LengthInputStreamMessageBodyWriter;
-import org.apache.hadoop.ozone.web.messages.StringMessageBodyWriter;
-import javax.ws.rs.core.Application;
- * Ozone Application.
-public class ObjectStoreApplication extends Application {
- public ObjectStoreApplication() {
- super();
- public Set<Class<?>> getClasses() {
- HashSet<Class<?>> set = new HashSet<>();
- set.add(BucketHandler.class);
- set.add(VolumeHandler.class);
- set.add(KeyHandler.class);
- set.add(OzoneExceptionMapper.class);
- set.add(LengthInputStreamMessageBodyWriter.class);
- set.add(StringMessageBodyWriter.class);
- return set;
- public Set<Object> getSingletons() {
- HashSet<Object> set = new HashSet<>();
- set.add(ServiceFilter.class);
-import org.apache.hadoop.ozone.HddsDatanodeService;
-import org.apache.hadoop.ozone.web.netty.ObjectStoreRestHttpServer;
-import org.apache.hadoop.util.ServicePlugin;
- * DataNode service plugin implementation to start ObjectStore rest server.
-public class OzoneHddsDatanodeService implements ServicePlugin {
- LoggerFactory.getLogger(OzoneHddsDatanodeService.class);
- private Configuration conf;
- private ObjectStoreHandler handler;
- private ObjectStoreRestHttpServer objectStoreRestHttpServer;
- public void start(Object service) {
- if (service instanceof HddsDatanodeService) {
- HddsDatanodeService hddsDatanodeService = (HddsDatanodeService) service;
- conf = hddsDatanodeService.getConf();
- handler = new ObjectStoreHandler(conf);
- objectStoreRestHttpServer = new ObjectStoreRestHttpServer(
- conf, null, handler);
- objectStoreRestHttpServer.start();
- DatanodeDetails.Port restPort = DatanodeDetails.newPort(
- DatanodeDetails.Port.Name.REST,
- objectStoreRestHttpServer.getHttpAddress().getPort());
- hddsDatanodeService.getDatanodeDetails().setPort(restPort);
- throw new RuntimeException("Can't start the Object Store Rest server",
- e);
- LOG.error("Not starting {}, as the plugin is not invoked through {}",
- OzoneHddsDatanodeService.class.getSimpleName(),
- HddsDatanodeService.class.getSimpleName());
- public void stop() {
- if (handler != null) {
- handler.close();
- throw new RuntimeException("Can't stop the Object Store Rest server", e);
- IOUtils.closeQuietly(objectStoreRestHttpServer);
- IOUtils.closeQuietly(handler);
@@ -1,225 +0,0 @@
-package org.apache.hadoop.ozone.web.exceptions;
-import static java.net.HttpURLConnection.HTTP_BAD_REQUEST;
-import static java.net.HttpURLConnection.HTTP_CONFLICT;
-import static java.net.HttpURLConnection.HTTP_FORBIDDEN;
-import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR;
-import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
-import static java.net.HttpURLConnection.HTTP_UNAUTHORIZED;
- * Error Table represents the Errors from Ozone Rest API layer.
- * Please note : The errors in this table are sorted by the HTTP_ERROR codes
- * if you add new error codes to this table please follow the same convention.
-public final class ErrorTable {
- /* Error 400 */
- public static final OzoneException MISSING_VERSION =
- new OzoneException(HTTP_BAD_REQUEST, "missingVersion",
- "x-ozone-version header is required.");
- public static final OzoneException MISSING_DATE =
- new OzoneException(HTTP_BAD_REQUEST, "missingDate",
- "Date header is required.");
- public static final OzoneException BAD_DATE =
- new OzoneException(HTTP_BAD_REQUEST, "badDate",
- "Unable to parse date format.");
- public static final OzoneException MALFORMED_QUOTA =
- new OzoneException(HTTP_BAD_REQUEST, "malformedQuota",
- "Invalid quota specified.");
- public static final OzoneException MALFORMED_ACL =
- new OzoneException(HTTP_BAD_REQUEST, "malformedACL",
- "Invalid ACL specified.");
- public static final OzoneException INVALID_VOLUME_NAME =
- new OzoneException(HTTP_BAD_REQUEST, "invalidVolumeName",
- "Invalid volume name.");
- public static final OzoneException INVALID_QUERY_PARAM =
- new OzoneException(HTTP_BAD_REQUEST, "invalidQueryParam",
- "Invalid query parameter.");
- public static final OzoneException INVALID_RESOURCE_NAME =
- new OzoneException(HTTP_BAD_REQUEST, "invalidResourceName",
- "Invalid volume, bucket or key name.");
- public static final OzoneException INVALID_BUCKET_NAME =
- new OzoneException(HTTP_BAD_REQUEST, "invalidBucketName",
- "Invalid bucket name.");
- public static final OzoneException INVALID_KEY =
- new OzoneException(HTTP_BAD_REQUEST, "invalidKey", "Invalid key.");
- public static final OzoneException INVALID_REQUEST =
- new OzoneException(HTTP_BAD_REQUEST, "invalidRequest",
- "Error in request.");
- public static final OzoneException MALFORMED_BUCKET_VERSION =
- new OzoneException(HTTP_BAD_REQUEST, "malformedBucketVersion",
- "Malformed bucket version or version not unique.");
- public static final OzoneException MALFORMED_STORAGE_TYPE =
- new OzoneException(HTTP_BAD_REQUEST, "malformedStorageType",
- "Invalid storage Type specified.");
- public static final OzoneException MALFORMED_STORAGE_CLASS =
- new OzoneException(HTTP_BAD_REQUEST, "malformedStorageClass",
- "Invalid storage class specified.");
- public static final OzoneException BAD_DIGEST =
- new OzoneException(HTTP_BAD_REQUEST, "badDigest",
- "Content MD5 does not match.");
- public static final OzoneException INCOMPLETE_BODY =
- new OzoneException(HTTP_BAD_REQUEST, "incompleteBody",
- "Content length does not match stream size.");
- public static final OzoneException BAD_AUTHORIZATION =
- new OzoneException(HTTP_BAD_REQUEST, "badAuthorization",
- "Missing authorization or authorization has to be " +
- "unique.");
- public static final OzoneException BAD_PROPERTY =
- new OzoneException(HTTP_BAD_REQUEST, "unknownProperty",
- "This property is not supported by this server.");
- /* Error 401 */
- public static final OzoneException UNAUTHORIZED =
- new OzoneException(HTTP_UNAUTHORIZED, "Unauthorized",
- "Access token is missing or invalid token.");
- /* Error 403 */
- public static final OzoneException ACCESS_DENIED =
- new OzoneException(HTTP_FORBIDDEN, "accessDenied", "Access denied.");
- /* Error 404 */
- public static final OzoneException USER_NOT_FOUND =
- new OzoneException(HTTP_NOT_FOUND, "userNotFound", "Invalid user name.");
- public static final OzoneException VOLUME_NOT_FOUND =
- new OzoneException(HTTP_NOT_FOUND, "volumeNotFound", "No such volume.");
- /* Error 409 */
- public static final OzoneException VOLUME_ALREADY_EXISTS =
- new OzoneException(HTTP_CONFLICT, "volumeAlreadyExists",
- "Duplicate volume name.");
- public static final OzoneException BUCKET_ALREADY_EXISTS =
- new OzoneException(HTTP_CONFLICT, "bucketAlreadyExists",
- "Duplicate bucket name.");
- public static final OzoneException VOLUME_NOT_EMPTY =
- new OzoneException(HTTP_CONFLICT, "volumeNotEmpty",
- "Volume must not have any buckets.");
- public static final OzoneException BUCKET_NOT_EMPTY =
- new OzoneException(HTTP_CONFLICT, "bucketNotEmpty",
- "Bucket must not have any keys.");
- public static final OzoneException KEY_OPERATION_CONFLICT =
- new OzoneException(HTTP_CONFLICT, "keyOperationConflict",
- "Conflicting operation on the specified key is going" +
- " on.");
- /* Error 500 */
- public static final OzoneException SERVER_ERROR =
- new OzoneException(HTTP_INTERNAL_ERROR, "internalServerError",
- "Internal server error.");
- * Create a new instance of Error.
- * @param e Error Template
- * @param requestID Request ID
- * @param resource Resource Name
- * @param hostID hostID
- * @return creates a new instance of error based on the template
- public static OzoneException newError(OzoneException e, String requestID,
- String resource, String hostID) {
- OzoneException err =
- new OzoneException(e.getHttpCode(), e.getShortMessage(),
- e.getMessage());
- err.setRequestId(requestID);
- err.setResource(resource);
- err.setHostID(hostID);
- return err;
- * Create new instance of Error.
- * @param e - Error Template
- * @param args - Args
- * @return Ozone Exception
- public static OzoneException newError(OzoneException e, UserArgs args) {
- if (args != null) {
- err.setRequestId(args.getRequestID());
- err.setResource(args.getResourceName());
- err.setHostID(args.getHostName());
- public static OzoneException newError(OzoneException e, UserArgs args,
- Exception ex) {
- new OzoneException(e.getHttpCode(), e.getShortMessage(), ex);
- if(args != null) {
- err.setMessage(ex.getMessage());
- private ErrorTable() {
- This package contains ozone client side libraries.
@@ -1,190 +0,0 @@
-import org.apache.hadoop.ozone.OzoneRestUtils;
-import org.apache.hadoop.ozone.web.interfaces.Bucket;
-import static org.apache.hadoop.ozone.OzoneConsts.OZONE_FUNCTION;
- * Bucket Class handles all ozone Bucket related actions.
-public class BucketHandler implements Bucket {
- * createBucket call handles the POST request for Creating a Bucket.
- * @param volume - Volume name
- * @param bucket - Bucket Name
- * @param req - Http request
- * @param headers - Http headers
- * @return Response
- public Response createBucket(String volume, String bucket, Request req,
- UriInfo info, HttpHeaders headers)
- MDC.put(OZONE_FUNCTION, "createBucket");
- return new BucketProcessTemplate() {
- public Response doProcess(BucketArgs args)
- StorageHandler fs = StorageHandlerBuilder.getStorageHandler();
- args.setVersioning(getVersioning(args));
- args.setStorageType(getStorageType(args));
- fs.createBucket(args);
- return OzoneRestUtils.getResponse(args, HTTP_CREATED, "");
- }.handleCall(volume, bucket, req, info, headers);
- * updateBucket call handles the PUT request for updating a Bucket.
- * There are only three possible actions currently with updateBucket.
- * They are add/remove on ACLS, Bucket Versioning and StorageType.
- * if you make a call with any other action, update just returns 200 OK.
- * @param volume - Storage volume name
- * @param bucket - Bucket name
- public Response updateBucket(String volume, String bucket, Request req,
- MDC.put(OZONE_FUNCTION, "updateBucket");
- if (args.getVersioning() != OzoneConsts.Versioning.NOT_DEFINED) {
- fs.setBucketVersioning(args);
- if (args.getStorageType() != null) {
- fs.setBucketStorageClass(args);
- return OzoneRestUtils.getResponse(args, HTTP_OK, "");
- * Deletes an empty bucket.
- * @param volume Volume name
- * @param bucket Bucket Name
- public Response deleteBucket(String volume, String bucket, Request req,
- MDC.put(OZONE_FUNCTION, "deleteBucket");
- fs.deleteBucket(args);
- * List Buckets allows the user to list the bucket.
- * @param volume - Storage Volume Name
- * @param prefix - Prefix for the keys to be fetched
- * @param maxKeys - MaxNumber of Keys to Return
- * @param startPage - Continuation Token
- * @return - Json Body
- public Response listBucket(String volume, String bucket, final String info,
- final String prefix, final int maxKeys,
- final String startPage, Request req,
- UriInfo uriInfo, HttpHeaders headers)
- MDC.put(OZONE_FUNCTION, "listBucket");
- switch (info) {
- case Header.OZONE_INFO_QUERY_KEY:
- ListArgs listArgs = new ListArgs(args, prefix, maxKeys, startPage);
- return getBucketKeysList(listArgs);
- case Header.OZONE_INFO_QUERY_BUCKET:
- return getBucketInfoResponse(args);
- OzoneException ozException =
- ErrorTable.newError(ErrorTable.INVALID_QUERY_PARAM, args);
- ozException.setMessage("Unrecognized query param : " + info);
- throw ozException;
- }.handleCall(volume, bucket, req, uriInfo, headers);
@@ -1,294 +0,0 @@
-import java.nio.file.DirectoryNotEmptyException;
-import java.nio.file.FileAlreadyExistsException;
-import java.nio.file.NoSuchFileException;
-import org.apache.hadoop.ozone.web.interfaces.UserAuth;
-import static org.apache.hadoop.ozone.OzoneConsts.OZONE_COMPONENT;
-import static org.apache.hadoop.ozone.OzoneConsts.OZONE_REQUEST;
-import static org.apache.hadoop.ozone.OzoneConsts.OZONE_RESOURCE;
-import static org.apache.hadoop.ozone.OzoneConsts.OZONE_USER;
- * This class abstracts way the repetitive tasks in
- * Bucket handling code.
-public abstract class BucketProcessTemplate {
- LoggerFactory.getLogger(BucketProcessTemplate.class);
- * This function serves as the common error handling function
- * for all bucket related operations.
- * @param volume - Volume Name
- * @param uriInfo - Http Uri
- * @param headers - Http Headers
- public Response handleCall(String volume, String bucket, Request request,
- // TODO : Add logging
- String reqID = OzoneUtils.getRequestID();
- String hostName = OzoneUtils.getHostName();
- MDC.put(OZONE_COMPONENT, "ozone");
- MDC.put(OZONE_REQUEST, reqID);
- UserArgs userArgs = null;
- userArgs = new UserArgs(reqID, hostName, request, uriInfo, headers);
- OzoneRestUtils.validate(request, headers, reqID, bucket, hostName);
- OzoneUtils.verifyResourceName(bucket);
- UserAuth auth = UserHandlerBuilder.getAuthHandler();
- userArgs.setUserName(auth.getUser(userArgs));
- MDC.put(OZONE_USER, userArgs.getUserName());
- BucketArgs args = new BucketArgs(volume, bucket, userArgs);
- MDC.put(OZONE_RESOURCE, args.getResourceName());
- Response response = doProcess(args);
- LOG.debug("Success");
- return response;
- } catch (IllegalArgumentException argEx) {
- LOG.error("Invalid bucket.", argEx);
- throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, userArgs,
- argEx);
- } catch (IOException fsExp) {
- handleIOException(bucket, reqID, hostName, fsExp);
- * Converts FileSystem IO exceptions to OZONE exceptions.
- * @param bucket Name of the bucket
- * @param reqID Request ID
- * @param hostName Machine Name
- * @param fsExp Exception
- void handleIOException(String bucket, String reqID, String hostName,
- IOException fsExp) throws OzoneException {
- LOG.error("IOException:", fsExp);
- OzoneException exp = null;
- if (fsExp instanceof FileAlreadyExistsException) {
- exp = ErrorTable
- .newError(ErrorTable.BUCKET_ALREADY_EXISTS, reqID, bucket, hostName);
- if (fsExp instanceof DirectoryNotEmptyException) {
- .newError(ErrorTable.BUCKET_NOT_EMPTY, reqID, bucket, hostName);
- if (fsExp instanceof NoSuchFileException) {
- .newError(ErrorTable.INVALID_BUCKET_NAME, reqID, bucket, hostName);
- // Default we don't handle this exception yet,
- // report a Server Internal Error.
- if (exp == null) {
- exp =
- ErrorTable.newError(ErrorTable.SERVER_ERROR, reqID, bucket, hostName);
- if (fsExp != null) {
- exp.setMessage(fsExp.getMessage());
- * Abstract function that gets implemented in the BucketHandler functions.
- * This function will just deal with the core file system related logic
- * and will rely on handleCall function for repetitive error checks
- * @param args - parsed bucket args, name, userName, ACLs etc
- public abstract Response doProcess(BucketArgs args)
- throws OzoneException, IOException;
- * Returns the ACL String if available.
- * This function ignores all ACLs that are not prefixed with either
- * ADD or Remove
- * @param tag - Tag for different type of acls
- * @return List of ACLs
- List<String> getAcls(BucketArgs args, String tag) {
- List<String> aclStrings =
- args.getHeaders().getRequestHeader(Header.OZONE_ACLS);
- List<String> filteredSet = null;
- if (aclStrings != null) {
- filteredSet = new ArrayList<>();
- for (String s : aclStrings) {
- if (s.startsWith(tag)) {
- filteredSet.add(s.replaceFirst(tag, ""));
- return filteredSet;
- * Returns bucket versioning Info.
- * @return - String
- OzoneConsts.Versioning getVersioning(BucketArgs args) throws OzoneException {
- List<String> versionStrings =
- args.getHeaders().getRequestHeader(Header.OZONE_BUCKET_VERSIONING);
- if (versionStrings == null) {
- if (versionStrings.size() > 1) {
- OzoneException ex =
- ErrorTable.newError(ErrorTable.MALFORMED_BUCKET_VERSION, args);
- ex.setMessage("Exactly one bucket version header required");
- String version = versionStrings.get(0);
- return OzoneConsts.Versioning.valueOf(version);
- } catch (IllegalArgumentException ex) {
- LOG.debug("Malformed Version. version: {}", version);
- throw ErrorTable.newError(ErrorTable.MALFORMED_BUCKET_VERSION, args, ex);
- * Returns Storage Class if Available or returns Default.
- * @param args - bucketArgs
- * @return StorageType
- StorageType getStorageType(BucketArgs args) throws OzoneException {
- List<String> storageClassString = null;
- storageClassString =
- args.getHeaders().getRequestHeader(Header.OZONE_STORAGE_TYPE);
- if (storageClassString == null) {
- if (storageClassString.size() > 1) {
- ErrorTable.newError(ErrorTable.MALFORMED_STORAGE_TYPE, args);
- ex.setMessage("Exactly one storage class header required");
- return StorageType.valueOf(storageClassString.get(0).toUpperCase());
- if(storageClassString != null) {
- LOG.debug("Malformed storage type. Type: {}",
- storageClassString.get(0).toUpperCase());
- throw ErrorTable.newError(ErrorTable.MALFORMED_STORAGE_TYPE, args, ex);
- * Returns BucketInfo response.
- Response getBucketInfoResponse(BucketArgs args)
- BucketInfo info = fs.getBucketInfo(args);
- return OzoneRestUtils.getResponse(args, HTTP_OK, info.toJsonString());
- * Returns list of objects in a bucket.
- * @param args - ListArgs
- Response getBucketKeysList(ListArgs args) throws IOException, OzoneException {
- ListKeys objects = fs.listKeys(args);
- return OzoneRestUtils.getResponse(args.getArgs(), HTTP_OK,
- objects.toJsonString());
@@ -1,302 +0,0 @@
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-import org.apache.hadoop.ozone.web.interfaces.Keys;
-import org.apache.commons.codec.binary.Hex;
- * KeyHandler deals with basic Key Operations.
-public class KeyHandler implements Keys {
- * Gets the Key/key information if it exists.
- * @param volume Storage Volume
- * @param key Name of the key
- * @param info Tag info
- * @param req Request
- * @param uriInfo Uri Info
- * @param headers Http Header
- public Response getKey(String volume, String bucket, String key, String info,
- Request req, UriInfo uriInfo, HttpHeaders headers)
- return new KeyProcessTemplate() {
- * Abstract function that gets implemented in the KeyHandler functions.
- * @param input - The body as an Input Stream
- * @param request - Http request
- * @param headers - Parsed http Headers.
- * @param uriInfo - UriInfo
- * @throws IOException - From the file system operations
- public Response doProcess(KeyArgs args, InputStream input,
- Request request, HttpHeaders headers,
- UriInfo uriInfo)
- throws IOException, OzoneException, NoSuchAlgorithmException {
- if (info == null) {
- return getKey(args);
- } else if (info.equals(Header.OZONE_INFO_QUERY_KEY)) {
- return getKeyInfo(args);
- } else if (info.equals(Header.OZONE_INFO_QUERY_KEY_DETAIL)) {
- return getKeyInfoDetail(args);
- OzoneException ozException = ErrorTable
- .newError(ErrorTable.INVALID_QUERY_PARAM, args);
- }.handleCall(volume, bucket, key, req, headers, uriInfo, null);
- * Gets the Key if it exists.
- private Response getKey(KeyArgs args)
- LengthInputStream stream = fs.newKeyReader(args);
- return OzoneRestUtils.getResponse(args, HTTP_OK, stream);
- * Gets the Key information if it exists.
- private Response getKeyInfo(KeyArgs args)
- KeyInfo keyInfo = fs.getKeyInfo(args);
- return OzoneRestUtils.getResponse(args, HTTP_OK, keyInfo.toJsonString());
- * Gets the Key detail information if it exists.
- private Response getKeyInfoDetail(KeyArgs args)
- KeyInfo keyInfo = fs.getKeyInfoDetails(args);
- * Adds a key to an existing bucket. If the object already exists this call
- * will overwrite or add with new version number if the bucket versioning is
- * turned on.
- * @param volume Storage Volume Name
- * @param keys Name of the Object
- * @param is InputStream or File Data
- * @param headers http headers
- public Response putKey(String volume, String bucket, String keys,
- InputStream is, Request req, UriInfo info,
- HttpHeaders headers) throws OzoneException {
- UriInfo info)
- final int eof = -1;
- byte[] buffer = new byte[4 * 1024];
- String contentLenString = getContentLength(headers, args);
- String newLen = contentLenString.replaceAll("\"", "");
- int contentLen = Integer.parseInt(newLen);
- args.setSize(contentLen);
- MessageDigest md5 = MessageDigest.getInstance("MD5");
- int bytesRead = 0;
- int len = 0;
- OutputStream stream = fs.newKeyWriter(args);
- while ((bytesRead < contentLen) && (len != eof)) {
- int readSize =
- (contentLen - bytesRead > buffer.length) ? buffer.length :
- contentLen - bytesRead;
- len = input.read(buffer, 0, readSize);
- if (len != eof) {
- stream.write(buffer, 0, len);
- md5.update(buffer, 0, len);
- bytesRead += len;
- checkFileLengthMatch(args, fs, contentLen, bytesRead);
- String hashString = Hex.encodeHexString(md5.digest());
-// TODO : Enable hash value checking.
-// String contentHash = getContentMD5(headers, args);
-// checkFileHashMatch(args, hashString, fs, contentHash);
- args.setHash(hashString);
- args.setSize(bytesRead);
- fs.commitKey(args, stream);
- }.handleCall(volume, bucket, keys, req, headers, info, is);
- * Deletes an existing key.
- * @param req http Request
- * @param headers HttpHeaders
- public Response deleteKey(String volume, String bucket, String keys,
- Request req, UriInfo info, HttpHeaders headers)
- fs.deleteKey(args);
- }.handleCall(volume, bucket, keys, req, headers, info, null);
- * Renames an existing key within a bucket.
- * @param key Name of the Object
- * @param toKeyName New name of the Object
- * @param info UriInfo
- public Response renameKey(String volume, String bucket, String key,
- String toKeyName, Request req, UriInfo info, HttpHeaders headers)
- fs.renameKey(args, toKeyName);
- }.handleCall(volume, bucket, key, req, headers, info, null);
@@ -1,235 +0,0 @@
-import org.apache.commons.codec.binary.Base64;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
-import static org.apache.hadoop.ozone.web.exceptions.ErrorTable.BAD_DIGEST;
-import static org.apache.hadoop.ozone.web.exceptions.ErrorTable.INCOMPLETE_BODY;
-import static org.apache.hadoop.ozone.web.exceptions.ErrorTable.INVALID_BUCKET_NAME;
-import static org.apache.hadoop.ozone.web.exceptions.ErrorTable.INVALID_REQUEST;
-import static org.apache.hadoop.ozone.web.exceptions.ErrorTable.SERVER_ERROR;
-import static org.apache.hadoop.ozone.web.exceptions.ErrorTable.newError;
- * This class abstracts way the repetitive tasks in Key handling code.
-public abstract class KeyProcessTemplate {
- LoggerFactory.getLogger(KeyProcessTemplate.class);
- * This function serves as the common error handling function for all Key
- * related operations.
- * @param bucket bucket Name
- * @param key the object name
- * @param headers Http headers
- * @param is Input XML stream
- public Response handleCall(String volume, String bucket, String key,
- Request request, HttpHeaders headers, UriInfo info,
- InputStream is) throws OzoneException {
- userArgs = new UserArgs(reqID, hostName, request, info, headers);
- KeyArgs args = new KeyArgs(volume, bucket, key, userArgs);
- Response response = doProcess(args, is, request, headers, info);
- } catch (IllegalArgumentException argExp) {
- LOG.error("Invalid bucket in key call.", argExp);
- throw newError(INVALID_BUCKET_NAME, userArgs, argExp);
- // Map KEY_NOT_FOUND to INVALID_KEY
- if (fsExp.getMessage().endsWith(
- OzoneManagerProtocolProtos.Status.KEY_NOT_FOUND.name())) {
- throw ErrorTable.newError(ErrorTable.INVALID_KEY, userArgs, fsExp);
- // TODO : Handle errors from the FileSystem , let us map to server error
- // for now.
- throw ErrorTable.newError(ErrorTable.SERVER_ERROR, userArgs, fsExp);
- } catch (NoSuchAlgorithmException algoEx) {
- LOG.error("NoSuchAlgorithmException. Probably indicates an unusual java "
- + "installation.", algoEx);
- throw ErrorTable.newError(SERVER_ERROR, userArgs, algoEx);
- * Abstract function that gets implemented in the KeyHandler functions. This
- * function will just deal with the core file system related logic and will
- * rely on handleCall function for repetitive error checks
- public abstract Response doProcess(KeyArgs args, InputStream input,
- throws IOException, OzoneException, NoSuchAlgorithmException;
- * checks if the File Content-MD5 we wrote matches the hash we computed from
- * the stream. if it does match we delete the file and throw and exception to
- * let the user know that we have a hash mismatch
- * @param args Object Args
- * @param computedString MD5 hash value
- * @param fs Pointer to File System so we can delete the file
- * @param contentHash User Specified hash string
- public void checkFileHashMatch(KeyArgs args, String computedString,
- StorageHandler fs, String contentHash)
- if (contentHash != null) {
- String contentString =
- new String(Base64.decodeBase64(contentHash), OzoneUtils.ENCODING)
- .trim();
- if (!contentString.equals(computedString)) {
- OzoneException ex = ErrorTable.newError(BAD_DIGEST, args);
- String msg = String.format("MD5 Digest mismatch. Expected %s Found " +
- "%s", contentString, computedString);
- ex.setMessage(msg);
- LOG.debug(msg);
- * check if the content-length matches the actual stream length. if we find a
- * mismatch we will delete the file and throw an exception to let the user
- * know that length mismatch detected
- * @param fs Pointer to File System Object, to delete the file that we
- * wrote
- * @param contentLen Http Content-Length Header
- * @param bytesRead Actual Bytes we read from the stream
- public void checkFileLengthMatch(KeyArgs args, StorageHandler fs,
- int contentLen, int bytesRead)
- if (bytesRead != contentLen) {
- OzoneException ex = ErrorTable.newError(INCOMPLETE_BODY, args);
- String msg = String.format("Body length mismatch. Expected length : %d" +
- " Found %d", contentLen, bytesRead);
- * Returns Content Length header value if available.
- * @return - String or null
- public String getContentLength(HttpHeaders headers, KeyArgs args)
- List<String> contentLengthList =
- headers.getRequestHeader(HttpHeaders.CONTENT_LENGTH);
- if ((contentLengthList != null) && (contentLengthList.size() > 0)) {
- return contentLengthList.get(0);
- OzoneException ex = ErrorTable.newError(INVALID_REQUEST, args);
- ex.setMessage("Content-Length is a required header for putting a key.");
- * Returns Content MD5 value if available.
- public String getContentMD5(HttpHeaders headers, KeyArgs args) {
- headers.getRequestHeader(Header.CONTENT_MD5);
-// TODO : Should we make this compulsory ?
-// OzoneException ex = ErrorTable.newError(ErrorTable.invalidRequest, args);
-// ex.setMessage("Content-MD5 is a required header for putting a key");
-// throw ex;
- return "";
-import com.sun.jersey.spi.container.ContainerRequest;
-import com.sun.jersey.spi.container.ContainerRequestFilter;
-import javax.ws.rs.core.UriBuilder;
-import javax.ws.rs.ext.Provider;
- * This class is used to intercept root URL requests and route it to
- * Volume List functionality.
-@Provider
-public class ServiceFilter implements ContainerRequestFilter {
- * Filter the request.
- * An implementation may modify the state of the request or
- * create a new instance.
- * @param request the request.
- * @return the request.
- public ContainerRequest filter(ContainerRequest request) {
- if (request.getRequestUri().getPath().length() > 1) {
- // Just re-route it to volume handler with some hypothetical volume name.
- // volume name is ignored.
- request.setUris(request.getBaseUri(),
- UriBuilder.fromUri(request.getRequestUri())
- .path("/service")
- .queryParam("info", Header.OZONE_LIST_QUERY_SERVICE)
- .build());
- * This class is responsible for providing a {@link StorageHandler}
- * implementation to object store web handlers.
-public final class StorageHandlerBuilder {
- LoggerFactory.getLogger(StorageHandlerBuilder.class);
- private static final ThreadLocal<StorageHandler>
- STORAGE_HANDLER_THREAD_LOCAL = new ThreadLocal<>();
- * Returns the configured StorageHandler from thread-local storage for this
- * thread.
- * @return StorageHandler from thread-local storage
- public static StorageHandler getStorageHandler() throws IOException {
- StorageHandler storageHandler = STORAGE_HANDLER_THREAD_LOCAL.get();
- if (storageHandler != null) {
- return storageHandler;
- LOG.error("No Storage Handler Configured.");
- throw new IOException("Invalid Handler Configuration");
- * Removes the configured StorageHandler from thread-local storage for this
- public static void removeStorageHandler() {
- STORAGE_HANDLER_THREAD_LOCAL.remove();
- * Sets the configured StorageHandler in thread-local storage for this thread.
- * @param storageHandler StorageHandler to set in thread-local storage
- public static void setStorageHandler(StorageHandler storageHandler) {
- STORAGE_HANDLER_THREAD_LOCAL.set(storageHandler);
- * There is no reason to instantiate this class.
- private StorageHandlerBuilder() {
@@ -1,75 +0,0 @@
-import org.apache.hadoop.ozone.web.userauth.Simple;
- * This class is responsible for providing a
- * {@link org.apache.hadoop.ozone.web.interfaces.UserAuth}
-public final class UserHandlerBuilder {
- private static final ThreadLocal<UserAuth> USER_AUTH_THREAD_LOCAL =
- new ThreadLocal<UserAuth>();
- * Returns the configured UserAuth from thread-local storage for this
- * @return UserAuth from thread-local storage
- public static UserAuth getAuthHandler() {
- UserAuth authHandler = USER_AUTH_THREAD_LOCAL.get();
- if (authHandler != null) {
- return authHandler;
- // This only happens while using mvn jetty:run for testing.
- return new Simple();
- * Removes the configured UserAuth from thread-local storage for this
- public static void removeAuthHandler() {
- USER_AUTH_THREAD_LOCAL.remove();
- * Sets the configured UserAuthHandler in thread-local storage for this
- * @param authHandler authHandler to set in thread-local storage
- public static void setAuthHandler(UserAuth authHandler) {
- USER_AUTH_THREAD_LOCAL.set(authHandler);
- private UserHandlerBuilder() {
@@ -1,274 +0,0 @@
-import org.apache.hadoop.ozone.web.interfaces.Volume;
- * VolumeHandler handles volume specific HTTP calls.
- * Most functions in this file follow a simple pattern.
- * All calls are handled by VolumeProcessTemplate.handleCall, which
- * calls back into doProcess function.
- * Everything common to volume handling is abstracted out in handleCall function
- * For Example : Checking that volume name is sane, we have a supported
- * ozone version number and a valid date. That is everything common is in
- * handleCall and actions specific to a call is inside doProcess callback.
-public class VolumeHandler implements Volume {
- private static final Logger LOG = LoggerFactory.getLogger(VolumeHandler
- .class);
- * Creates a volume.
- * @param volume Volume Name, this has to be unique at Ozone cluster level
- * @param quota Quota for this Storage Volume
- * - {@literal <int>(<BYTES|MB|GB|TB>)}
- * @param req Request Object
- * @param uriInfo URI info
- * @param headers Http Headers
- * @return Standard JAX-RS Response
- public Response createVolume(String volume, final String quota, Request req,
- MDC.put(OZONE_FUNCTION, "createVolume");
- return new VolumeProcessTemplate() {
- public Response doProcess(VolumeArgs args)
- if (auth.isAdmin(args)) {
- args.setAdminName(args.getUserName());
- String volumeOwner = auth.getOzoneUser(args);
- if (volumeOwner == null) {
- throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, args);
- if (!auth.isUser(volumeOwner, args)) {
- args.setUserName(volumeOwner);
- args.setGroups(auth.getGroups(args));
- if (!quota.equals(Header.OZONE_QUOTA_UNDEFINED)) {
- setQuotaArgs(args, quota);
- fs.createVolume(args);
- throw ErrorTable.newError(ErrorTable.ACCESS_DENIED, args);
- }.handleCall(volume, req, uriInfo, headers);
- * Updates volume metadata.
- * There are only two actions possible currently with updateVolume.
- * Change the volume ownership or update quota. if you make a call
- * with neither of these actions, update just returns 200 OK.
- * @param volume Volume Name, this has to be unique at Ozone Level
- * @param quota Quota for this volume
- * - {@literal <int>(<BYTES|MB|GB|TB>)}|remove
- * @param req - Request Object
- * @param uriInfo - URI info
- public Response updateVolume(String volume, final String quota, Request req,
- MDC.put(OZONE_FUNCTION, "updateVolume");
- String newVolumeOwner = auth.getOzoneUser(args);
- if (newVolumeOwner != null) {
- if (!auth.isUser(newVolumeOwner, args)) {
- args.setUserName(newVolumeOwner);
- fs.setVolumeOwner(args);
- if (quota.equals(Header.OZONE_QUOTA_REMOVE)) {
- // if it is remove, just tell the file system to remove quota
- fs.setVolumeQuota(args, true);
- fs.setVolumeQuota(args, false);
- // Only Admins are allowed to update volumes
- * Deletes a volume if it is empty.
- * @param volume Volume Name
- * @param req - Http Request
- * @param uriInfo - http URI
- public Response deleteVolume(String volume, Request req, UriInfo uriInfo,
- MDC.put(OZONE_FUNCTION, "deleteVolume");
- fs.deleteVolume(args);
- * Returns Volume info. This API can be invoked either by admin or the owner
- * @param info - Info attribute
- * @param prefix - Prefix key
- * @param maxKeys - Max results
- * @param prevKey - PrevKey
- * @param req - Http Req
- * @param uriInfo - UriInfo.
- * @return
- public Response getVolumeInfo(String volume, final String info,
- final String prefix,
- final int maxKeys,
- final String prevKey,
- final boolean rootScan,
- Request req,
- final UriInfo uriInfo, HttpHeaders headers)
- MDC.put(OZONE_FUNCTION, "ListBucket");
- return getBucketsInVolume(args, prefix, maxKeys, prevKey);
- case Header.OZONE_INFO_QUERY_VOLUME:
- MDC.put(OZONE_FUNCTION, "InfoVolume");
- assertNoListParamPresent(uriInfo, args);
- return getVolumeInfoResponse(args); // Return volume info
- case Header.OZONE_LIST_QUERY_SERVICE:
- MDC.put(OZONE_FUNCTION, "ListVolume");
- return getVolumesByUser(args, prefix, maxKeys, prevKey, rootScan);
- LOG.debug("Unrecognized query param : {} ", info);
- OzoneException ozoneException =
- ozoneException.setMessage("Unrecognized query param : " + info);
- throw ozoneException;
- * Asserts no list query param is present during this call.
- * @param uriInfo - UriInfo. - UriInfo
- * @param args - Volume Args - VolumeArgs.
- private void assertNoListParamPresent(final UriInfo uriInfo, VolumeArgs
- args) throws
- String prefix = uriInfo.getQueryParameters().getFirst("prefix");
- String maxKeys = uriInfo.getQueryParameters().getFirst("max_keys");
- String prevKey = uriInfo.getQueryParameters().getFirst("prev_key");
- if ((prefix != null && !prefix.equals(Header.OZONE_EMPTY_STRING)) ||
- (maxKeys != null && !maxKeys.equals(Header.OZONE_DEFAULT_LIST_SIZE)) ||
- (prevKey != null && !prevKey.equals(Header.OZONE_EMPTY_STRING))) {
- throw ErrorTable.newError(ErrorTable.INVALID_QUERY_PARAM, args);
@@ -1,276 +0,0 @@
- * handling volume related code.
-public abstract class VolumeProcessTemplate {
- LoggerFactory.getLogger(VolumeProcessTemplate.class);
- * The handle call is the common functionality for Volume
- * handling code.
- * @param volume - Name of the Volume
- * @param request - request
- public Response handleCall(String volume, Request request, UriInfo info,
- OzoneRestUtils.validate(request, headers, reqID, volume, hostName);
- // we use the same logic for both bucket and volume names
- OzoneUtils.verifyResourceName(volume);
- VolumeArgs args = new VolumeArgs(volume, userArgs);
- LOG.info("Success");
- LOG.error("Illegal argument.", ex);
- throw ErrorTable.newError(ErrorTable.INVALID_VOLUME_NAME, userArgs, ex);
- handleIOException(volume, reqID, hostName, ex);
- * Specific handler for each call.
- * @param args - Volume Args
- * @return - Response
- public abstract Response doProcess(VolumeArgs args)
- throws IOException, OzoneException;
- * Maps Java File System Exceptions to Ozone Exceptions in the Volume path.
- * @param reqID - Request ID
- * @param hostName - HostName
- * @param fsExp - Exception
- private void handleIOException(String volume, String reqID, String hostName,
- if ((fsExp != null && fsExp.getMessage().endsWith(
- OzoneManagerProtocolProtos.Status.VOLUME_ALREADY_EXISTS.name()))
- || fsExp instanceof FileAlreadyExistsException) {
- .newError(ErrorTable.VOLUME_ALREADY_EXISTS, reqID, volume, hostName);
- .newError(ErrorTable.VOLUME_NOT_EMPTY, reqID, volume, hostName);
- .newError(ErrorTable.INVALID_VOLUME_NAME, reqID, volume, hostName);
- if ((fsExp != null) && (exp != null)) {
- // We don't handle that FS error yet, report a Server Internal Error
- ErrorTable.newError(ErrorTable.SERVER_ERROR, reqID, volume, hostName);
- * Set the user provided string into args and throw ozone exception
- * if needed.
- * @param args - volume args
- * @param quota - quota sting
- void setQuotaArgs(VolumeArgs args, String quota) throws OzoneException {
- args.setQuota(quota);
- LOG.debug("Malformed Quota.", ex);
- throw ErrorTable.newError(ErrorTable.MALFORMED_QUOTA, args, ex);
- * Wraps calls into volumeInfo data.
- * @param args - volumeArgs
- * @return - VolumeInfo
- Response getVolumeInfoResponse(VolumeArgs args)
- VolumeInfo info = fs.getVolumeInfo(args);
- * Returns all the volumes belonging to a user.
- * @param user - userArgs
- Response getVolumesByUser(UserArgs user, String prefix, int maxKeys,
- String prevKey, boolean rootScan) throws OzoneException, IOException {
- String validatedUser = user.getUserName();
- if(rootScan && !auth.isAdmin(user)) {
- throw ErrorTable.newError(ErrorTable.UNAUTHORIZED, user);
- if (auth.isAdmin(user)) {
- validatedUser = auth.getOzoneUser(user);
- if (validatedUser == null) {
- validatedUser = auth.getUser(user);
- UserArgs onBehalfOf =
- new UserArgs(validatedUser, user.getRequestID(), user.getHostName(),
- user.getRequest(), user.getUri(), user.getHeaders());
- ListArgs<UserArgs> listArgs = new ListArgs<>(onBehalfOf, prefix,
- maxKeys, prevKey);
- listArgs.setRootScan(rootScan);
- ListVolumes volumes = fs.listVolumes(listArgs);
- return OzoneRestUtils.getResponse(user, HTTP_OK, volumes.toJsonString());
- LOG.debug("unable to get the volume list for the user.", ex);
- OzoneException exp = ErrorTable.newError(ErrorTable.SERVER_ERROR,
- user, ex);
- exp.setMessage("unable to get the volume list for the user");
- * Returns a list of Buckets in a Volume.
- * @param args - VolumeArgs
- * @param prefix - Prefix to Match
- * @param maxKeys - Max results to return.
- * @return List of Buckets
- Response getBucketsInVolume(VolumeArgs args, String prefix, int maxKeys,
- String prevKey) throws OzoneException {
- // UserAuth auth = UserHandlerBuilder.getAuthHandler();
- // TODO : Check ACLS.
- ListArgs<VolumeArgs> listArgs = new ListArgs<>(args, prefix,
- ListBuckets bucketList = fs.listBuckets(listArgs);
- return OzoneRestUtils
- .getResponse(args, HTTP_OK, bucketList.toJsonString());
- LOG.debug("unable to get the bucket list for the specified volume.", ex);
- ErrorTable.newError(ErrorTable.SERVER_ERROR, args, ex);
- exp.setMessage("unable to get the bucket list for the specified volume.");