HDDS-167. Rename KeySpaceManager to OzoneManager. Contributed by Arpit Agarwal.
This commit is contained in:
parent
e4bf38cf50
commit
061b168529
|
@ -50,6 +50,10 @@ patchprocess/
|
||||||
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/package-lock.json
|
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/package-lock.json
|
||||||
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/yarn-error.log
|
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/yarn-error.log
|
||||||
|
|
||||||
|
# Ignore files generated by HDDS acceptance tests.
|
||||||
|
hadoop-ozone/acceptance-test/docker-compose.log
|
||||||
|
hadoop-ozone/acceptance-test/junit-results.xml
|
||||||
|
|
||||||
#robotframework outputs
|
#robotframework outputs
|
||||||
log.html
|
log.html
|
||||||
output.xml
|
output.xml
|
||||||
|
|
|
@ -148,7 +148,7 @@ run copy "${ROOT}/hadoop-ozone/tools/target/hadoop-ozone-tools-${HDDS_VERSION}"
|
||||||
mkdir -p "./share/hadoop/ozonefs"
|
mkdir -p "./share/hadoop/ozonefs"
|
||||||
cp "${ROOT}/hadoop-ozone/ozonefs/target/hadoop-ozone-filesystem-${HDDS_VERSION}.jar" "./share/hadoop/ozonefs/hadoop-ozone-filesystem.jar"
|
cp "${ROOT}/hadoop-ozone/ozonefs/target/hadoop-ozone-filesystem-${HDDS_VERSION}.jar" "./share/hadoop/ozonefs/hadoop-ozone-filesystem.jar"
|
||||||
# Optional documentation, could be missing
|
# Optional documentation, could be missing
|
||||||
cp -r "${ROOT}/hadoop-ozone/docs/target/classes/webapps/docs" ./share/hadoop/ozone/webapps/ksm/
|
cp -r "${ROOT}/hadoop-ozone/docs/target/classes/webapps/docs" ./share/hadoop/ozone/webapps/ozoneManager/
|
||||||
cp -r "${ROOT}/hadoop-ozone/docs/target/classes/webapps/docs" ./share/hadoop/hdds/webapps/scm/
|
cp -r "${ROOT}/hadoop-ozone/docs/target/classes/webapps/docs" ./share/hadoop/hdds/webapps/scm/
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -404,13 +404,13 @@ esac
|
||||||
# export HDFS_DFSROUTER_OPTS=""
|
# export HDFS_DFSROUTER_OPTS=""
|
||||||
|
|
||||||
###
|
###
|
||||||
# HDFS Key Space Manager specific parameters
|
# Ozone Manager specific parameters
|
||||||
###
|
###
|
||||||
# Specify the JVM options to be used when starting the HDFS Key Space Manager.
|
# Specify the JVM options to be used when starting the Ozone Manager.
|
||||||
# These options will be appended to the options specified as HADOOP_OPTS
|
# These options will be appended to the options specified as HADOOP_OPTS
|
||||||
# and therefore may override any similar flags set in HADOOP_OPTS
|
# and therefore may override any similar flags set in HADOOP_OPTS
|
||||||
#
|
#
|
||||||
# export HDFS_KSM_OPTS=""
|
# export HDFS_OM_OPTS=""
|
||||||
|
|
||||||
###
|
###
|
||||||
# HDFS StorageContainerManager specific parameters
|
# HDFS StorageContainerManager specific parameters
|
||||||
|
|
|
@ -25,17 +25,17 @@ services:
|
||||||
command: ["/opt/hadoop/bin/ozone","datanode"]
|
command: ["/opt/hadoop/bin/ozone","datanode"]
|
||||||
env_file:
|
env_file:
|
||||||
- ./docker-config
|
- ./docker-config
|
||||||
ksm:
|
ozoneManager:
|
||||||
image: apache/hadoop-runner
|
image: apache/hadoop-runner
|
||||||
volumes:
|
volumes:
|
||||||
- ../../ozone:/opt/hadoop
|
- ../../ozone:/opt/hadoop
|
||||||
ports:
|
ports:
|
||||||
- 9874:9874
|
- 9874:9874
|
||||||
environment:
|
environment:
|
||||||
ENSURE_KSM_INITIALIZED: /data/metadata/ksm/current/VERSION
|
ENSURE_OM_INITIALIZED: /data/metadata/ozoneManager/current/VERSION
|
||||||
env_file:
|
env_file:
|
||||||
- ./docker-config
|
- ./docker-config
|
||||||
command: ["/opt/hadoop/bin/ozone","ksm"]
|
command: ["/opt/hadoop/bin/ozone","om"]
|
||||||
scm:
|
scm:
|
||||||
image: apache/hadoop-runner
|
image: apache/hadoop-runner
|
||||||
volumes:
|
volumes:
|
||||||
|
|
|
@ -14,7 +14,7 @@
|
||||||
# See the License for the specific language governing permissions and
|
# See the License for the specific language governing permissions and
|
||||||
# limitations under the License.
|
# limitations under the License.
|
||||||
|
|
||||||
OZONE-SITE.XML_ozone.ksm.address=ksm
|
OZONE-SITE.XML_ozone.om.address=ozoneManager
|
||||||
OZONE-SITE.XML_ozone.scm.names=scm
|
OZONE-SITE.XML_ozone.scm.names=scm
|
||||||
OZONE-SITE.XML_ozone.enabled=True
|
OZONE-SITE.XML_ozone.enabled=True
|
||||||
OZONE-SITE.XML_ozone.scm.datanode.id=/data/datanode.id
|
OZONE-SITE.XML_ozone.scm.datanode.id=/data/datanode.id
|
||||||
|
|
|
@ -67,7 +67,7 @@ http://localhost:9090/graph
|
||||||
Example queries:
|
Example queries:
|
||||||
|
|
||||||
```
|
```
|
||||||
Hadoop_KeySpaceManager_NumKeyCommits
|
Hadoop_OzoneManager_NumKeyCommits
|
||||||
rate(Hadoop_KeySpaceManager_NumKeyCommits[10m])
|
rate(Hadoop_OzoneManager_NumKeyCommits[10m])
|
||||||
rate(Hadoop_Ozone_BYTES_WRITTEN[10m])
|
rate(Hadoop_Ozone_BYTES_WRITTEN[10m])
|
||||||
```
|
```
|
||||||
|
|
|
@ -26,7 +26,7 @@ services:
|
||||||
command: ["/opt/hadoop/bin/ozone","datanode"]
|
command: ["/opt/hadoop/bin/ozone","datanode"]
|
||||||
env_file:
|
env_file:
|
||||||
- ./docker-config
|
- ./docker-config
|
||||||
ksm:
|
ozoneManager:
|
||||||
image: apache/hadoop-runner
|
image: apache/hadoop-runner
|
||||||
volumes:
|
volumes:
|
||||||
- ../../ozone:/opt/hadoop
|
- ../../ozone:/opt/hadoop
|
||||||
|
@ -34,10 +34,10 @@ services:
|
||||||
ports:
|
ports:
|
||||||
- 9874:9874
|
- 9874:9874
|
||||||
environment:
|
environment:
|
||||||
ENSURE_KSM_INITIALIZED: /data/metadata/ksm/current/VERSION
|
ENSURE_OM_INITIALIZED: /data/metadata/ozoneManager/current/VERSION
|
||||||
env_file:
|
env_file:
|
||||||
- ./docker-config
|
- ./docker-config
|
||||||
command: ["/opt/hadoop/bin/ozone","ksm"]
|
command: ["/opt/hadoop/bin/ozone","om"]
|
||||||
scm:
|
scm:
|
||||||
image: apache/hadoop-runner
|
image: apache/hadoop-runner
|
||||||
volumes:
|
volumes:
|
||||||
|
|
|
@ -14,7 +14,7 @@
|
||||||
# See the License for the specific language governing permissions and
|
# See the License for the specific language governing permissions and
|
||||||
# limitations under the License.
|
# limitations under the License.
|
||||||
|
|
||||||
OZONE-SITE.XML_ozone.ksm.address=ksm
|
OZONE-SITE.XML_ozone.om.address=ozoneManager
|
||||||
OZONE-SITE.XML_ozone.scm.names=scm
|
OZONE-SITE.XML_ozone.scm.names=scm
|
||||||
OZONE-SITE.XML_ozone.enabled=True
|
OZONE-SITE.XML_ozone.enabled=True
|
||||||
OZONE-SITE.XML_ozone.scm.datanode.id=/data/datanode.id
|
OZONE-SITE.XML_ozone.scm.datanode.id=/data/datanode.id
|
||||||
|
|
|
@ -94,7 +94,7 @@ public final class ScmConfigKeys {
|
||||||
"ozone.scm.datanode.port";
|
"ozone.scm.datanode.port";
|
||||||
public static final int OZONE_SCM_DATANODE_PORT_DEFAULT = 9861;
|
public static final int OZONE_SCM_DATANODE_PORT_DEFAULT = 9861;
|
||||||
|
|
||||||
// OZONE_KSM_PORT_DEFAULT = 9862
|
// OZONE_OM_PORT_DEFAULT = 9862
|
||||||
public static final String OZONE_SCM_BLOCK_CLIENT_PORT_KEY =
|
public static final String OZONE_SCM_BLOCK_CLIENT_PORT_KEY =
|
||||||
"ozone.scm.block.client.port";
|
"ozone.scm.block.client.port";
|
||||||
public static final int OZONE_SCM_BLOCK_CLIENT_PORT_DEFAULT = 9863;
|
public static final int OZONE_SCM_BLOCK_CLIENT_PORT_DEFAULT = 9863;
|
||||||
|
|
|
@ -93,7 +93,7 @@ public final class OzoneConsts {
|
||||||
public static final String BLOCK_DB = "block.db";
|
public static final String BLOCK_DB = "block.db";
|
||||||
public static final String OPEN_CONTAINERS_DB = "openContainers.db";
|
public static final String OPEN_CONTAINERS_DB = "openContainers.db";
|
||||||
public static final String DELETED_BLOCK_DB = "deletedBlock.db";
|
public static final String DELETED_BLOCK_DB = "deletedBlock.db";
|
||||||
public static final String KSM_DB_NAME = "ksm.db";
|
public static final String OM_DB_NAME = "om.db";
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Supports Bucket Versioning.
|
* Supports Bucket Versioning.
|
||||||
|
@ -119,13 +119,13 @@ public final class OzoneConsts {
|
||||||
public static final String OPEN_KEY_ID_DELIMINATOR = "#";
|
public static final String OPEN_KEY_ID_DELIMINATOR = "#";
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* KSM LevelDB prefixes.
|
* OM LevelDB prefixes.
|
||||||
*
|
*
|
||||||
* KSM DB stores metadata as KV pairs with certain prefixes,
|
* OM DB stores metadata as KV pairs with certain prefixes,
|
||||||
* prefix is used to improve the performance to get related
|
* prefix is used to improve the performance to get related
|
||||||
* metadata.
|
* metadata.
|
||||||
*
|
*
|
||||||
* KSM DB Schema:
|
* OM DB Schema:
|
||||||
* ----------------------------------------------------------
|
* ----------------------------------------------------------
|
||||||
* | KEY | VALUE |
|
* | KEY | VALUE |
|
||||||
* ----------------------------------------------------------
|
* ----------------------------------------------------------
|
||||||
|
@ -140,13 +140,13 @@ public final class OzoneConsts {
|
||||||
* | #deleting#/volumeName/bucketName/keyName | KeyInfo |
|
* | #deleting#/volumeName/bucketName/keyName | KeyInfo |
|
||||||
* ----------------------------------------------------------
|
* ----------------------------------------------------------
|
||||||
*/
|
*/
|
||||||
public static final String KSM_VOLUME_PREFIX = "/#";
|
public static final String OM_VOLUME_PREFIX = "/#";
|
||||||
public static final String KSM_BUCKET_PREFIX = "/#";
|
public static final String OM_BUCKET_PREFIX = "/#";
|
||||||
public static final String KSM_KEY_PREFIX = "/";
|
public static final String OM_KEY_PREFIX = "/";
|
||||||
public static final String KSM_USER_PREFIX = "$";
|
public static final String OM_USER_PREFIX = "$";
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Max KSM Quota size of 1024 PB.
|
* Max OM Quota size of 1024 PB.
|
||||||
*/
|
*/
|
||||||
public static final long MAX_QUOTA_IN_BYTES = 1024L * 1024 * TB;
|
public static final long MAX_QUOTA_IN_BYTES = 1024L * 1024 * TB;
|
||||||
|
|
||||||
|
@ -168,9 +168,9 @@ public final class OzoneConsts {
|
||||||
public static final int INVALID_PORT = -1;
|
public static final int INVALID_PORT = -1;
|
||||||
|
|
||||||
|
|
||||||
// The ServiceListJSONServlet context attribute where KeySpaceManager
|
// The ServiceListJSONServlet context attribute where OzoneManager
|
||||||
// instance gets stored.
|
// instance gets stored.
|
||||||
public static final String KSM_CONTEXT_ATTRIBUTE = "ozone.ksm";
|
public static final String OM_CONTEXT_ATTRIBUTE = "ozone.om";
|
||||||
|
|
||||||
private OzoneConsts() {
|
private OzoneConsts() {
|
||||||
// Never Constructed
|
// Never Constructed
|
||||||
|
|
|
@ -38,7 +38,7 @@ import java.util.Properties;
|
||||||
* Local storage information is stored in a separate file VERSION.
|
* Local storage information is stored in a separate file VERSION.
|
||||||
* It contains type of the node,
|
* It contains type of the node,
|
||||||
* the storage layout version, the SCM id, and
|
* the storage layout version, the SCM id, and
|
||||||
* the KSM/SCM state creation time.
|
* the OM/SCM state creation time.
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
|
@ -127,7 +127,7 @@ public abstract class Storage {
|
||||||
abstract protected Properties getNodeProperties();
|
abstract protected Properties getNodeProperties();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Sets the Node properties spaecific to KSM/SCM.
|
* Sets the Node properties spaecific to OM/SCM.
|
||||||
*/
|
*/
|
||||||
private void setNodeProperties() {
|
private void setNodeProperties() {
|
||||||
Properties nodeProperties = getNodeProperties();
|
Properties nodeProperties = getNodeProperties();
|
||||||
|
@ -152,7 +152,7 @@ public abstract class Storage {
|
||||||
* File {@code VERSION} contains the following fields:
|
* File {@code VERSION} contains the following fields:
|
||||||
* <ol>
|
* <ol>
|
||||||
* <li>node type</li>
|
* <li>node type</li>
|
||||||
* <li>KSM/SCM state creation time</li>
|
* <li>OM/SCM state creation time</li>
|
||||||
* <li>other fields specific for this node type</li>
|
* <li>other fields specific for this node type</li>
|
||||||
* </ol>
|
* </ol>
|
||||||
* The version file is always written last during storage directory updates.
|
* The version file is always written last during storage directory updates.
|
||||||
|
|
|
@ -46,7 +46,7 @@ message AllocateScmBlockRequestProto {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A delete key request sent by KSM to SCM, it contains
|
* A delete key request sent by OM to SCM, it contains
|
||||||
* multiple number of keys (and their blocks).
|
* multiple number of keys (and their blocks).
|
||||||
*/
|
*/
|
||||||
message DeleteScmKeyBlocksRequestProto {
|
message DeleteScmKeyBlocksRequestProto {
|
||||||
|
@ -56,9 +56,9 @@ message DeleteScmKeyBlocksRequestProto {
|
||||||
/**
|
/**
|
||||||
* A object key and all its associated blocks.
|
* A object key and all its associated blocks.
|
||||||
* We need to encapsulate object key name plus the blocks in this potocol
|
* We need to encapsulate object key name plus the blocks in this potocol
|
||||||
* because SCM needs to response KSM with the keys it has deleted.
|
* because SCM needs to response OM with the keys it has deleted.
|
||||||
* If the response only contains blocks, it will be very expensive for
|
* If the response only contains blocks, it will be very expensive for
|
||||||
* KSM to figure out what keys have been deleted.
|
* OM to figure out what keys have been deleted.
|
||||||
*/
|
*/
|
||||||
message KeyBlocks {
|
message KeyBlocks {
|
||||||
required string key = 1;
|
required string key = 1;
|
||||||
|
@ -66,7 +66,7 @@ message KeyBlocks {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A delete key response from SCM to KSM, it contains multiple child-results.
|
* A delete key response from SCM to OM, it contains multiple child-results.
|
||||||
* Each child-result represents a key deletion result, only if all blocks of
|
* Each child-result represents a key deletion result, only if all blocks of
|
||||||
* a key are successfully deleted, this key result is considered as succeed.
|
* a key are successfully deleted, this key result is considered as succeed.
|
||||||
*/
|
*/
|
||||||
|
@ -111,7 +111,7 @@ message AllocateScmBlockResponseProto {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Protocol used from KeySpaceManager to StorageContainerManager.
|
* Protocol used from OzoneManager to StorageContainerManager.
|
||||||
* See request and response messages for details of the RPC calls.
|
* See request and response messages for details of the RPC calls.
|
||||||
*/
|
*/
|
||||||
service ScmBlockLocationProtocolService {
|
service ScmBlockLocationProtocolService {
|
||||||
|
|
|
@ -58,9 +58,9 @@ message KeyValue {
|
||||||
* Type of the node.
|
* Type of the node.
|
||||||
*/
|
*/
|
||||||
enum NodeType {
|
enum NodeType {
|
||||||
KSM = 1;
|
OM = 1; // Ozone Manager
|
||||||
SCM = 2;
|
SCM = 2; // Storage Container Manager
|
||||||
DATANODE = 3;
|
DATANODE = 3; // DataNode
|
||||||
}
|
}
|
||||||
|
|
||||||
// Should we rename NodeState to DatanodeState?
|
// Should we rename NodeState to DatanodeState?
|
||||||
|
|
|
@ -21,7 +21,7 @@
|
||||||
<!-- there. If ozone-site.xml does not already exist, create it. -->
|
<!-- there. If ozone-site.xml does not already exist, create it. -->
|
||||||
|
|
||||||
<!--Tags supported are OZONE, CBLOCK, MANAGEMENT, SECURITY, PERFORMANCE, -->
|
<!--Tags supported are OZONE, CBLOCK, MANAGEMENT, SECURITY, PERFORMANCE, -->
|
||||||
<!--DEBUG, CLIENT, SERVER, KSM, SCM, CRITICAL, RATIS, CONTAINER, REQUIRED, -->
|
<!--DEBUG, CLIENT, SERVER, OM, SCM, CRITICAL, RATIS, CONTAINER, REQUIRED, -->
|
||||||
<!--REST, STORAGE, PIPELINE, STANDALONE -->
|
<!--REST, STORAGE, PIPELINE, STANDALONE -->
|
||||||
|
|
||||||
<configuration>
|
<configuration>
|
||||||
|
@ -254,122 +254,122 @@
|
||||||
<description>
|
<description>
|
||||||
Tells ozone which storage handler to use. The possible values are:
|
Tells ozone which storage handler to use. The possible values are:
|
||||||
distributed - The Ozone distributed storage handler, which speaks to
|
distributed - The Ozone distributed storage handler, which speaks to
|
||||||
KSM/SCM on the backend and provides REST services to clients.
|
OM/SCM on the backend and provides REST services to clients.
|
||||||
local - Local Storage handler strictly for testing - To be removed.
|
local - Local Storage handler strictly for testing - To be removed.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.key.deleting.limit.per.task</name>
|
<name>ozone.key.deleting.limit.per.task</name>
|
||||||
<value>1000</value>
|
<value>1000</value>
|
||||||
<tag>KSM, PERFORMANCE</tag>
|
<tag>OM, PERFORMANCE</tag>
|
||||||
<description>
|
<description>
|
||||||
A maximum number of keys to be scanned by key deleting service
|
A maximum number of keys to be scanned by key deleting service
|
||||||
per time interval in KSM. Those keys are sent to delete metadata and
|
per time interval in OM. Those keys are sent to delete metadata and
|
||||||
generate transactions in SCM for next async deletion between SCM
|
generate transactions in SCM for next async deletion between SCM
|
||||||
and DataNode.
|
and DataNode.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.ksm.address</name>
|
<name>ozone.om.address</name>
|
||||||
<value/>
|
<value/>
|
||||||
<tag>KSM, REQUIRED</tag>
|
<tag>OM, REQUIRED</tag>
|
||||||
<description>
|
<description>
|
||||||
The address of the Ozone KSM service. This allows clients to discover
|
The address of the Ozone OM service. This allows clients to discover
|
||||||
the KSMs address.
|
the address of the OM.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.ksm.group.rights</name>
|
<name>ozone.om.group.rights</name>
|
||||||
<value>READ_WRITE</value>
|
<value>READ_WRITE</value>
|
||||||
<tag>KSM, SECURITY</tag>
|
<tag>OM, SECURITY</tag>
|
||||||
<description>
|
<description>
|
||||||
Default group permissions in Ozone KSM.
|
Default group permissions in Ozone OM.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.ksm.handler.count.key</name>
|
<name>ozone.om.handler.count.key</name>
|
||||||
<value>20</value>
|
<value>20</value>
|
||||||
<tag>KSM, PERFORMANCE</tag>
|
<tag>OM, PERFORMANCE</tag>
|
||||||
<description>
|
<description>
|
||||||
The number of RPC handler threads for KSM service endpoints.
|
The number of RPC handler threads for OM service endpoints.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.ksm.http-address</name>
|
<name>ozone.om.http-address</name>
|
||||||
<value>0.0.0.0:9874</value>
|
<value>0.0.0.0:9874</value>
|
||||||
<tag>KSM, MANAGEMENT</tag>
|
<tag>OM, MANAGEMENT</tag>
|
||||||
<description>
|
<description>
|
||||||
The address and the base port where the KSM web UI will listen on.
|
The address and the base port where the OM web UI will listen on.
|
||||||
|
|
||||||
If the port is 0, then the server will start on a free port. However, it
|
If the port is 0, then the server will start on a free port. However, it
|
||||||
is best to specify a well-known port, so it is easy to connect and see
|
is best to specify a well-known port, so it is easy to connect and see
|
||||||
the KSM management UI.
|
the OM management UI.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.ksm.http-bind-host</name>
|
<name>ozone.om.http-bind-host</name>
|
||||||
<value>0.0.0.0</value>
|
<value>0.0.0.0</value>
|
||||||
<tag>KSM, MANAGEMENT</tag>
|
<tag>OM, MANAGEMENT</tag>
|
||||||
<description>
|
<description>
|
||||||
The actual address the KSM web server will bind to. If this optional
|
The actual address the OM web server will bind to. If this optional
|
||||||
the address is set, it overrides only the hostname portion of
|
the address is set, it overrides only the hostname portion of
|
||||||
ozone.ksm.http-address.
|
ozone.om.http-address.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.ksm.http.enabled</name>
|
<name>ozone.om.http.enabled</name>
|
||||||
<value>true</value>
|
<value>true</value>
|
||||||
<tag>KSM, MANAGEMENT</tag>
|
<tag>OM, MANAGEMENT</tag>
|
||||||
<description>
|
<description>
|
||||||
Property to enable or disable KSM web user interface.
|
Property to enable or disable OM web user interface.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.ksm.https-address</name>
|
<name>ozone.om.https-address</name>
|
||||||
<value>0.0.0.0:9875</value>
|
<value>0.0.0.0:9875</value>
|
||||||
<tag>KSM, MANAGEMENT, SECURITY</tag>
|
<tag>OM, MANAGEMENT, SECURITY</tag>
|
||||||
<description>
|
<description>
|
||||||
The address and the base port where the KSM web UI will listen
|
The address and the base port where the OM web UI will listen
|
||||||
on using HTTPS.
|
on using HTTPS.
|
||||||
If the port is 0 then the server will start on a free port.
|
If the port is 0 then the server will start on a free port.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.ksm.https-bind-host</name>
|
<name>ozone.om.https-bind-host</name>
|
||||||
<value>0.0.0.0</value>
|
<value>0.0.0.0</value>
|
||||||
<tag>KSM, MANAGEMENT, SECURITY</tag>
|
<tag>OM, MANAGEMENT, SECURITY</tag>
|
||||||
<description>
|
<description>
|
||||||
The actual address the KSM web server will bind to using HTTPS.
|
The actual address the OM web server will bind to using HTTPS.
|
||||||
If this optional address is set, it overrides only the hostname portion of
|
If this optional address is set, it overrides only the hostname portion of
|
||||||
ozone.ksm.http-address.
|
ozone.om.http-address.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.ksm.keytab.file</name>
|
<name>ozone.om.keytab.file</name>
|
||||||
<value/>
|
<value/>
|
||||||
<tag>KSM, SECURITY</tag>
|
<tag>OM, SECURITY</tag>
|
||||||
<description>
|
<description>
|
||||||
The keytab file for Kerberos authentication in KSM.
|
The keytab file for Kerberos authentication in OM.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.ksm.db.cache.size.mb</name>
|
<name>ozone.om.db.cache.size.mb</name>
|
||||||
<value>128</value>
|
<value>128</value>
|
||||||
<tag>KSM, PERFORMANCE</tag>
|
<tag>OM, PERFORMANCE</tag>
|
||||||
<description>
|
<description>
|
||||||
The size of KSM DB cache in MB that used for caching files.
|
The size of OM DB cache in MB that used for caching files.
|
||||||
This value is set to an abnormally low value in the default configuration.
|
This value is set to an abnormally low value in the default configuration.
|
||||||
That is to make unit testing easy. Generally, this value should be set to
|
That is to make unit testing easy. Generally, this value should be set to
|
||||||
something like 16GB or more, if you intend to use Ozone at scale.
|
something like 16GB or more, if you intend to use Ozone at scale.
|
||||||
|
|
||||||
A large value for this key allows a proportionally larger amount of KSM
|
A large value for this key allows a proportionally larger amount of OM
|
||||||
metadata to be cached in memory. This makes KSM operations faster.
|
metadata to be cached in memory. This makes OM operations faster.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.ksm.user.max.volume</name>
|
<name>ozone.om.user.max.volume</name>
|
||||||
<value>1024</value>
|
<value>1024</value>
|
||||||
<tag>KSM, MANAGEMENT</tag>
|
<tag>OM, MANAGEMENT</tag>
|
||||||
<description>
|
<description>
|
||||||
The maximum number of volumes a user can have on a cluster.Increasing or
|
The maximum number of volumes a user can have on a cluster.Increasing or
|
||||||
decreasing this number has no real impact on ozone cluster. This is
|
decreasing this number has no real impact on ozone cluster. This is
|
||||||
|
@ -379,11 +379,11 @@
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.ksm.user.rights</name>
|
<name>ozone.om.user.rights</name>
|
||||||
<value>READ_WRITE</value>
|
<value>READ_WRITE</value>
|
||||||
<tag>KSM, SECURITY</tag>
|
<tag>OM, SECURITY</tag>
|
||||||
<description>
|
<description>
|
||||||
Default user permissions used in KSM.
|
Default user permissions used in OM.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
|
@ -393,20 +393,20 @@
|
||||||
<description>
|
<description>
|
||||||
This is used only for testing purposes. This value is used by the local
|
This is used only for testing purposes. This value is used by the local
|
||||||
storage handler to simulate a REST backend. This is useful only when
|
storage handler to simulate a REST backend. This is useful only when
|
||||||
debugging the REST front end independent of KSM and SCM. To be removed.
|
debugging the REST front end independent of OM and SCM. To be removed.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.metadata.dirs</name>
|
<name>ozone.metadata.dirs</name>
|
||||||
<value/>
|
<value/>
|
||||||
<tag>OZONE, KSM, SCM, CONTAINER, REQUIRED, STORAGE</tag>
|
<tag>OZONE, OM, SCM, CONTAINER, REQUIRED, STORAGE</tag>
|
||||||
<description>
|
<description>
|
||||||
Ozone metadata is shared among KSM, which acts as the namespace
|
Ozone metadata is shared among OM, which acts as the namespace
|
||||||
manager for ozone, SCM which acts as the block manager and data nodes
|
manager for ozone, SCM which acts as the block manager and data nodes
|
||||||
which maintain the name of the key(Key Name and BlockIDs). This
|
which maintain the name of the key(Key Name and BlockIDs). This
|
||||||
replicated and distributed metadata store is maintained under the
|
replicated and distributed metadata store is maintained under the
|
||||||
directory pointed by this key. Since metadata can be I/O intensive, at
|
directory pointed by this key. Since metadata can be I/O intensive, at
|
||||||
least on KSM and SCM we recommend having SSDs. If you have the luxury
|
least on OM and SCM we recommend having SSDs. If you have the luxury
|
||||||
of mapping this path to SSDs on all machines in the cluster, that will
|
of mapping this path to SSDs on all machines in the cluster, that will
|
||||||
be excellent.
|
be excellent.
|
||||||
|
|
||||||
|
@ -417,10 +417,10 @@
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.metastore.impl</name>
|
<name>ozone.metastore.impl</name>
|
||||||
<value>RocksDB</value>
|
<value>RocksDB</value>
|
||||||
<tag>OZONE, KSM, SCM, CONTAINER, STORAGE</tag>
|
<tag>OZONE, OM, SCM, CONTAINER, STORAGE</tag>
|
||||||
<description>
|
<description>
|
||||||
Ozone metadata store implementation. Ozone metadata are well
|
Ozone metadata store implementation. Ozone metadata are well
|
||||||
distributed to multiple services such as ksm, scm. They are stored in
|
distributed to multiple services such as ozoneManager, scm. They are stored in
|
||||||
some local key-value databases. This property determines which database
|
some local key-value databases. This property determines which database
|
||||||
library to use. Supported value is either LevelDB or RocksDB.
|
library to use. Supported value is either LevelDB or RocksDB.
|
||||||
</description>
|
</description>
|
||||||
|
@ -429,7 +429,7 @@
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.metastore.rocksdb.statistics</name>
|
<name>ozone.metastore.rocksdb.statistics</name>
|
||||||
<value>ALL</value>
|
<value>ALL</value>
|
||||||
<tag>OZONE, KSM, SCM, STORAGE, PERFORMANCE</tag>
|
<tag>OZONE, OM, SCM, STORAGE, PERFORMANCE</tag>
|
||||||
<description>
|
<description>
|
||||||
The statistics level of the rocksdb store. If you use any value from
|
The statistics level of the rocksdb store. If you use any value from
|
||||||
org.rocksdb.StatsLevel (eg. ALL or EXCEPT_DETAILED_TIMERS), the rocksdb
|
org.rocksdb.StatsLevel (eg. ALL or EXCEPT_DETAILED_TIMERS), the rocksdb
|
||||||
|
@ -672,7 +672,7 @@
|
||||||
The heartbeat interval from a data node to SCM. Yes,
|
The heartbeat interval from a data node to SCM. Yes,
|
||||||
it is not three but 30, since most data nodes will heart beating via Ratis
|
it is not three but 30, since most data nodes will heart beating via Ratis
|
||||||
heartbeats. If a client is not able to talk to a data node, it will notify
|
heartbeats. If a client is not able to talk to a data node, it will notify
|
||||||
KSM/SCM eventually. So a 30 second HB seems to work. This assumes that
|
OM/SCM eventually. So a 30 second HB seems to work. This assumes that
|
||||||
replication strategy used is Ratis if not, this value should be set to
|
replication strategy used is Ratis if not, this value should be set to
|
||||||
something smaller like 3 seconds.
|
something smaller like 3 seconds.
|
||||||
</description>
|
</description>
|
||||||
|
@ -808,7 +808,7 @@
|
||||||
<value/>
|
<value/>
|
||||||
<tag>OZONE, SECURITY</tag>
|
<tag>OZONE, SECURITY</tag>
|
||||||
<description>
|
<description>
|
||||||
The server principal used by the SCM and KSM for web UI SPNEGO
|
The server principal used by the SCM and OM for web UI SPNEGO
|
||||||
authentication when Kerberos security is enabled. This is typically set to
|
authentication when Kerberos security is enabled. This is typically set to
|
||||||
HTTP/_HOST@REALM.TLD The SPNEGO server principal begins with the prefix
|
HTTP/_HOST@REALM.TLD The SPNEGO server principal begins with the prefix
|
||||||
HTTP/ by convention.
|
HTTP/ by convention.
|
||||||
|
@ -867,9 +867,9 @@
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.key.preallocation.maxsize</name>
|
<name>ozone.key.preallocation.maxsize</name>
|
||||||
<value>134217728</value>
|
<value>134217728</value>
|
||||||
<tag>OZONE, KSM, PERFORMANCE</tag>
|
<tag>OZONE, OM, PERFORMANCE</tag>
|
||||||
<description>
|
<description>
|
||||||
When a new key write request is sent to KSM, if a size is requested, at most
|
When a new key write request is sent to OM, if a size is requested, at most
|
||||||
128MB of size is allocated at request time. If client needs more space for the
|
128MB of size is allocated at request time. If client needs more space for the
|
||||||
write, separate block allocation requests will be made.
|
write, separate block allocation requests will be made.
|
||||||
</description>
|
</description>
|
||||||
|
@ -938,7 +938,7 @@
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.open.key.cleanup.service.interval.seconds</name>
|
<name>ozone.open.key.cleanup.service.interval.seconds</name>
|
||||||
<value>86400</value>
|
<value>86400</value>
|
||||||
<tag>OZONE, KSM, PERFORMANCE</tag>
|
<tag>OZONE, OM, PERFORMANCE</tag>
|
||||||
<description>
|
<description>
|
||||||
A background job periodically checks open key entries and delete the expired ones. This entry controls the
|
A background job periodically checks open key entries and delete the expired ones. This entry controls the
|
||||||
interval of this cleanup check.
|
interval of this cleanup check.
|
||||||
|
@ -948,7 +948,7 @@
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.open.key.expire.threshold</name>
|
<name>ozone.open.key.expire.threshold</name>
|
||||||
<value>86400</value>
|
<value>86400</value>
|
||||||
<tag>OZONE, KSM, PERFORMANCE</tag>
|
<tag>OZONE, OM, PERFORMANCE</tag>
|
||||||
<description>
|
<description>
|
||||||
Controls how long an open key operation is considered active. Specifically, if a key
|
Controls how long an open key operation is considered active. Specifically, if a key
|
||||||
has been open longer than the value of this config entry, that open key is considered as
|
has been open longer than the value of this config entry, that open key is considered as
|
||||||
|
@ -958,12 +958,12 @@
|
||||||
|
|
||||||
<property>
|
<property>
|
||||||
<name>hadoop.tags.custom</name>
|
<name>hadoop.tags.custom</name>
|
||||||
<value>OZONE,MANAGEMENT,SECURITY,PERFORMANCE,DEBUG,CLIENT,SERVER,KSM,SCM,CRITICAL,RATIS,CONTAINER,REQUIRED,REST,STORAGE,PIPELINE,STANDALONE</value>
|
<value>OZONE,MANAGEMENT,SECURITY,PERFORMANCE,DEBUG,CLIENT,SERVER,OM,SCM,CRITICAL,RATIS,CONTAINER,REQUIRED,REST,STORAGE,PIPELINE,STANDALONE</value>
|
||||||
</property>
|
</property>
|
||||||
|
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.tags.system</name>
|
<name>ozone.tags.system</name>
|
||||||
<value>OZONE,MANAGEMENT,SECURITY,PERFORMANCE,DEBUG,CLIENT,SERVER,KSM,SCM,CRITICAL,RATIS,CONTAINER,REQUIRED,REST,STORAGE,PIPELINE,STANDALONE</value>
|
<value>OZONE,MANAGEMENT,SECURITY,PERFORMANCE,DEBUG,CLIENT,SERVER,OM,SCM,CRITICAL,RATIS,CONTAINER,REQUIRED,REST,STORAGE,PIPELINE,STANDALONE</value>
|
||||||
</property>
|
</property>
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -270,7 +270,7 @@
|
||||||
$http.get("conf?cmd=getOzoneTags")
|
$http.get("conf?cmd=getOzoneTags")
|
||||||
.then(function(response) {
|
.then(function(response) {
|
||||||
ctrl.tags = response.data;
|
ctrl.tags = response.data;
|
||||||
var excludedTags = ['CBLOCK', 'KSM', 'SCM'];
|
var excludedTags = ['CBLOCK', 'OM', 'SCM'];
|
||||||
for (var i = 0; i < excludedTags.length; i++) {
|
for (var i = 0; i < excludedTags.length; i++) {
|
||||||
var idx = ctrl.tags.indexOf(excludedTags[i]);
|
var idx = ctrl.tags.indexOf(excludedTags[i]);
|
||||||
// Remove CBLOCK related properties
|
// Remove CBLOCK related properties
|
||||||
|
@ -302,7 +302,7 @@
|
||||||
}
|
}
|
||||||
|
|
||||||
ctrl.loadAll = function() {
|
ctrl.loadAll = function() {
|
||||||
$http.get("conf?cmd=getPropertyByTag&tags=KSM,SCM," + ctrl.tags)
|
$http.get("conf?cmd=getPropertyByTag&tags=OM,SCM," + ctrl.tags)
|
||||||
.then(function(response) {
|
.then(function(response) {
|
||||||
|
|
||||||
ctrl.convertToArray(response.data);
|
ctrl.convertToArray(response.data);
|
||||||
|
|
|
@ -27,8 +27,8 @@
|
||||||
ng-click="$ctrl.switchto('All')">All
|
ng-click="$ctrl.switchto('All')">All
|
||||||
</a>
|
</a>
|
||||||
<a class="btn"
|
<a class="btn"
|
||||||
ng-class="$ctrl.allSelected('KSM') ? 'btn-primary' :'btn-secondary'"
|
ng-class="$ctrl.allSelected('OM') ? 'btn-primary' :'btn-secondary'"
|
||||||
ng-click="$ctrl.switchto('KSM')">KSM</a>
|
ng-click="$ctrl.switchto('OM')">OM</a>
|
||||||
<a class="btn"
|
<a class="btn"
|
||||||
ng-class="$ctrl.allSelected('SCM') ? 'btn-primary' :'btn-secondary'"
|
ng-class="$ctrl.allSelected('SCM') ? 'btn-primary' :'btn-secondary'"
|
||||||
ng-click="$ctrl.switchto('SCM')">SCM</a>
|
ng-click="$ctrl.switchto('SCM')">SCM</a>
|
||||||
|
|
|
@ -28,7 +28,7 @@ import java.util.Map;
|
||||||
/**
|
/**
|
||||||
* The DeletedBlockLog is a persisted log in SCM to keep tracking
|
* The DeletedBlockLog is a persisted log in SCM to keep tracking
|
||||||
* container blocks which are under deletion. It maintains info
|
* container blocks which are under deletion. It maintains info
|
||||||
* about under-deletion container blocks that notified by KSM,
|
* about under-deletion container blocks that notified by OM,
|
||||||
* and the state how it is processed.
|
* and the state how it is processed.
|
||||||
*/
|
*/
|
||||||
public interface DeletedBlockLog extends Closeable {
|
public interface DeletedBlockLog extends Closeable {
|
||||||
|
|
|
@ -53,9 +53,9 @@ import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
|
||||||
* client to able to write to it.
|
* client to able to write to it.
|
||||||
* <p>
|
* <p>
|
||||||
* 2. Owners - Each instance of Name service, for example, Namenode of HDFS or
|
* 2. Owners - Each instance of Name service, for example, Namenode of HDFS or
|
||||||
* Key Space Manager (KSM) of Ozone or CBlockServer -- is an owner. It is
|
* Ozone Manager (OM) of Ozone or CBlockServer -- is an owner. It is
|
||||||
* possible to have many KSMs for a Ozone cluster and only one SCM. But SCM
|
* possible to have many OMs for a Ozone cluster and only one SCM. But SCM
|
||||||
* keeps the data from each KSM in separate bucket, never mixing them. To
|
* keeps the data from each OM in separate bucket, never mixing them. To
|
||||||
* write data, often we have to find all open containers for a specific owner.
|
* write data, often we have to find all open containers for a specific owner.
|
||||||
* <p>
|
* <p>
|
||||||
* 3. ReplicationType - The clients are allowed to specify what kind of
|
* 3. ReplicationType - The clients are allowed to specify what kind of
|
||||||
|
|
|
@ -33,7 +33,7 @@ import java.util.concurrent.locks.ReentrantLock;
|
||||||
/**
|
/**
|
||||||
* Command Queue is queue of commands for the datanode.
|
* Command Queue is queue of commands for the datanode.
|
||||||
* <p>
|
* <p>
|
||||||
* Node manager, container Manager and key space managers can queue commands for
|
* Node manager, container Manager and Ozone managers can queue commands for
|
||||||
* datanodes into this queue. These commands will be send in the order in which
|
* datanodes into this queue. These commands will be send in the order in which
|
||||||
* there where queued.
|
* there where queued.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -152,7 +152,7 @@ public class SCMBlockProtocolServer implements ScmBlockLocationProtocol {
|
||||||
@Override
|
@Override
|
||||||
public List<DeleteBlockGroupResult> deleteKeyBlocks(
|
public List<DeleteBlockGroupResult> deleteKeyBlocks(
|
||||||
List<BlockGroup> keyBlocksInfoList) throws IOException {
|
List<BlockGroup> keyBlocksInfoList) throws IOException {
|
||||||
LOG.info("SCM is informed by KSM to delete {} blocks", keyBlocksInfoList
|
LOG.info("SCM is informed by OM to delete {} blocks", keyBlocksInfoList
|
||||||
.size());
|
.size());
|
||||||
List<DeleteBlockGroupResult> results = new ArrayList<>();
|
List<DeleteBlockGroupResult> results = new ArrayList<>();
|
||||||
for (BlockGroup keyBlocks : keyBlocksInfoList) {
|
for (BlockGroup keyBlocks : keyBlocksInfoList) {
|
||||||
|
|
|
@ -27,7 +27,7 @@ import java.io.IOException;
|
||||||
import java.net.URISyntaxException;
|
import java.net.URISyntaxException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class is the base CLI for scm, ksm and scmadm.
|
* This class is the base CLI for scm, om and scmadm.
|
||||||
*/
|
*/
|
||||||
public abstract class OzoneBaseCLI extends Configured implements Tool {
|
public abstract class OzoneBaseCLI extends Configured implements Tool {
|
||||||
|
|
||||||
|
|
|
@ -41,10 +41,10 @@ Test rest interface
|
||||||
Check webui static resources
|
Check webui static resources
|
||||||
${result} = Execute on scm curl -s -I http://localhost:9876/static/bootstrap-3.3.7/js/bootstrap.min.js
|
${result} = Execute on scm curl -s -I http://localhost:9876/static/bootstrap-3.3.7/js/bootstrap.min.js
|
||||||
Should contain ${result} 200
|
Should contain ${result} 200
|
||||||
${result} = Execute on ksm curl -s -I http://localhost:9874/static/bootstrap-3.3.7/js/bootstrap.min.js
|
${result} = Execute on ozoneManager curl -s -I http://localhost:9874/static/bootstrap-3.3.7/js/bootstrap.min.js
|
||||||
Should contain ${result} 200
|
Should contain ${result} 200
|
||||||
|
|
||||||
Start freon testing
|
Start freon testing
|
||||||
${result} = Execute on ksm ozone freon -numOfVolumes 5 -numOfBuckets 5 -numOfKeys 5 -numOfThreads 10
|
${result} = Execute on ozoneManager ozone freon -numOfVolumes 5 -numOfBuckets 5 -numOfKeys 5 -numOfThreads 10
|
||||||
Wait Until Keyword Succeeds 3min 10sec Should contain ${result} Number of Keys added: 125
|
Wait Until Keyword Succeeds 3min 10sec Should contain ${result} Number of Keys added: 125
|
||||||
Should Not Contain ${result} ERROR
|
Should Not Contain ${result} ERROR
|
||||||
|
|
|
@ -25,18 +25,18 @@ services:
|
||||||
command: ["/opt/hadoop/bin/ozone","datanode"]
|
command: ["/opt/hadoop/bin/ozone","datanode"]
|
||||||
env_file:
|
env_file:
|
||||||
- ./docker-config
|
- ./docker-config
|
||||||
ksm:
|
ozoneManager:
|
||||||
image: apache/hadoop-runner
|
image: apache/hadoop-runner
|
||||||
hostname: ksm
|
hostname: ozoneManager
|
||||||
volumes:
|
volumes:
|
||||||
- ${OZONEDIR}:/opt/hadoop
|
- ${OZONEDIR}:/opt/hadoop
|
||||||
ports:
|
ports:
|
||||||
- 9874
|
- 9874
|
||||||
environment:
|
environment:
|
||||||
ENSURE_KSM_INITIALIZED: /data/metadata/ksm/current/VERSION
|
ENSURE_OM_INITIALIZED: /data/metadata/ozoneManager/current/VERSION
|
||||||
env_file:
|
env_file:
|
||||||
- ./docker-config
|
- ./docker-config
|
||||||
command: ["/opt/hadoop/bin/ozone","ksm"]
|
command: ["/opt/hadoop/bin/ozone","om"]
|
||||||
scm:
|
scm:
|
||||||
image: apache/hadoop-runner
|
image: apache/hadoop-runner
|
||||||
volumes:
|
volumes:
|
||||||
|
|
|
@ -14,8 +14,8 @@
|
||||||
# See the License for the specific language governing permissions and
|
# See the License for the specific language governing permissions and
|
||||||
# limitations under the License.
|
# limitations under the License.
|
||||||
|
|
||||||
OZONE-SITE.XML_ozone.ksm.address=ksm
|
OZONE-SITE.XML_ozone.om.address=ozoneManager
|
||||||
OZONE-SITE.XML_ozone.ksm.http-address=ksm:9874
|
OZONE-SITE.XML_ozone.om.http-address=ozoneManager:9874
|
||||||
OZONE-SITE.XML_ozone.scm.names=scm
|
OZONE-SITE.XML_ozone.scm.names=scm
|
||||||
OZONE-SITE.XML_ozone.enabled=True
|
OZONE-SITE.XML_ozone.enabled=True
|
||||||
OZONE-SITE.XML_ozone.scm.datanode.id=/data/datanode.id
|
OZONE-SITE.XML_ozone.scm.datanode.id=/data/datanode.id
|
||||||
|
|
|
@ -28,16 +28,16 @@ ${PROJECTDIR} ${CURDIR}/../../../../../..
|
||||||
|
|
||||||
*** Test Cases ***
|
*** Test Cases ***
|
||||||
RestClient without http port
|
RestClient without http port
|
||||||
Test ozone shell http:// ksm restwoport True
|
Test ozone shell http:// ozoneManager restwoport True
|
||||||
|
|
||||||
RestClient with http port
|
RestClient with http port
|
||||||
Test ozone shell http:// ksm:9874 restwport True
|
Test ozone shell http:// ozoneManager:9874 restwport True
|
||||||
|
|
||||||
RestClient without host name
|
RestClient without host name
|
||||||
Test ozone shell http:// ${EMPTY} restwohost True
|
Test ozone shell http:// ${EMPTY} restwohost True
|
||||||
|
|
||||||
RpcClient with port
|
RpcClient with port
|
||||||
Test ozone shell o3:// ksm:9862 rpcwoport False
|
Test ozone shell o3:// ozoneManager:9862 rpcwoport False
|
||||||
|
|
||||||
RpcClient without host
|
RpcClient without host
|
||||||
Test ozone shell o3:// ${EMPTY} rpcwport False
|
Test ozone shell o3:// ${EMPTY} rpcwport False
|
||||||
|
@ -52,7 +52,7 @@ Test ozone shell
|
||||||
${result} = Execute on datanode ozone oz -createVolume ${protocol}${server}/${volume} -user bilbo -quota 100TB -root
|
${result} = Execute on datanode ozone oz -createVolume ${protocol}${server}/${volume} -user bilbo -quota 100TB -root
|
||||||
Should not contain ${result} Failed
|
Should not contain ${result} Failed
|
||||||
Should contain ${result} Creating Volume: ${volume}
|
Should contain ${result} Creating Volume: ${volume}
|
||||||
${result} = Execute on datanode ozone oz -listVolume o3://ksm -user bilbo | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.[] | select(.volumeName=="${volume}")'
|
${result} = Execute on datanode ozone oz -listVolume o3://ozoneManager -user bilbo | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.[] | select(.volumeName=="${volume}")'
|
||||||
Should contain ${result} createdOn
|
Should contain ${result} createdOn
|
||||||
Execute on datanode ozone oz -updateVolume ${protocol}${server}/${volume} -user bill -quota 10TB
|
Execute on datanode ozone oz -updateVolume ${protocol}${server}/${volume} -user bill -quota 10TB
|
||||||
${result} = Execute on datanode ozone oz -infoVolume ${protocol}${server}/${volume} | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.volumeName=="${volume}") | .owner | .name'
|
${result} = Execute on datanode ozone oz -infoVolume ${protocol}${server}/${volume} | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.volumeName=="${volume}") | .owner | .name'
|
||||||
|
@ -66,7 +66,7 @@ Test ozone shell
|
||||||
Should Be Equal ${result} GROUP
|
Should Be Equal ${result} GROUP
|
||||||
${result} = Execute on datanode ozone oz -updateBucket ${protocol}${server}/${volume}/bb1 -removeAcl group:samwise:r | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.bucketName=="bb1") | .acls | .[] | select(.name=="frodo") | .type'
|
${result} = Execute on datanode ozone oz -updateBucket ${protocol}${server}/${volume}/bb1 -removeAcl group:samwise:r | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.bucketName=="bb1") | .acls | .[] | select(.name=="frodo") | .type'
|
||||||
Should Be Equal ${result} USER
|
Should Be Equal ${result} USER
|
||||||
${result} = Execute on datanode ozone oz -listBucket o3://ksm/${volume}/ | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.[] | select(.bucketName=="bb1") | .volumeName'
|
${result} = Execute on datanode ozone oz -listBucket o3://ozoneManager/${volume}/ | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.[] | select(.bucketName=="bb1") | .volumeName'
|
||||||
Should Be Equal ${result} ${volume}
|
Should Be Equal ${result} ${volume}
|
||||||
Run Keyword and Return If ${withkeytest} Test key handling ${protocol} ${server} ${volume}
|
Run Keyword and Return If ${withkeytest} Test key handling ${protocol} ${server} ${volume}
|
||||||
Execute on datanode ozone oz -deleteBucket ${protocol}${server}/${volume}/bb1
|
Execute on datanode ozone oz -deleteBucket ${protocol}${server}/${volume}/bb1
|
||||||
|
@ -80,6 +80,6 @@ Test key handling
|
||||||
Execute on datanode ls -l NOTICE.txt.1
|
Execute on datanode ls -l NOTICE.txt.1
|
||||||
${result} = Execute on datanode ozone oz -infoKey ${protocol}${server}/${volume}/bb1/key1 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.keyName=="key1")'
|
${result} = Execute on datanode ozone oz -infoKey ${protocol}${server}/${volume}/bb1/key1 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '. | select(.keyName=="key1")'
|
||||||
Should contain ${result} createdOn
|
Should contain ${result} createdOn
|
||||||
${result} = Execute on datanode ozone oz -listKey o3://ksm/${volume}/bb1 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.[] | select(.keyName=="key1") | .keyName'
|
${result} = Execute on datanode ozone oz -listKey o3://ozoneManager/${volume}/bb1 | grep -Ev 'Removed|WARN|DEBUG|ERROR|INFO|TRACE' | jq -r '.[] | select(.keyName=="key1") | .keyName'
|
||||||
Should Be Equal ${result} key1
|
Should Be Equal ${result} key1
|
||||||
Execute on datanode ozone oz -deleteKey ${protocol}${server}/${volume}/bb1/key1 -v
|
Execute on datanode ozone oz -deleteKey ${protocol}${server}/${volume}/bb1/key1 -v
|
||||||
|
|
|
@ -21,12 +21,12 @@ Startup Ozone cluster with size
|
||||||
Run echo "Starting new docker-compose environment" >> docker-compose.log
|
Run echo "Starting new docker-compose environment" >> docker-compose.log
|
||||||
${rc} ${output} = Run docker compose up -d
|
${rc} ${output} = Run docker compose up -d
|
||||||
Should Be Equal As Integers ${rc} 0
|
Should Be Equal As Integers ${rc} 0
|
||||||
Wait Until Keyword Succeeds 1min 5sec Is Daemon started ksm HTTP server of KSM is listening
|
Wait Until Keyword Succeeds 1min 5sec Is Daemon started ozoneManager HTTP server of OZONEMANAGER is listening
|
||||||
Daemons are running without error
|
Daemons are running without error
|
||||||
Scale datanodes up 5
|
Scale datanodes up 5
|
||||||
|
|
||||||
Daemons are running without error
|
Daemons are running without error
|
||||||
Is daemon running without error ksm
|
Is daemon running without error ozoneManager
|
||||||
Is daemon running without error scm
|
Is daemon running without error scm
|
||||||
Is daemon running without error datanode
|
Is daemon running without error datanode
|
||||||
|
|
||||||
|
|
|
@ -25,18 +25,18 @@ services:
|
||||||
command: ["/opt/hadoop/bin/ozone","datanode"]
|
command: ["/opt/hadoop/bin/ozone","datanode"]
|
||||||
env_file:
|
env_file:
|
||||||
- ./docker-config
|
- ./docker-config
|
||||||
ksm:
|
ozoneManager:
|
||||||
image: apache/hadoop-runner
|
image: apache/hadoop-runner
|
||||||
hostname: ksm
|
hostname: ozoneManager
|
||||||
volumes:
|
volumes:
|
||||||
- ${OZONEDIR}:/opt/hadoop
|
- ${OZONEDIR}:/opt/hadoop
|
||||||
ports:
|
ports:
|
||||||
- 9874
|
- 9874
|
||||||
environment:
|
environment:
|
||||||
ENSURE_KSM_INITIALIZED: /data/metadata/ksm/current/VERSION
|
ENSURE_OM_INITIALIZED: /data/metadata/ozoneManager/current/VERSION
|
||||||
env_file:
|
env_file:
|
||||||
- ./docker-config
|
- ./docker-config
|
||||||
command: ["/opt/hadoop/bin/ozone","ksm"]
|
command: ["/opt/hadoop/bin/ozone","om"]
|
||||||
scm:
|
scm:
|
||||||
image: apache/hadoop-runner
|
image: apache/hadoop-runner
|
||||||
volumes:
|
volumes:
|
||||||
|
|
|
@ -15,8 +15,8 @@
|
||||||
# limitations under the License.
|
# limitations under the License.
|
||||||
|
|
||||||
CORE-SITE.XML_fs.o3.impl=org.apache.hadoop.fs.ozone.OzoneFileSystem
|
CORE-SITE.XML_fs.o3.impl=org.apache.hadoop.fs.ozone.OzoneFileSystem
|
||||||
OZONE-SITE.XML_ozone.ksm.address=ksm
|
OZONE-SITE.XML_ozone.om.address=ozoneManager
|
||||||
OZONE-SITE.XML_ozone.ksm.http-address=ksm:9874
|
OZONE-SITE.XML_ozone.om.http-address=ozoneManager:9874
|
||||||
OZONE-SITE.XML_ozone.scm.names=scm
|
OZONE-SITE.XML_ozone.scm.names=scm
|
||||||
OZONE-SITE.XML_ozone.enabled=True
|
OZONE-SITE.XML_ozone.enabled=True
|
||||||
OZONE-SITE.XML_ozone.scm.datanode.id=/data/datanode.id
|
OZONE-SITE.XML_ozone.scm.datanode.id=/data/datanode.id
|
||||||
|
|
|
@ -27,13 +27,13 @@ ${PROJECTDIR} ${CURDIR}/../../../../../..
|
||||||
|
|
||||||
*** Test Cases ***
|
*** Test Cases ***
|
||||||
Create volume and bucket
|
Create volume and bucket
|
||||||
Execute on datanode ozone oz -createVolume http://ksm/fstest -user bilbo -quota 100TB -root
|
Execute on datanode ozone oz -createVolume http://ozoneManager/fstest -user bilbo -quota 100TB -root
|
||||||
Execute on datanode ozone oz -createBucket http://ksm/fstest/bucket1
|
Execute on datanode ozone oz -createBucket http://ozoneManager/fstest/bucket1
|
||||||
|
|
||||||
Check volume from ozonefs
|
Check volume from ozonefs
|
||||||
${result} = Execute on hadooplast hdfs dfs -ls o3://bucket1.fstest/
|
${result} = Execute on hadooplast hdfs dfs -ls o3://bucket1.fstest/
|
||||||
|
|
||||||
Create directory from ozonefs
|
Create directory from ozonefs
|
||||||
Execute on hadooplast hdfs dfs -mkdir -p o3://bucket1.fstest/testdir/deep
|
Execute on hadooplast hdfs dfs -mkdir -p o3://bucket1.fstest/testdir/deep
|
||||||
${result} = Execute on ksm ozone oz -listKey o3://ksm/fstest/bucket1 | grep -v WARN | jq -r '.[].keyName'
|
${result} = Execute on ozoneManager ozone oz -listKey o3://ozoneManager/fstest/bucket1 | grep -v WARN | jq -r '.[].keyName'
|
||||||
Should contain ${result} testdir/deep
|
Should contain ${result} testdir/deep
|
||||||
|
|
|
@ -81,7 +81,7 @@ public final class BucketArgs {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns new builder class that builds a KsmBucketInfo.
|
* Returns new builder class that builds a OmBucketInfo.
|
||||||
*
|
*
|
||||||
* @return Builder
|
* @return Builder
|
||||||
*/
|
*/
|
||||||
|
@ -90,7 +90,7 @@ public final class BucketArgs {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Builder for KsmBucketInfo.
|
* Builder for OmBucketInfo.
|
||||||
*/
|
*/
|
||||||
public static class Builder {
|
public static class Builder {
|
||||||
private Boolean versioning;
|
private Boolean versioning;
|
||||||
|
|
|
@ -21,7 +21,7 @@ package org.apache.hadoop.ozone.client;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.ozone.KsmUtils;
|
import org.apache.hadoop.ozone.OmUtils;
|
||||||
import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
|
import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
|
||||||
import org.apache.hadoop.ozone.client.rest.RestClient;
|
import org.apache.hadoop.ozone.client.rest.RestClient;
|
||||||
import org.apache.hadoop.ozone.client.rpc.RpcClient;
|
import org.apache.hadoop.ozone.client.rpc.RpcClient;
|
||||||
|
@ -34,11 +34,9 @@ import java.lang.reflect.Constructor;
|
||||||
import java.lang.reflect.InvocationTargetException;
|
import java.lang.reflect.InvocationTargetException;
|
||||||
import java.lang.reflect.Proxy;
|
import java.lang.reflect.Proxy;
|
||||||
|
|
||||||
import static org.apache.hadoop.ozone.OzoneConfigKeys
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_CLIENT_PROTOCOL;
|
||||||
.OZONE_CLIENT_PROTOCOL;
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
|
||||||
import static org.apache.hadoop.ozone.ksm.KSMConfigKeys
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY;
|
||||||
.OZONE_KSM_HTTP_ADDRESS_KEY;
|
|
||||||
import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_ADDRESS_KEY;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Factory class to create different types of OzoneClients.
|
* Factory class to create different types of OzoneClients.
|
||||||
|
@ -97,46 +95,46 @@ public final class OzoneClientFactory {
|
||||||
/**
|
/**
|
||||||
* Returns an OzoneClient which will use RPC protocol.
|
* Returns an OzoneClient which will use RPC protocol.
|
||||||
*
|
*
|
||||||
* @param ksmHost
|
* @param omHost
|
||||||
* hostname of KeySpaceManager to connect.
|
* hostname of OzoneManager to connect.
|
||||||
*
|
*
|
||||||
* @return OzoneClient
|
* @return OzoneClient
|
||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static OzoneClient getRpcClient(String ksmHost)
|
public static OzoneClient getRpcClient(String omHost)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Configuration config = new OzoneConfiguration();
|
Configuration config = new OzoneConfiguration();
|
||||||
int port = KsmUtils.getKsmRpcPort(config);
|
int port = OmUtils.getOmRpcPort(config);
|
||||||
return getRpcClient(ksmHost, port, config);
|
return getRpcClient(omHost, port, config);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns an OzoneClient which will use RPC protocol.
|
* Returns an OzoneClient which will use RPC protocol.
|
||||||
*
|
*
|
||||||
* @param ksmHost
|
* @param omHost
|
||||||
* hostname of KeySpaceManager to connect.
|
* hostname of OzoneManager to connect.
|
||||||
*
|
*
|
||||||
* @param ksmRpcPort
|
* @param omRpcPort
|
||||||
* RPC port of KeySpaceManager.
|
* RPC port of OzoneManager.
|
||||||
*
|
*
|
||||||
* @return OzoneClient
|
* @return OzoneClient
|
||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static OzoneClient getRpcClient(String ksmHost, Integer ksmRpcPort)
|
public static OzoneClient getRpcClient(String omHost, Integer omRpcPort)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return getRpcClient(ksmHost, ksmRpcPort, new OzoneConfiguration());
|
return getRpcClient(omHost, omRpcPort, new OzoneConfiguration());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns an OzoneClient which will use RPC protocol.
|
* Returns an OzoneClient which will use RPC protocol.
|
||||||
*
|
*
|
||||||
* @param ksmHost
|
* @param omHost
|
||||||
* hostname of KeySpaceManager to connect.
|
* hostname of OzoneManager to connect.
|
||||||
*
|
*
|
||||||
* @param ksmRpcPort
|
* @param omRpcPort
|
||||||
* RPC port of KeySpaceManager.
|
* RPC port of OzoneManager.
|
||||||
*
|
*
|
||||||
* @param config
|
* @param config
|
||||||
* Configuration to be used for OzoneClient creation
|
* Configuration to be used for OzoneClient creation
|
||||||
|
@ -145,13 +143,13 @@ public final class OzoneClientFactory {
|
||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static OzoneClient getRpcClient(String ksmHost, Integer ksmRpcPort,
|
public static OzoneClient getRpcClient(String omHost, Integer omRpcPort,
|
||||||
Configuration config)
|
Configuration config)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Preconditions.checkNotNull(ksmHost);
|
Preconditions.checkNotNull(omHost);
|
||||||
Preconditions.checkNotNull(ksmRpcPort);
|
Preconditions.checkNotNull(omRpcPort);
|
||||||
Preconditions.checkNotNull(config);
|
Preconditions.checkNotNull(config);
|
||||||
config.set(OZONE_KSM_ADDRESS_KEY, ksmHost + ":" + ksmRpcPort);
|
config.set(OZONE_OM_ADDRESS_KEY, omHost + ":" + omRpcPort);
|
||||||
return getRpcClient(config);
|
return getRpcClient(config);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -175,46 +173,46 @@ public final class OzoneClientFactory {
|
||||||
/**
|
/**
|
||||||
* Returns an OzoneClient which will use REST protocol.
|
* Returns an OzoneClient which will use REST protocol.
|
||||||
*
|
*
|
||||||
* @param ksmHost
|
* @param omHost
|
||||||
* hostname of KeySpaceManager to connect.
|
* hostname of OzoneManager to connect.
|
||||||
*
|
*
|
||||||
* @return OzoneClient
|
* @return OzoneClient
|
||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static OzoneClient getRestClient(String ksmHost)
|
public static OzoneClient getRestClient(String omHost)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Configuration config = new OzoneConfiguration();
|
Configuration config = new OzoneConfiguration();
|
||||||
int port = KsmUtils.getKsmRestPort(config);
|
int port = OmUtils.getOmRestPort(config);
|
||||||
return getRestClient(ksmHost, port, config);
|
return getRestClient(omHost, port, config);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns an OzoneClient which will use REST protocol.
|
* Returns an OzoneClient which will use REST protocol.
|
||||||
*
|
*
|
||||||
* @param ksmHost
|
* @param omHost
|
||||||
* hostname of KeySpaceManager to connect.
|
* hostname of OzoneManager to connect.
|
||||||
*
|
*
|
||||||
* @param ksmHttpPort
|
* @param omHttpPort
|
||||||
* HTTP port of KeySpaceManager.
|
* HTTP port of OzoneManager.
|
||||||
*
|
*
|
||||||
* @return OzoneClient
|
* @return OzoneClient
|
||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static OzoneClient getRestClient(String ksmHost, Integer ksmHttpPort)
|
public static OzoneClient getRestClient(String omHost, Integer omHttpPort)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return getRestClient(ksmHost, ksmHttpPort, new OzoneConfiguration());
|
return getRestClient(omHost, omHttpPort, new OzoneConfiguration());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns an OzoneClient which will use REST protocol.
|
* Returns an OzoneClient which will use REST protocol.
|
||||||
*
|
*
|
||||||
* @param ksmHost
|
* @param omHost
|
||||||
* hostname of KeySpaceManager to connect.
|
* hostname of OzoneManager to connect.
|
||||||
*
|
*
|
||||||
* @param ksmHttpPort
|
* @param omHttpPort
|
||||||
* HTTP port of KeySpaceManager.
|
* HTTP port of OzoneManager.
|
||||||
*
|
*
|
||||||
* @param config
|
* @param config
|
||||||
* Configuration to be used for OzoneClient creation
|
* Configuration to be used for OzoneClient creation
|
||||||
|
@ -223,13 +221,13 @@ public final class OzoneClientFactory {
|
||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static OzoneClient getRestClient(String ksmHost, Integer ksmHttpPort,
|
public static OzoneClient getRestClient(String omHost, Integer omHttpPort,
|
||||||
Configuration config)
|
Configuration config)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Preconditions.checkNotNull(ksmHost);
|
Preconditions.checkNotNull(omHost);
|
||||||
Preconditions.checkNotNull(ksmHttpPort);
|
Preconditions.checkNotNull(omHttpPort);
|
||||||
Preconditions.checkNotNull(config);
|
Preconditions.checkNotNull(config);
|
||||||
config.set(OZONE_KSM_HTTP_ADDRESS_KEY, ksmHost + ":" + ksmHttpPort);
|
config.set(OZONE_OM_HTTP_ADDRESS_KEY, omHost + ":" + omHttpPort);
|
||||||
return getRestClient(config);
|
return getRestClient(config);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -49,7 +49,7 @@ public class OzoneKey {
|
||||||
private long modificationTime;
|
private long modificationTime;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructs OzoneKey from KsmKeyInfo.
|
* Constructs OzoneKey from OmKeyInfo.
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
public OzoneKey(String volumeName, String bucketName,
|
public OzoneKey(String volumeName, String bucketName,
|
||||||
|
|
|
@ -77,7 +77,7 @@ public final class VolumeArgs {
|
||||||
return acls;
|
return acls;
|
||||||
}
|
}
|
||||||
/**
|
/**
|
||||||
* Returns new builder class that builds a KsmVolumeArgs.
|
* Returns new builder class that builds a OmVolumeArgs.
|
||||||
*
|
*
|
||||||
* @return Builder
|
* @return Builder
|
||||||
*/
|
*/
|
||||||
|
@ -86,7 +86,7 @@ public final class VolumeArgs {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Builder for KsmVolumeArgs.
|
* Builder for OmVolumeArgs.
|
||||||
*/
|
*/
|
||||||
public static class Builder {
|
public static class Builder {
|
||||||
private String adminName;
|
private String adminName;
|
||||||
|
|
|
@ -23,8 +23,8 @@ import org.apache.hadoop.fs.Seekable;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
||||||
import org.apache.hadoop.hdds.client.BlockID;
|
import org.apache.hadoop.hdds.client.BlockID;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
||||||
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
||||||
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
|
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
|
||||||
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
|
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
|
||||||
|
@ -255,28 +255,29 @@ public class ChunkGroupInputStream extends InputStream implements Seekable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static LengthInputStream getFromKsmKeyInfo(KsmKeyInfo keyInfo,
|
public static LengthInputStream getFromOmKeyInfo(
|
||||||
|
OmKeyInfo keyInfo,
|
||||||
XceiverClientManager xceiverClientManager,
|
XceiverClientManager xceiverClientManager,
|
||||||
StorageContainerLocationProtocolClientSideTranslatorPB
|
StorageContainerLocationProtocolClientSideTranslatorPB
|
||||||
storageContainerLocationClient, String requestId)
|
storageContainerLocationClient,
|
||||||
throws IOException {
|
String requestId) throws IOException {
|
||||||
long length = 0;
|
long length = 0;
|
||||||
long containerKey;
|
long containerKey;
|
||||||
ChunkGroupInputStream groupInputStream = new ChunkGroupInputStream();
|
ChunkGroupInputStream groupInputStream = new ChunkGroupInputStream();
|
||||||
groupInputStream.key = keyInfo.getKeyName();
|
groupInputStream.key = keyInfo.getKeyName();
|
||||||
List<KsmKeyLocationInfo> keyLocationInfos =
|
List<OmKeyLocationInfo> keyLocationInfos =
|
||||||
keyInfo.getLatestVersionLocations().getBlocksLatestVersionOnly();
|
keyInfo.getLatestVersionLocations().getBlocksLatestVersionOnly();
|
||||||
groupInputStream.streamOffset = new long[keyLocationInfos.size()];
|
groupInputStream.streamOffset = new long[keyLocationInfos.size()];
|
||||||
for (int i = 0; i < keyLocationInfos.size(); i++) {
|
for (int i = 0; i < keyLocationInfos.size(); i++) {
|
||||||
KsmKeyLocationInfo ksmKeyLocationInfo = keyLocationInfos.get(i);
|
OmKeyLocationInfo omKeyLocationInfo = keyLocationInfos.get(i);
|
||||||
BlockID blockID = ksmKeyLocationInfo.getBlockID();
|
BlockID blockID = omKeyLocationInfo.getBlockID();
|
||||||
long containerID = blockID.getContainerID();
|
long containerID = blockID.getContainerID();
|
||||||
ContainerWithPipeline containerWithPipeline =
|
ContainerWithPipeline containerWithPipeline =
|
||||||
storageContainerLocationClient.getContainerWithPipeline(containerID);
|
storageContainerLocationClient.getContainerWithPipeline(containerID);
|
||||||
XceiverClientSpi xceiverClient = xceiverClientManager
|
XceiverClientSpi xceiverClient = xceiverClientManager
|
||||||
.acquireClient(containerWithPipeline.getPipeline(), containerID);
|
.acquireClient(containerWithPipeline.getPipeline(), containerID);
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
containerKey = ksmKeyLocationInfo.getLocalID();
|
containerKey = omKeyLocationInfo.getLocalID();
|
||||||
try {
|
try {
|
||||||
LOG.debug("get key accessing {} {}",
|
LOG.debug("get key accessing {} {}",
|
||||||
containerID, containerKey);
|
containerID, containerKey);
|
||||||
|
@ -292,11 +293,10 @@ public class ChunkGroupInputStream extends InputStream implements Seekable {
|
||||||
}
|
}
|
||||||
success = true;
|
success = true;
|
||||||
ChunkInputStream inputStream = new ChunkInputStream(
|
ChunkInputStream inputStream = new ChunkInputStream(
|
||||||
ksmKeyLocationInfo.getBlockID(), xceiverClientManager,
|
omKeyLocationInfo.getBlockID(), xceiverClientManager, xceiverClient,
|
||||||
xceiverClient,
|
|
||||||
chunks, requestId);
|
chunks, requestId);
|
||||||
groupInputStream.addStream(inputStream,
|
groupInputStream.addStream(inputStream,
|
||||||
ksmKeyLocationInfo.getLength());
|
omKeyLocationInfo.getLength());
|
||||||
} finally {
|
} finally {
|
||||||
if (!success) {
|
if (!success) {
|
||||||
xceiverClientManager.releaseClient(xceiverClient);
|
xceiverClientManager.releaseClient(xceiverClient);
|
||||||
|
|
|
@ -24,15 +24,15 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.client.BlockID;
|
import org.apache.hadoop.hdds.client.BlockID;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfoGroup;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ObjectStageChangeRequestProto;
|
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ObjectStageChangeRequestProto;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession;
|
import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
|
||||||
import org.apache.hadoop.ozone.ksm.protocolPB.KeySpaceManagerProtocolClientSideTranslatorPB;
|
import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolClientSideTranslatorPB;
|
||||||
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
||||||
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
|
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers
|
import org.apache.hadoop.hdds.scm.container.common.helpers
|
||||||
|
@ -67,10 +67,10 @@ public class ChunkGroupOutputStream extends OutputStream {
|
||||||
private final ArrayList<ChunkOutputStreamEntry> streamEntries;
|
private final ArrayList<ChunkOutputStreamEntry> streamEntries;
|
||||||
private int currentStreamIndex;
|
private int currentStreamIndex;
|
||||||
private long byteOffset;
|
private long byteOffset;
|
||||||
private final KeySpaceManagerProtocolClientSideTranslatorPB ksmClient;
|
private final OzoneManagerProtocolClientSideTranslatorPB omClient;
|
||||||
private final
|
private final
|
||||||
StorageContainerLocationProtocolClientSideTranslatorPB scmClient;
|
StorageContainerLocationProtocolClientSideTranslatorPB scmClient;
|
||||||
private final KsmKeyArgs keyArgs;
|
private final OmKeyArgs keyArgs;
|
||||||
private final int openID;
|
private final int openID;
|
||||||
private final XceiverClientManager xceiverClientManager;
|
private final XceiverClientManager xceiverClientManager;
|
||||||
private final int chunkSize;
|
private final int chunkSize;
|
||||||
|
@ -83,7 +83,7 @@ public class ChunkGroupOutputStream extends OutputStream {
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
public ChunkGroupOutputStream() {
|
public ChunkGroupOutputStream() {
|
||||||
streamEntries = new ArrayList<>();
|
streamEntries = new ArrayList<>();
|
||||||
ksmClient = null;
|
omClient = null;
|
||||||
scmClient = null;
|
scmClient = null;
|
||||||
keyArgs = null;
|
keyArgs = null;
|
||||||
openID = -1;
|
openID = -1;
|
||||||
|
@ -113,16 +113,16 @@ public class ChunkGroupOutputStream extends OutputStream {
|
||||||
public ChunkGroupOutputStream(
|
public ChunkGroupOutputStream(
|
||||||
OpenKeySession handler, XceiverClientManager xceiverClientManager,
|
OpenKeySession handler, XceiverClientManager xceiverClientManager,
|
||||||
StorageContainerLocationProtocolClientSideTranslatorPB scmClient,
|
StorageContainerLocationProtocolClientSideTranslatorPB scmClient,
|
||||||
KeySpaceManagerProtocolClientSideTranslatorPB ksmClient,
|
OzoneManagerProtocolClientSideTranslatorPB omClient,
|
||||||
int chunkSize, String requestId, ReplicationFactor factor,
|
int chunkSize, String requestId, ReplicationFactor factor,
|
||||||
ReplicationType type) throws IOException {
|
ReplicationType type) throws IOException {
|
||||||
this.streamEntries = new ArrayList<>();
|
this.streamEntries = new ArrayList<>();
|
||||||
this.currentStreamIndex = 0;
|
this.currentStreamIndex = 0;
|
||||||
this.byteOffset = 0;
|
this.byteOffset = 0;
|
||||||
this.ksmClient = ksmClient;
|
this.omClient = omClient;
|
||||||
this.scmClient = scmClient;
|
this.scmClient = scmClient;
|
||||||
KsmKeyInfo info = handler.getKeyInfo();
|
OmKeyInfo info = handler.getKeyInfo();
|
||||||
this.keyArgs = new KsmKeyArgs.Builder()
|
this.keyArgs = new OmKeyArgs.Builder()
|
||||||
.setVolumeName(info.getVolumeName())
|
.setVolumeName(info.getVolumeName())
|
||||||
.setBucketName(info.getBucketName())
|
.setBucketName(info.getBucketName())
|
||||||
.setKeyName(info.getKeyName())
|
.setKeyName(info.getKeyName())
|
||||||
|
@ -150,19 +150,19 @@ public class ChunkGroupOutputStream extends OutputStream {
|
||||||
* @param openVersion the version corresponding to the pre-allocation.
|
* @param openVersion the version corresponding to the pre-allocation.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public void addPreallocateBlocks(KsmKeyLocationInfoGroup version,
|
public void addPreallocateBlocks(OmKeyLocationInfoGroup version,
|
||||||
long openVersion) throws IOException {
|
long openVersion) throws IOException {
|
||||||
// server may return any number of blocks, (0 to any)
|
// server may return any number of blocks, (0 to any)
|
||||||
// only the blocks allocated in this open session (block createVersion
|
// only the blocks allocated in this open session (block createVersion
|
||||||
// equals to open session version)
|
// equals to open session version)
|
||||||
for (KsmKeyLocationInfo subKeyInfo : version.getLocationList()) {
|
for (OmKeyLocationInfo subKeyInfo : version.getLocationList()) {
|
||||||
if (subKeyInfo.getCreateVersion() == openVersion) {
|
if (subKeyInfo.getCreateVersion() == openVersion) {
|
||||||
checkKeyLocationInfo(subKeyInfo);
|
checkKeyLocationInfo(subKeyInfo);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void checkKeyLocationInfo(KsmKeyLocationInfo subKeyInfo)
|
private void checkKeyLocationInfo(OmKeyLocationInfo subKeyInfo)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
ContainerWithPipeline containerWithPipeline = scmClient
|
ContainerWithPipeline containerWithPipeline = scmClient
|
||||||
.getContainerWithPipeline(subKeyInfo.getContainerID());
|
.getContainerWithPipeline(subKeyInfo.getContainerID());
|
||||||
|
@ -210,7 +210,7 @@ public class ChunkGroupOutputStream extends OutputStream {
|
||||||
checkNotClosed();
|
checkNotClosed();
|
||||||
|
|
||||||
if (streamEntries.size() <= currentStreamIndex) {
|
if (streamEntries.size() <= currentStreamIndex) {
|
||||||
Preconditions.checkNotNull(ksmClient);
|
Preconditions.checkNotNull(omClient);
|
||||||
// allocate a new block, if a exception happens, log an error and
|
// allocate a new block, if a exception happens, log an error and
|
||||||
// throw exception to the caller directly, and the write fails.
|
// throw exception to the caller directly, and the write fails.
|
||||||
try {
|
try {
|
||||||
|
@ -258,7 +258,7 @@ public class ChunkGroupOutputStream extends OutputStream {
|
||||||
int succeededAllocates = 0;
|
int succeededAllocates = 0;
|
||||||
while (len > 0) {
|
while (len > 0) {
|
||||||
if (streamEntries.size() <= currentStreamIndex) {
|
if (streamEntries.size() <= currentStreamIndex) {
|
||||||
Preconditions.checkNotNull(ksmClient);
|
Preconditions.checkNotNull(omClient);
|
||||||
// allocate a new block, if a exception happens, log an error and
|
// allocate a new block, if a exception happens, log an error and
|
||||||
// throw exception to the caller directly, and the write fails.
|
// throw exception to the caller directly, and the write fails.
|
||||||
try {
|
try {
|
||||||
|
@ -286,7 +286,7 @@ public class ChunkGroupOutputStream extends OutputStream {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Contact KSM to get a new block. Set the new block with the index (e.g.
|
* Contact OM to get a new block. Set the new block with the index (e.g.
|
||||||
* first block has index = 0, second has index = 1 etc.)
|
* first block has index = 0, second has index = 1 etc.)
|
||||||
*
|
*
|
||||||
* The returned block is made to new ChunkOutputStreamEntry to write.
|
* The returned block is made to new ChunkOutputStreamEntry to write.
|
||||||
|
@ -295,7 +295,7 @@ public class ChunkGroupOutputStream extends OutputStream {
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
private void allocateNewBlock(int index) throws IOException {
|
private void allocateNewBlock(int index) throws IOException {
|
||||||
KsmKeyLocationInfo subKeyInfo = ksmClient.allocateBlock(keyArgs, openID);
|
OmKeyLocationInfo subKeyInfo = omClient.allocateBlock(keyArgs, openID);
|
||||||
checkKeyLocationInfo(subKeyInfo);
|
checkKeyLocationInfo(subKeyInfo);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -311,7 +311,7 @@ public class ChunkGroupOutputStream extends OutputStream {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Commit the key to KSM, this will add the blocks as the new key blocks.
|
* Commit the key to OM, this will add the blocks as the new key blocks.
|
||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
|
@ -329,7 +329,7 @@ public class ChunkGroupOutputStream extends OutputStream {
|
||||||
if (keyArgs != null) {
|
if (keyArgs != null) {
|
||||||
// in test, this could be null
|
// in test, this could be null
|
||||||
keyArgs.setDataSize(byteOffset);
|
keyArgs.setDataSize(byteOffset);
|
||||||
ksmClient.commitKey(keyArgs, openID);
|
omClient.commitKey(keyArgs, openID);
|
||||||
} else {
|
} else {
|
||||||
LOG.warn("Closing ChunkGroupOutputStream, but key args is null");
|
LOG.warn("Closing ChunkGroupOutputStream, but key args is null");
|
||||||
}
|
}
|
||||||
|
@ -342,7 +342,7 @@ public class ChunkGroupOutputStream extends OutputStream {
|
||||||
private OpenKeySession openHandler;
|
private OpenKeySession openHandler;
|
||||||
private XceiverClientManager xceiverManager;
|
private XceiverClientManager xceiverManager;
|
||||||
private StorageContainerLocationProtocolClientSideTranslatorPB scmClient;
|
private StorageContainerLocationProtocolClientSideTranslatorPB scmClient;
|
||||||
private KeySpaceManagerProtocolClientSideTranslatorPB ksmClient;
|
private OzoneManagerProtocolClientSideTranslatorPB omClient;
|
||||||
private int chunkSize;
|
private int chunkSize;
|
||||||
private String requestID;
|
private String requestID;
|
||||||
private ReplicationType type;
|
private ReplicationType type;
|
||||||
|
@ -364,9 +364,9 @@ public class ChunkGroupOutputStream extends OutputStream {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Builder setKsmClient(
|
public Builder setOmClient(
|
||||||
KeySpaceManagerProtocolClientSideTranslatorPB client) {
|
OzoneManagerProtocolClientSideTranslatorPB client) {
|
||||||
this.ksmClient = client;
|
this.omClient = client;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -392,7 +392,7 @@ public class ChunkGroupOutputStream extends OutputStream {
|
||||||
|
|
||||||
public ChunkGroupOutputStream build() throws IOException {
|
public ChunkGroupOutputStream build() throws IOException {
|
||||||
return new ChunkGroupOutputStream(openHandler, xceiverManager, scmClient,
|
return new ChunkGroupOutputStream(openHandler, xceiverManager, scmClient,
|
||||||
ksmClient, chunkSize, requestID, factor, type);
|
omClient, chunkSize, requestID, factor, type);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -18,7 +18,7 @@
|
||||||
|
|
||||||
package org.apache.hadoop.ozone.client.rest;
|
package org.apache.hadoop.ozone.client.rest;
|
||||||
|
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
|
import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
|
|
@ -45,10 +45,9 @@ import org.apache.hadoop.ozone.client.rest.headers.Header;
|
||||||
import org.apache.hadoop.ozone.client.rest.response.BucketInfo;
|
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.KeyInfo;
|
||||||
import org.apache.hadoop.ozone.client.rest.response.VolumeInfo;
|
import org.apache.hadoop.ozone.client.rest.response.VolumeInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
|
import org.apache.hadoop.ozone.om.OMConfigKeys;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
|
import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServicePort;
|
||||||
.KeySpaceManagerProtocolProtos.ServicePort;
|
|
||||||
import org.apache.hadoop.ozone.web.response.ListBuckets;
|
import org.apache.hadoop.ozone.web.response.ListBuckets;
|
||||||
import org.apache.hadoop.ozone.web.response.ListKeys;
|
import org.apache.hadoop.ozone.web.response.ListKeys;
|
||||||
import org.apache.hadoop.ozone.web.response.ListVolumes;
|
import org.apache.hadoop.ozone.web.response.ListVolumes;
|
||||||
|
@ -152,8 +151,8 @@ public class RestClient implements ClientProtocol {
|
||||||
.build())
|
.build())
|
||||||
.build();
|
.build();
|
||||||
this.ugi = UserGroupInformation.getCurrentUser();
|
this.ugi = UserGroupInformation.getCurrentUser();
|
||||||
this.userRights = conf.getEnum(KSMConfigKeys.OZONE_KSM_USER_RIGHTS,
|
this.userRights = conf.getEnum(OMConfigKeys.OZONE_OM_USER_RIGHTS,
|
||||||
KSMConfigKeys.OZONE_KSM_USER_RIGHTS_DEFAULT);
|
OMConfigKeys.OZONE_OM_USER_RIGHTS_DEFAULT);
|
||||||
|
|
||||||
// TODO: Add new configuration parameter to configure RestServerSelector.
|
// TODO: Add new configuration parameter to configure RestServerSelector.
|
||||||
RestServerSelector defaultSelector = new DefaultRestServerSelector();
|
RestServerSelector defaultSelector = new DefaultRestServerSelector();
|
||||||
|
@ -171,11 +170,11 @@ public class RestClient implements ClientProtocol {
|
||||||
|
|
||||||
private InetSocketAddress getOzoneRestServerAddress(
|
private InetSocketAddress getOzoneRestServerAddress(
|
||||||
RestServerSelector selector) throws IOException {
|
RestServerSelector selector) throws IOException {
|
||||||
String httpAddress = conf.get(KSMConfigKeys.OZONE_KSM_HTTP_ADDRESS_KEY);
|
String httpAddress = conf.get(OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY);
|
||||||
|
|
||||||
if (httpAddress == null) {
|
if (httpAddress == null) {
|
||||||
throw new IllegalArgumentException(
|
throw new IllegalArgumentException(
|
||||||
KSMConfigKeys.OZONE_KSM_HTTP_ADDRESS_KEY + " must be defined. See" +
|
OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY + " must be defined. See" +
|
||||||
" https://wiki.apache.org/hadoop/Ozone#Configuration for" +
|
" https://wiki.apache.org/hadoop/Ozone#Configuration for" +
|
||||||
" details on configuring Ozone.");
|
" details on configuring Ozone.");
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,7 +18,7 @@
|
||||||
|
|
||||||
package org.apache.hadoop.ozone.client.rest;
|
package org.apache.hadoop.ozone.client.rest;
|
||||||
|
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
|
import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
|
|
@ -27,7 +27,7 @@ import org.apache.hadoop.io.IOUtils;
|
||||||
import org.apache.hadoop.ipc.Client;
|
import org.apache.hadoop.ipc.Client;
|
||||||
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
||||||
import org.apache.hadoop.ipc.RPC;
|
import org.apache.hadoop.ipc.RPC;
|
||||||
import org.apache.hadoop.ozone.KsmUtils;
|
import org.apache.hadoop.ozone.OmUtils;
|
||||||
import org.apache.hadoop.ozone.OzoneConsts;
|
import org.apache.hadoop.ozone.OzoneConsts;
|
||||||
import org.apache.hadoop.ozone.client.BucketArgs;
|
import org.apache.hadoop.ozone.client.BucketArgs;
|
||||||
import org.apache.hadoop.ozone.client.OzoneBucket;
|
import org.apache.hadoop.ozone.client.OzoneBucket;
|
||||||
|
@ -43,24 +43,22 @@ import org.apache.hadoop.ozone.client.io.LengthInputStream;
|
||||||
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
|
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
|
||||||
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
||||||
import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
|
import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession;
|
import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
|
import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.protocolPB
|
import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolClientSideTranslatorPB;
|
||||||
.KeySpaceManagerProtocolClientSideTranslatorPB;
|
import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB;
|
||||||
import org.apache.hadoop.ozone.ksm.protocolPB
|
|
||||||
.KeySpaceManagerProtocolPB;
|
|
||||||
import org.apache.hadoop.net.NetUtils;
|
import org.apache.hadoop.net.NetUtils;
|
||||||
import org.apache.hadoop.ozone.OzoneAcl;
|
import org.apache.hadoop.ozone.OzoneAcl;
|
||||||
import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
|
import org.apache.hadoop.ozone.om.OMConfigKeys;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.ServicePort;
|
.OzoneManagerProtocolProtos.ServicePort;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
|
import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
|
||||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||||
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
||||||
import org.apache.hadoop.hdds.scm.protocolPB
|
import org.apache.hadoop.hdds.scm.protocolPB
|
||||||
|
@ -80,7 +78,7 @@ import java.util.UUID;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Ozone RPC Client Implementation, it connects to KSM, SCM and DataNode
|
* Ozone RPC Client Implementation, it connects to OM, SCM and DataNode
|
||||||
* to execute client calls. This uses RPC protocol for communication
|
* to execute client calls. This uses RPC protocol for communication
|
||||||
* with the servers.
|
* with the servers.
|
||||||
*/
|
*/
|
||||||
|
@ -92,8 +90,8 @@ public class RpcClient implements ClientProtocol {
|
||||||
private final OzoneConfiguration conf;
|
private final OzoneConfiguration conf;
|
||||||
private final StorageContainerLocationProtocolClientSideTranslatorPB
|
private final StorageContainerLocationProtocolClientSideTranslatorPB
|
||||||
storageContainerLocationClient;
|
storageContainerLocationClient;
|
||||||
private final KeySpaceManagerProtocolClientSideTranslatorPB
|
private final OzoneManagerProtocolClientSideTranslatorPB
|
||||||
keySpaceManagerClient;
|
ozoneManagerClient;
|
||||||
private final XceiverClientManager xceiverClientManager;
|
private final XceiverClientManager xceiverClientManager;
|
||||||
private final int chunkSize;
|
private final int chunkSize;
|
||||||
private final UserGroupInformation ugi;
|
private final UserGroupInformation ugi;
|
||||||
|
@ -109,20 +107,20 @@ public class RpcClient implements ClientProtocol {
|
||||||
Preconditions.checkNotNull(conf);
|
Preconditions.checkNotNull(conf);
|
||||||
this.conf = new OzoneConfiguration(conf);
|
this.conf = new OzoneConfiguration(conf);
|
||||||
this.ugi = UserGroupInformation.getCurrentUser();
|
this.ugi = UserGroupInformation.getCurrentUser();
|
||||||
this.userRights = conf.getEnum(KSMConfigKeys.OZONE_KSM_USER_RIGHTS,
|
this.userRights = conf.getEnum(OMConfigKeys.OZONE_OM_USER_RIGHTS,
|
||||||
KSMConfigKeys.OZONE_KSM_USER_RIGHTS_DEFAULT);
|
OMConfigKeys.OZONE_OM_USER_RIGHTS_DEFAULT);
|
||||||
this.groupRights = conf.getEnum(KSMConfigKeys.OZONE_KSM_GROUP_RIGHTS,
|
this.groupRights = conf.getEnum(OMConfigKeys.OZONE_OM_GROUP_RIGHTS,
|
||||||
KSMConfigKeys.OZONE_KSM_GROUP_RIGHTS_DEFAULT);
|
OMConfigKeys.OZONE_OM_GROUP_RIGHTS_DEFAULT);
|
||||||
long ksmVersion =
|
long omVersion =
|
||||||
RPC.getProtocolVersion(KeySpaceManagerProtocolPB.class);
|
RPC.getProtocolVersion(OzoneManagerProtocolPB.class);
|
||||||
InetSocketAddress ksmAddress = KsmUtils
|
InetSocketAddress omAddress = OmUtils
|
||||||
.getKsmAddressForClients(conf);
|
.getOmAddressForClients(conf);
|
||||||
RPC.setProtocolEngine(conf, KeySpaceManagerProtocolPB.class,
|
RPC.setProtocolEngine(conf, OzoneManagerProtocolPB.class,
|
||||||
ProtobufRpcEngine.class);
|
ProtobufRpcEngine.class);
|
||||||
this.keySpaceManagerClient =
|
this.ozoneManagerClient =
|
||||||
new KeySpaceManagerProtocolClientSideTranslatorPB(
|
new OzoneManagerProtocolClientSideTranslatorPB(
|
||||||
RPC.getProxy(KeySpaceManagerProtocolPB.class, ksmVersion,
|
RPC.getProxy(OzoneManagerProtocolPB.class, omVersion,
|
||||||
ksmAddress, UserGroupInformation.getCurrentUser(), conf,
|
omAddress, UserGroupInformation.getCurrentUser(), conf,
|
||||||
NetUtils.getDefaultSocketFactory(conf),
|
NetUtils.getDefaultSocketFactory(conf),
|
||||||
Client.getRpcTimeout(conf)));
|
Client.getRpcTimeout(conf)));
|
||||||
|
|
||||||
|
@ -155,7 +153,7 @@ public class RpcClient implements ClientProtocol {
|
||||||
}
|
}
|
||||||
|
|
||||||
private InetSocketAddress getScmAddressForClient() throws IOException {
|
private InetSocketAddress getScmAddressForClient() throws IOException {
|
||||||
List<ServiceInfo> services = keySpaceManagerClient.getServiceList();
|
List<ServiceInfo> services = ozoneManagerClient.getServiceList();
|
||||||
ServiceInfo scmInfo = services.stream().filter(
|
ServiceInfo scmInfo = services.stream().filter(
|
||||||
a -> a.getNodeType().equals(HddsProtos.NodeType.SCM))
|
a -> a.getNodeType().equals(HddsProtos.NodeType.SCM))
|
||||||
.collect(Collectors.toList()).get(0);
|
.collect(Collectors.toList()).get(0);
|
||||||
|
@ -195,7 +193,7 @@ public class RpcClient implements ClientProtocol {
|
||||||
listOfAcls.addAll(volArgs.getAcls());
|
listOfAcls.addAll(volArgs.getAcls());
|
||||||
}
|
}
|
||||||
|
|
||||||
KsmVolumeArgs.Builder builder = KsmVolumeArgs.newBuilder();
|
OmVolumeArgs.Builder builder = OmVolumeArgs.newBuilder();
|
||||||
builder.setVolume(volumeName);
|
builder.setVolume(volumeName);
|
||||||
builder.setAdminName(admin);
|
builder.setAdminName(admin);
|
||||||
builder.setOwnerName(owner);
|
builder.setOwnerName(owner);
|
||||||
|
@ -204,12 +202,12 @@ public class RpcClient implements ClientProtocol {
|
||||||
//Remove duplicates and add ACLs
|
//Remove duplicates and add ACLs
|
||||||
for (OzoneAcl ozoneAcl :
|
for (OzoneAcl ozoneAcl :
|
||||||
listOfAcls.stream().distinct().collect(Collectors.toList())) {
|
listOfAcls.stream().distinct().collect(Collectors.toList())) {
|
||||||
builder.addOzoneAcls(KSMPBHelper.convertOzoneAcl(ozoneAcl));
|
builder.addOzoneAcls(OMPBHelper.convertOzoneAcl(ozoneAcl));
|
||||||
}
|
}
|
||||||
|
|
||||||
LOG.info("Creating Volume: {}, with {} as owner and quota set to {} bytes.",
|
LOG.info("Creating Volume: {}, with {} as owner and quota set to {} bytes.",
|
||||||
volumeName, owner, quota);
|
volumeName, owner, quota);
|
||||||
keySpaceManagerClient.createVolume(builder.build());
|
ozoneManagerClient.createVolume(builder.build());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -217,7 +215,7 @@ public class RpcClient implements ClientProtocol {
|
||||||
throws IOException {
|
throws IOException {
|
||||||
HddsClientUtils.verifyResourceName(volumeName);
|
HddsClientUtils.verifyResourceName(volumeName);
|
||||||
Preconditions.checkNotNull(owner);
|
Preconditions.checkNotNull(owner);
|
||||||
keySpaceManagerClient.setOwner(volumeName, owner);
|
ozoneManagerClient.setOwner(volumeName, owner);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -226,14 +224,14 @@ public class RpcClient implements ClientProtocol {
|
||||||
HddsClientUtils.verifyResourceName(volumeName);
|
HddsClientUtils.verifyResourceName(volumeName);
|
||||||
Preconditions.checkNotNull(quota);
|
Preconditions.checkNotNull(quota);
|
||||||
long quotaInBytes = quota.sizeInBytes();
|
long quotaInBytes = quota.sizeInBytes();
|
||||||
keySpaceManagerClient.setQuota(volumeName, quotaInBytes);
|
ozoneManagerClient.setQuota(volumeName, quotaInBytes);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public OzoneVolume getVolumeDetails(String volumeName)
|
public OzoneVolume getVolumeDetails(String volumeName)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
HddsClientUtils.verifyResourceName(volumeName);
|
HddsClientUtils.verifyResourceName(volumeName);
|
||||||
KsmVolumeArgs volume = keySpaceManagerClient.getVolumeInfo(volumeName);
|
OmVolumeArgs volume = ozoneManagerClient.getVolumeInfo(volumeName);
|
||||||
return new OzoneVolume(
|
return new OzoneVolume(
|
||||||
conf,
|
conf,
|
||||||
this,
|
this,
|
||||||
|
@ -243,7 +241,7 @@ public class RpcClient implements ClientProtocol {
|
||||||
volume.getQuotaInBytes(),
|
volume.getQuotaInBytes(),
|
||||||
volume.getCreationTime(),
|
volume.getCreationTime(),
|
||||||
volume.getAclMap().ozoneAclGetProtobuf().stream().
|
volume.getAclMap().ozoneAclGetProtobuf().stream().
|
||||||
map(KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
|
map(OMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -255,14 +253,14 @@ public class RpcClient implements ClientProtocol {
|
||||||
@Override
|
@Override
|
||||||
public void deleteVolume(String volumeName) throws IOException {
|
public void deleteVolume(String volumeName) throws IOException {
|
||||||
HddsClientUtils.verifyResourceName(volumeName);
|
HddsClientUtils.verifyResourceName(volumeName);
|
||||||
keySpaceManagerClient.deleteVolume(volumeName);
|
ozoneManagerClient.deleteVolume(volumeName);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<OzoneVolume> listVolumes(String volumePrefix, String prevVolume,
|
public List<OzoneVolume> listVolumes(String volumePrefix, String prevVolume,
|
||||||
int maxListResult)
|
int maxListResult)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
List<KsmVolumeArgs> volumes = keySpaceManagerClient.listAllVolumes(
|
List<OmVolumeArgs> volumes = ozoneManagerClient.listAllVolumes(
|
||||||
volumePrefix, prevVolume, maxListResult);
|
volumePrefix, prevVolume, maxListResult);
|
||||||
|
|
||||||
return volumes.stream().map(volume -> new OzoneVolume(
|
return volumes.stream().map(volume -> new OzoneVolume(
|
||||||
|
@ -274,7 +272,7 @@ public class RpcClient implements ClientProtocol {
|
||||||
volume.getQuotaInBytes(),
|
volume.getQuotaInBytes(),
|
||||||
volume.getCreationTime(),
|
volume.getCreationTime(),
|
||||||
volume.getAclMap().ozoneAclGetProtobuf().stream().
|
volume.getAclMap().ozoneAclGetProtobuf().stream().
|
||||||
map(KSMPBHelper::convertOzoneAcl).collect(Collectors.toList())))
|
map(OMPBHelper::convertOzoneAcl).collect(Collectors.toList())))
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -282,7 +280,7 @@ public class RpcClient implements ClientProtocol {
|
||||||
public List<OzoneVolume> listVolumes(String user, String volumePrefix,
|
public List<OzoneVolume> listVolumes(String user, String volumePrefix,
|
||||||
String prevVolume, int maxListResult)
|
String prevVolume, int maxListResult)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
List<KsmVolumeArgs> volumes = keySpaceManagerClient.listVolumeByUser(
|
List<OmVolumeArgs> volumes = ozoneManagerClient.listVolumeByUser(
|
||||||
user, volumePrefix, prevVolume, maxListResult);
|
user, volumePrefix, prevVolume, maxListResult);
|
||||||
|
|
||||||
return volumes.stream().map(volume -> new OzoneVolume(
|
return volumes.stream().map(volume -> new OzoneVolume(
|
||||||
|
@ -294,7 +292,7 @@ public class RpcClient implements ClientProtocol {
|
||||||
volume.getQuotaInBytes(),
|
volume.getQuotaInBytes(),
|
||||||
volume.getCreationTime(),
|
volume.getCreationTime(),
|
||||||
volume.getAclMap().ozoneAclGetProtobuf().stream().
|
volume.getAclMap().ozoneAclGetProtobuf().stream().
|
||||||
map(KSMPBHelper::convertOzoneAcl).collect(Collectors.toList())))
|
map(OMPBHelper::convertOzoneAcl).collect(Collectors.toList())))
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -329,7 +327,7 @@ public class RpcClient implements ClientProtocol {
|
||||||
listOfAcls.addAll(bucketArgs.getAcls());
|
listOfAcls.addAll(bucketArgs.getAcls());
|
||||||
}
|
}
|
||||||
|
|
||||||
KsmBucketInfo.Builder builder = KsmBucketInfo.newBuilder();
|
OmBucketInfo.Builder builder = OmBucketInfo.newBuilder();
|
||||||
builder.setVolumeName(volumeName)
|
builder.setVolumeName(volumeName)
|
||||||
.setBucketName(bucketName)
|
.setBucketName(bucketName)
|
||||||
.setIsVersionEnabled(isVersionEnabled)
|
.setIsVersionEnabled(isVersionEnabled)
|
||||||
|
@ -339,7 +337,7 @@ public class RpcClient implements ClientProtocol {
|
||||||
LOG.info("Creating Bucket: {}/{}, with Versioning {} and " +
|
LOG.info("Creating Bucket: {}/{}, with Versioning {} and " +
|
||||||
"Storage Type set to {}", volumeName, bucketName, isVersionEnabled,
|
"Storage Type set to {}", volumeName, bucketName, isVersionEnabled,
|
||||||
storageType);
|
storageType);
|
||||||
keySpaceManagerClient.createBucket(builder.build());
|
ozoneManagerClient.createBucket(builder.build());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -348,11 +346,11 @@ public class RpcClient implements ClientProtocol {
|
||||||
throws IOException {
|
throws IOException {
|
||||||
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
||||||
Preconditions.checkNotNull(addAcls);
|
Preconditions.checkNotNull(addAcls);
|
||||||
KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
|
OmBucketArgs.Builder builder = OmBucketArgs.newBuilder();
|
||||||
builder.setVolumeName(volumeName)
|
builder.setVolumeName(volumeName)
|
||||||
.setBucketName(bucketName)
|
.setBucketName(bucketName)
|
||||||
.setAddAcls(addAcls);
|
.setAddAcls(addAcls);
|
||||||
keySpaceManagerClient.setBucketProperty(builder.build());
|
ozoneManagerClient.setBucketProperty(builder.build());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -361,11 +359,11 @@ public class RpcClient implements ClientProtocol {
|
||||||
throws IOException {
|
throws IOException {
|
||||||
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
||||||
Preconditions.checkNotNull(removeAcls);
|
Preconditions.checkNotNull(removeAcls);
|
||||||
KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
|
OmBucketArgs.Builder builder = OmBucketArgs.newBuilder();
|
||||||
builder.setVolumeName(volumeName)
|
builder.setVolumeName(volumeName)
|
||||||
.setBucketName(bucketName)
|
.setBucketName(bucketName)
|
||||||
.setRemoveAcls(removeAcls);
|
.setRemoveAcls(removeAcls);
|
||||||
keySpaceManagerClient.setBucketProperty(builder.build());
|
ozoneManagerClient.setBucketProperty(builder.build());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -374,11 +372,11 @@ public class RpcClient implements ClientProtocol {
|
||||||
throws IOException {
|
throws IOException {
|
||||||
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
||||||
Preconditions.checkNotNull(versioning);
|
Preconditions.checkNotNull(versioning);
|
||||||
KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
|
OmBucketArgs.Builder builder = OmBucketArgs.newBuilder();
|
||||||
builder.setVolumeName(volumeName)
|
builder.setVolumeName(volumeName)
|
||||||
.setBucketName(bucketName)
|
.setBucketName(bucketName)
|
||||||
.setIsVersionEnabled(versioning);
|
.setIsVersionEnabled(versioning);
|
||||||
keySpaceManagerClient.setBucketProperty(builder.build());
|
ozoneManagerClient.setBucketProperty(builder.build());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -387,18 +385,18 @@ public class RpcClient implements ClientProtocol {
|
||||||
throws IOException {
|
throws IOException {
|
||||||
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
||||||
Preconditions.checkNotNull(storageType);
|
Preconditions.checkNotNull(storageType);
|
||||||
KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
|
OmBucketArgs.Builder builder = OmBucketArgs.newBuilder();
|
||||||
builder.setVolumeName(volumeName)
|
builder.setVolumeName(volumeName)
|
||||||
.setBucketName(bucketName)
|
.setBucketName(bucketName)
|
||||||
.setStorageType(storageType);
|
.setStorageType(storageType);
|
||||||
keySpaceManagerClient.setBucketProperty(builder.build());
|
ozoneManagerClient.setBucketProperty(builder.build());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void deleteBucket(
|
public void deleteBucket(
|
||||||
String volumeName, String bucketName) throws IOException {
|
String volumeName, String bucketName) throws IOException {
|
||||||
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
||||||
keySpaceManagerClient.deleteBucket(volumeName, bucketName);
|
ozoneManagerClient.deleteBucket(volumeName, bucketName);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -411,8 +409,8 @@ public class RpcClient implements ClientProtocol {
|
||||||
public OzoneBucket getBucketDetails(
|
public OzoneBucket getBucketDetails(
|
||||||
String volumeName, String bucketName) throws IOException {
|
String volumeName, String bucketName) throws IOException {
|
||||||
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
||||||
KsmBucketInfo bucketArgs =
|
OmBucketInfo bucketArgs =
|
||||||
keySpaceManagerClient.getBucketInfo(volumeName, bucketName);
|
ozoneManagerClient.getBucketInfo(volumeName, bucketName);
|
||||||
return new OzoneBucket(
|
return new OzoneBucket(
|
||||||
conf,
|
conf,
|
||||||
this,
|
this,
|
||||||
|
@ -428,7 +426,7 @@ public class RpcClient implements ClientProtocol {
|
||||||
public List<OzoneBucket> listBuckets(String volumeName, String bucketPrefix,
|
public List<OzoneBucket> listBuckets(String volumeName, String bucketPrefix,
|
||||||
String prevBucket, int maxListResult)
|
String prevBucket, int maxListResult)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
List<KsmBucketInfo> buckets = keySpaceManagerClient.listBuckets(
|
List<OmBucketInfo> buckets = ozoneManagerClient.listBuckets(
|
||||||
volumeName, prevBucket, bucketPrefix, maxListResult);
|
volumeName, prevBucket, bucketPrefix, maxListResult);
|
||||||
|
|
||||||
return buckets.stream().map(bucket -> new OzoneBucket(
|
return buckets.stream().map(bucket -> new OzoneBucket(
|
||||||
|
@ -451,7 +449,7 @@ public class RpcClient implements ClientProtocol {
|
||||||
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
||||||
HddsClientUtils.checkNotNull(keyName, type, factor);
|
HddsClientUtils.checkNotNull(keyName, type, factor);
|
||||||
String requestId = UUID.randomUUID().toString();
|
String requestId = UUID.randomUUID().toString();
|
||||||
KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
|
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
|
||||||
.setVolumeName(volumeName)
|
.setVolumeName(volumeName)
|
||||||
.setBucketName(bucketName)
|
.setBucketName(bucketName)
|
||||||
.setKeyName(keyName)
|
.setKeyName(keyName)
|
||||||
|
@ -460,13 +458,13 @@ public class RpcClient implements ClientProtocol {
|
||||||
.setFactor(HddsProtos.ReplicationFactor.valueOf(factor.getValue()))
|
.setFactor(HddsProtos.ReplicationFactor.valueOf(factor.getValue()))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
OpenKeySession openKey = keySpaceManagerClient.openKey(keyArgs);
|
OpenKeySession openKey = ozoneManagerClient.openKey(keyArgs);
|
||||||
ChunkGroupOutputStream groupOutputStream =
|
ChunkGroupOutputStream groupOutputStream =
|
||||||
new ChunkGroupOutputStream.Builder()
|
new ChunkGroupOutputStream.Builder()
|
||||||
.setHandler(openKey)
|
.setHandler(openKey)
|
||||||
.setXceiverClientManager(xceiverClientManager)
|
.setXceiverClientManager(xceiverClientManager)
|
||||||
.setScmClient(storageContainerLocationClient)
|
.setScmClient(storageContainerLocationClient)
|
||||||
.setKsmClient(keySpaceManagerClient)
|
.setOmClient(ozoneManagerClient)
|
||||||
.setChunkSize(chunkSize)
|
.setChunkSize(chunkSize)
|
||||||
.setRequestID(requestId)
|
.setRequestID(requestId)
|
||||||
.setType(HddsProtos.ReplicationType.valueOf(type.toString()))
|
.setType(HddsProtos.ReplicationType.valueOf(type.toString()))
|
||||||
|
@ -485,14 +483,14 @@ public class RpcClient implements ClientProtocol {
|
||||||
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
||||||
Preconditions.checkNotNull(keyName);
|
Preconditions.checkNotNull(keyName);
|
||||||
String requestId = UUID.randomUUID().toString();
|
String requestId = UUID.randomUUID().toString();
|
||||||
KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
|
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
|
||||||
.setVolumeName(volumeName)
|
.setVolumeName(volumeName)
|
||||||
.setBucketName(bucketName)
|
.setBucketName(bucketName)
|
||||||
.setKeyName(keyName)
|
.setKeyName(keyName)
|
||||||
.build();
|
.build();
|
||||||
KsmKeyInfo keyInfo = keySpaceManagerClient.lookupKey(keyArgs);
|
OmKeyInfo keyInfo = ozoneManagerClient.lookupKey(keyArgs);
|
||||||
LengthInputStream lengthInputStream =
|
LengthInputStream lengthInputStream =
|
||||||
ChunkGroupInputStream.getFromKsmKeyInfo(
|
ChunkGroupInputStream.getFromOmKeyInfo(
|
||||||
keyInfo, xceiverClientManager, storageContainerLocationClient,
|
keyInfo, xceiverClientManager, storageContainerLocationClient,
|
||||||
requestId);
|
requestId);
|
||||||
return new OzoneInputStream(
|
return new OzoneInputStream(
|
||||||
|
@ -505,12 +503,12 @@ public class RpcClient implements ClientProtocol {
|
||||||
throws IOException {
|
throws IOException {
|
||||||
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
||||||
Preconditions.checkNotNull(keyName);
|
Preconditions.checkNotNull(keyName);
|
||||||
KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
|
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
|
||||||
.setVolumeName(volumeName)
|
.setVolumeName(volumeName)
|
||||||
.setBucketName(bucketName)
|
.setBucketName(bucketName)
|
||||||
.setKeyName(keyName)
|
.setKeyName(keyName)
|
||||||
.build();
|
.build();
|
||||||
keySpaceManagerClient.deleteKey(keyArgs);
|
ozoneManagerClient.deleteKey(keyArgs);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -518,12 +516,12 @@ public class RpcClient implements ClientProtocol {
|
||||||
String fromKeyName, String toKeyName) throws IOException {
|
String fromKeyName, String toKeyName) throws IOException {
|
||||||
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
HddsClientUtils.verifyResourceName(volumeName, bucketName);
|
||||||
HddsClientUtils.checkNotNull(fromKeyName, toKeyName);
|
HddsClientUtils.checkNotNull(fromKeyName, toKeyName);
|
||||||
KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
|
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
|
||||||
.setVolumeName(volumeName)
|
.setVolumeName(volumeName)
|
||||||
.setBucketName(bucketName)
|
.setBucketName(bucketName)
|
||||||
.setKeyName(fromKeyName)
|
.setKeyName(fromKeyName)
|
||||||
.build();
|
.build();
|
||||||
keySpaceManagerClient.renameKey(keyArgs, toKeyName);
|
ozoneManagerClient.renameKey(keyArgs, toKeyName);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -531,7 +529,7 @@ public class RpcClient implements ClientProtocol {
|
||||||
String keyPrefix, String prevKey,
|
String keyPrefix, String prevKey,
|
||||||
int maxListResult)
|
int maxListResult)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
List<KsmKeyInfo> keys = keySpaceManagerClient.listKeys(
|
List<OmKeyInfo> keys = ozoneManagerClient.listKeys(
|
||||||
volumeName, bucketName, prevKey, keyPrefix, maxListResult);
|
volumeName, bucketName, prevKey, keyPrefix, maxListResult);
|
||||||
|
|
||||||
return keys.stream().map(key -> new OzoneKey(
|
return keys.stream().map(key -> new OzoneKey(
|
||||||
|
@ -551,12 +549,12 @@ public class RpcClient implements ClientProtocol {
|
||||||
Preconditions.checkNotNull(volumeName);
|
Preconditions.checkNotNull(volumeName);
|
||||||
Preconditions.checkNotNull(bucketName);
|
Preconditions.checkNotNull(bucketName);
|
||||||
Preconditions.checkNotNull(keyName);
|
Preconditions.checkNotNull(keyName);
|
||||||
KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
|
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
|
||||||
.setVolumeName(volumeName)
|
.setVolumeName(volumeName)
|
||||||
.setBucketName(bucketName)
|
.setBucketName(bucketName)
|
||||||
.setKeyName(keyName)
|
.setKeyName(keyName)
|
||||||
.build();
|
.build();
|
||||||
KsmKeyInfo keyInfo = keySpaceManagerClient.lookupKey(keyArgs);
|
OmKeyInfo keyInfo = ozoneManagerClient.lookupKey(keyArgs);
|
||||||
return new OzoneKey(keyInfo.getVolumeName(),
|
return new OzoneKey(keyInfo.getVolumeName(),
|
||||||
keyInfo.getBucketName(),
|
keyInfo.getBucketName(),
|
||||||
keyInfo.getKeyName(),
|
keyInfo.getKeyName(),
|
||||||
|
@ -568,7 +566,7 @@ public class RpcClient implements ClientProtocol {
|
||||||
@Override
|
@Override
|
||||||
public void close() throws IOException {
|
public void close() throws IOException {
|
||||||
IOUtils.cleanupWithLogger(LOG, storageContainerLocationClient);
|
IOUtils.cleanupWithLogger(LOG, storageContainerLocationClient);
|
||||||
IOUtils.cleanupWithLogger(LOG, keySpaceManagerClient);
|
IOUtils.cleanupWithLogger(LOG, ozoneManagerClient);
|
||||||
IOUtils.cleanupWithLogger(LOG, xceiverClientManager);
|
IOUtils.cleanupWithLogger(LOG, xceiverClientManager);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.hadoop.ozone.client;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
|
import org.apache.hadoop.ozone.om.OMConfigKeys;
|
||||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -30,7 +30,7 @@ import org.junit.rules.Timeout;
|
||||||
import java.net.InetSocketAddress;
|
import java.net.InetSocketAddress;
|
||||||
|
|
||||||
import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
|
import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
|
||||||
import static org.apache.hadoop.ozone.KsmUtils.getKsmAddress;
|
import static org.apache.hadoop.ozone.OmUtils.getOmAddress;
|
||||||
import static org.hamcrest.core.Is.is;
|
import static org.hamcrest.core.Is.is;
|
||||||
import static org.junit.Assert.assertThat;
|
import static org.junit.Assert.assertThat;
|
||||||
|
|
||||||
|
@ -79,27 +79,27 @@ public class TestHddsClientUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testGetKSMAddress() {
|
public void testGetOmAddress() {
|
||||||
final Configuration conf = new OzoneConfiguration();
|
final Configuration conf = new OzoneConfiguration();
|
||||||
|
|
||||||
// First try a client address with just a host name. Verify it falls
|
// First try a client address with just a host name. Verify it falls
|
||||||
// back to the default port.
|
// back to the default port.
|
||||||
conf.set(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY, "1.2.3.4");
|
conf.set(OMConfigKeys.OZONE_OM_ADDRESS_KEY, "1.2.3.4");
|
||||||
InetSocketAddress addr = getKsmAddress(conf);
|
InetSocketAddress addr = getOmAddress(conf);
|
||||||
assertThat(addr.getHostString(), is("1.2.3.4"));
|
assertThat(addr.getHostString(), is("1.2.3.4"));
|
||||||
assertThat(addr.getPort(), is(KSMConfigKeys.OZONE_KSM_PORT_DEFAULT));
|
assertThat(addr.getPort(), is(OMConfigKeys.OZONE_OM_PORT_DEFAULT));
|
||||||
|
|
||||||
// Next try a client address with just a host name and port. Verify the port
|
// Next try a client address with just a host name and port. Verify the port
|
||||||
// is ignored and the default KSM port is used.
|
// is ignored and the default OM port is used.
|
||||||
conf.set(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY, "1.2.3.4:100");
|
conf.set(OMConfigKeys.OZONE_OM_ADDRESS_KEY, "1.2.3.4:100");
|
||||||
addr = getKsmAddress(conf);
|
addr = getOmAddress(conf);
|
||||||
assertThat(addr.getHostString(), is("1.2.3.4"));
|
assertThat(addr.getHostString(), is("1.2.3.4"));
|
||||||
assertThat(addr.getPort(), is(100));
|
assertThat(addr.getPort(), is(100));
|
||||||
|
|
||||||
// Assert the we are able to use default configs if no value is specified.
|
// Assert the we are able to use default configs if no value is specified.
|
||||||
conf.set(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY, "");
|
conf.set(OMConfigKeys.OZONE_OM_ADDRESS_KEY, "");
|
||||||
addr = getKsmAddress(conf);
|
addr = getOmAddress(conf);
|
||||||
assertThat(addr.getHostString(), is("0.0.0.0"));
|
assertThat(addr.getHostString(), is("0.0.0.0"));
|
||||||
assertThat(addr.getPort(), is(KSMConfigKeys.OZONE_KSM_PORT_DEFAULT));
|
assertThat(addr.getPort(), is(OMConfigKeys.OZONE_OM_PORT_DEFAULT));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -69,7 +69,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||||
<source>
|
<source>
|
||||||
<directory>${basedir}/src/main/proto</directory>
|
<directory>${basedir}/src/main/proto</directory>
|
||||||
<includes>
|
<includes>
|
||||||
<include>KeySpaceManagerProtocol.proto</include>
|
<include>OzoneManagerProtocol.proto</include>
|
||||||
</includes>
|
</includes>
|
||||||
</source>
|
</source>
|
||||||
</configuration>
|
</configuration>
|
||||||
|
|
|
@ -38,10 +38,9 @@ function hadoop_usage
|
||||||
hadoop_add_subcommand "envvars" client "display computed Hadoop environment variables"
|
hadoop_add_subcommand "envvars" client "display computed Hadoop environment variables"
|
||||||
hadoop_add_subcommand "freon" client "runs an ozone data generator"
|
hadoop_add_subcommand "freon" client "runs an ozone data generator"
|
||||||
hadoop_add_subcommand "genesis" client "runs a collection of ozone benchmarks to help with tuning."
|
hadoop_add_subcommand "genesis" client "runs a collection of ozone benchmarks to help with tuning."
|
||||||
hadoop_add_subcommand "getozoneconf" client "get ozone config values from
|
hadoop_add_subcommand "getozoneconf" client "get ozone config values from configuration"
|
||||||
configuration"
|
|
||||||
hadoop_add_subcommand "jmxget" admin "get JMX exported values from NameNode or DataNode."
|
hadoop_add_subcommand "jmxget" admin "get JMX exported values from NameNode or DataNode."
|
||||||
hadoop_add_subcommand "ksm" daemon "Ozone keyspace manager"
|
hadoop_add_subcommand "om" daemon "Ozone Manager"
|
||||||
hadoop_add_subcommand "o3" client "command line interface for ozone"
|
hadoop_add_subcommand "o3" client "command line interface for ozone"
|
||||||
hadoop_add_subcommand "noz" client "ozone debug tool, convert ozone metadata into relational data"
|
hadoop_add_subcommand "noz" client "ozone debug tool, convert ozone metadata into relational data"
|
||||||
hadoop_add_subcommand "scm" daemon "run the Storage Container Manager service"
|
hadoop_add_subcommand "scm" daemon "run the Storage Container Manager service"
|
||||||
|
@ -94,9 +93,9 @@ function ozonecmd_case
|
||||||
getozoneconf)
|
getozoneconf)
|
||||||
HADOOP_CLASSNAME=org.apache.hadoop.ozone.freon.OzoneGetConf;
|
HADOOP_CLASSNAME=org.apache.hadoop.ozone.freon.OzoneGetConf;
|
||||||
;;
|
;;
|
||||||
ksm)
|
om)
|
||||||
HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
|
HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
|
||||||
HADOOP_CLASSNAME=org.apache.hadoop.ozone.ksm.KeySpaceManager
|
HADOOP_CLASSNAME=org.apache.hadoop.ozone.om.OzoneManager
|
||||||
;;
|
;;
|
||||||
oz)
|
oz)
|
||||||
HADOOP_CLASSNAME=org.apache.hadoop.ozone.web.ozShell.Shell
|
HADOOP_CLASSNAME=org.apache.hadoop.ozone.web.ozShell.Shell
|
||||||
|
|
|
@ -179,19 +179,19 @@ if [[ "${AUTOHA_ENABLED}" = "true" ]]; then
|
||||||
fi
|
fi
|
||||||
|
|
||||||
#---------------------------------------------------------
|
#---------------------------------------------------------
|
||||||
# Ozone keyspacemanager nodes
|
# Ozone ozonemanager nodes
|
||||||
KSM_NODES=$("${HADOOP_HDFS_HOME}/bin/ozone" getozoneconf -keyspacemanagers 2>/dev/null)
|
OM_NODES=$("${HADOOP_HDFS_HOME}/bin/ozone" getozoneconf -ozonemanagers 2>/dev/null)
|
||||||
echo "Starting key space manager nodes [${KSM_NODES}]"
|
echo "Starting Ozone Manager nodes [${OM_NODES}]"
|
||||||
if [[ "${KSM_NODES}" == "0.0.0.0" ]]; then
|
if [[ "${OM_NODES}" == "0.0.0.0" ]]; then
|
||||||
KSM_NODES=$(hostname)
|
OM_NODES=$(hostname)
|
||||||
fi
|
fi
|
||||||
|
|
||||||
hadoop_uservar_su hdfs ksm "${HADOOP_HDFS_HOME}/bin/ozone" \
|
hadoop_uservar_su hdfs om "${HADOOP_HDFS_HOME}/bin/ozone" \
|
||||||
--workers \
|
--workers \
|
||||||
--config "${HADOOP_CONF_DIR}" \
|
--config "${HADOOP_CONF_DIR}" \
|
||||||
--hostnames "${KSM_NODES}" \
|
--hostnames "${OM_NODES}" \
|
||||||
--daemon start \
|
--daemon start \
|
||||||
ksm
|
om
|
||||||
|
|
||||||
HADOOP_JUMBO_RETCOUNTER=$?
|
HADOOP_JUMBO_RETCOUNTER=$?
|
||||||
|
|
||||||
|
|
|
@ -73,19 +73,19 @@ else
|
||||||
fi
|
fi
|
||||||
|
|
||||||
#---------------------------------------------------------
|
#---------------------------------------------------------
|
||||||
# Ozone keyspacemanager nodes
|
# Ozone Manager nodes
|
||||||
KSM_NODES=$("${HADOOP_HDFS_HOME}/bin/ozone" getozoneconf -keyspacemanagers 2>/dev/null)
|
OM_NODES=$("${HADOOP_HDFS_HOME}/bin/ozone" getozoneconf -ozonemanagers 2>/dev/null)
|
||||||
echo "Stopping key space manager nodes [${KSM_NODES}]"
|
echo "Stopping Ozone Manager nodes [${OM_NODES}]"
|
||||||
if [[ "${KSM_NODES}" == "0.0.0.0" ]]; then
|
if [[ "${OM_NODES}" == "0.0.0.0" ]]; then
|
||||||
KSM_NODES=$(hostname)
|
OM_NODES=$(hostname)
|
||||||
fi
|
fi
|
||||||
|
|
||||||
hadoop_uservar_su hdfs ksm "${HADOOP_HDFS_HOME}/bin/ozone" \
|
hadoop_uservar_su hdfs om "${HADOOP_HDFS_HOME}/bin/ozone" \
|
||||||
--workers \
|
--workers \
|
||||||
--config "${HADOOP_CONF_DIR}" \
|
--config "${HADOOP_CONF_DIR}" \
|
||||||
--hostnames "${KSM_NODES}" \
|
--hostnames "${OM_NODES}" \
|
||||||
--daemon stop \
|
--daemon stop \
|
||||||
ksm
|
om
|
||||||
|
|
||||||
#---------------------------------------------------------
|
#---------------------------------------------------------
|
||||||
# Ozone storagecontainermanager nodes
|
# Ozone storagecontainermanager nodes
|
||||||
|
|
|
@ -25,71 +25,70 @@ import org.apache.hadoop.net.NetUtils;
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import static org.apache.hadoop.hdds.HddsUtils.getHostNameFromConfigKeys;
|
import static org.apache.hadoop.hdds.HddsUtils.getHostNameFromConfigKeys;
|
||||||
import static org.apache.hadoop.hdds.HddsUtils.getPortNumberFromConfigKeys;
|
import static org.apache.hadoop.hdds.HddsUtils.getPortNumberFromConfigKeys;
|
||||||
import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_ADDRESS_KEY;
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
|
||||||
import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_HTTP_ADDRESS_KEY;
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_BIND_HOST_DEFAULT;
|
||||||
import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_HTTP_BIND_PORT_DEFAULT;
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY;
|
||||||
import static org.apache.hadoop.ozone.ksm.KSMConfigKeys
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_BIND_PORT_DEFAULT;
|
||||||
.OZONE_KSM_BIND_HOST_DEFAULT;
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_PORT_DEFAULT;
|
||||||
import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_PORT_DEFAULT;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Stateless helper functions for the server and client side of KSM
|
* Stateless helper functions for the server and client side of OM
|
||||||
* communication.
|
* communication.
|
||||||
*/
|
*/
|
||||||
public final class KsmUtils {
|
public final class OmUtils {
|
||||||
|
|
||||||
private KsmUtils() {
|
private OmUtils() {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Retrieve the socket address that is used by KSM.
|
* Retrieve the socket address that is used by OM.
|
||||||
* @param conf
|
* @param conf
|
||||||
* @return Target InetSocketAddress for the SCM service endpoint.
|
* @return Target InetSocketAddress for the SCM service endpoint.
|
||||||
*/
|
*/
|
||||||
public static InetSocketAddress getKsmAddress(
|
public static InetSocketAddress getOmAddress(
|
||||||
Configuration conf) {
|
Configuration conf) {
|
||||||
final Optional<String> host = getHostNameFromConfigKeys(conf,
|
final Optional<String> host = getHostNameFromConfigKeys(conf,
|
||||||
OZONE_KSM_ADDRESS_KEY);
|
OZONE_OM_ADDRESS_KEY);
|
||||||
|
|
||||||
return NetUtils.createSocketAddr(
|
return NetUtils.createSocketAddr(
|
||||||
host.or(OZONE_KSM_BIND_HOST_DEFAULT) + ":" +
|
host.or(OZONE_OM_BIND_HOST_DEFAULT) + ":" +
|
||||||
getKsmRpcPort(conf));
|
getOmRpcPort(conf));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Retrieve the socket address that should be used by clients to connect
|
* Retrieve the socket address that should be used by clients to connect
|
||||||
* to KSM.
|
* to OM.
|
||||||
* @param conf
|
* @param conf
|
||||||
* @return Target InetSocketAddress for the KSM service endpoint.
|
* @return Target InetSocketAddress for the OM service endpoint.
|
||||||
*/
|
*/
|
||||||
public static InetSocketAddress getKsmAddressForClients(
|
public static InetSocketAddress getOmAddressForClients(
|
||||||
Configuration conf) {
|
Configuration conf) {
|
||||||
final Optional<String> host = getHostNameFromConfigKeys(conf,
|
final Optional<String> host = getHostNameFromConfigKeys(conf,
|
||||||
OZONE_KSM_ADDRESS_KEY);
|
OZONE_OM_ADDRESS_KEY);
|
||||||
|
|
||||||
if (!host.isPresent()) {
|
if (!host.isPresent()) {
|
||||||
throw new IllegalArgumentException(
|
throw new IllegalArgumentException(
|
||||||
OZONE_KSM_ADDRESS_KEY + " must be defined. See" +
|
OZONE_OM_ADDRESS_KEY + " must be defined. See" +
|
||||||
" https://wiki.apache.org/hadoop/Ozone#Configuration for" +
|
" https://wiki.apache.org/hadoop/Ozone#Configuration for" +
|
||||||
" details on configuring Ozone.");
|
" details on configuring Ozone.");
|
||||||
}
|
}
|
||||||
|
|
||||||
return NetUtils.createSocketAddr(
|
return NetUtils.createSocketAddr(
|
||||||
host.get() + ":" + getKsmRpcPort(conf));
|
host.get() + ":" + getOmRpcPort(conf));
|
||||||
}
|
}
|
||||||
|
|
||||||
public static int getKsmRpcPort(Configuration conf) {
|
public static int getOmRpcPort(Configuration conf) {
|
||||||
// If no port number is specified then we'll just try the defaultBindPort.
|
// If no port number is specified then we'll just try the defaultBindPort.
|
||||||
final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
|
final Optional<Integer> port = getPortNumberFromConfigKeys(conf,
|
||||||
OZONE_KSM_ADDRESS_KEY);
|
OZONE_OM_ADDRESS_KEY);
|
||||||
return port.or(OZONE_KSM_PORT_DEFAULT);
|
return port.or(OZONE_OM_PORT_DEFAULT);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static int getKsmRestPort(Configuration conf) {
|
public static int getOmRestPort(Configuration conf) {
|
||||||
// If no port number is specified then we'll just try the default
|
// If no port number is specified then we'll just try the default
|
||||||
// HTTP BindPort.
|
// HTTP BindPort.
|
||||||
final Optional<Integer> port =
|
final Optional<Integer> port =
|
||||||
getPortNumberFromConfigKeys(conf, OZONE_KSM_HTTP_ADDRESS_KEY);
|
getPortNumberFromConfigKeys(conf, OZONE_OM_HTTP_ADDRESS_KEY);
|
||||||
return port.or(OZONE_KSM_HTTP_BIND_PORT_DEFAULT);
|
return port.or(OZONE_OM_HTTP_BIND_PORT_DEFAULT);
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -32,7 +32,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
|
||||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||||
import org.apache.hadoop.hdds.HddsUtils;
|
import org.apache.hadoop.hdds.HddsUtils;
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.ozone.KsmUtils;
|
import org.apache.hadoop.ozone.OmUtils;
|
||||||
import org.apache.hadoop.security.SecurityUtil;
|
import org.apache.hadoop.security.SecurityUtil;
|
||||||
import org.apache.hadoop.util.StringUtils;
|
import org.apache.hadoop.util.StringUtils;
|
||||||
import org.apache.hadoop.util.Tool;
|
import org.apache.hadoop.util.Tool;
|
||||||
|
@ -53,8 +53,8 @@ public class OzoneGetConf extends Configured implements Tool {
|
||||||
EXCLUDE_FILE("-excludeFile",
|
EXCLUDE_FILE("-excludeFile",
|
||||||
"gets the exclude file path that defines the datanodes " +
|
"gets the exclude file path that defines the datanodes " +
|
||||||
"that need to decommissioned."),
|
"that need to decommissioned."),
|
||||||
KEYSPACEMANAGER("-keyspacemanagers",
|
OZONEMANAGER("-ozonemanagers",
|
||||||
"gets list of ozone key space manager nodes in the cluster"),
|
"gets list of Ozone Manager nodes in the cluster"),
|
||||||
STORAGECONTAINERMANAGER("-storagecontainermanagers",
|
STORAGECONTAINERMANAGER("-storagecontainermanagers",
|
||||||
"gets list of ozone storage container manager nodes in the cluster"),
|
"gets list of ozone storage container manager nodes in the cluster"),
|
||||||
CONFKEY("-confKey [key]", "gets a specific key from the configuration");
|
CONFKEY("-confKey [key]", "gets a specific key from the configuration");
|
||||||
|
@ -63,8 +63,8 @@ public class OzoneGetConf extends Configured implements Tool {
|
||||||
|
|
||||||
static {
|
static {
|
||||||
HANDLERS = new HashMap<String, OzoneGetConf.CommandHandler>();
|
HANDLERS = new HashMap<String, OzoneGetConf.CommandHandler>();
|
||||||
HANDLERS.put(StringUtils.toLowerCase(KEYSPACEMANAGER.getName()),
|
HANDLERS.put(StringUtils.toLowerCase(OZONEMANAGER.getName()),
|
||||||
new KeySpaceManagersCommandHandler());
|
new OzoneManagersCommandHandler());
|
||||||
HANDLERS.put(StringUtils.toLowerCase(STORAGECONTAINERMANAGER.getName()),
|
HANDLERS.put(StringUtils.toLowerCase(STORAGECONTAINERMANAGER.getName()),
|
||||||
new StorageContainerManagersCommandHandler());
|
new StorageContainerManagersCommandHandler());
|
||||||
HANDLERS.put(StringUtils.toLowerCase(CONFKEY.getName()),
|
HANDLERS.put(StringUtils.toLowerCase(CONFKEY.getName()),
|
||||||
|
@ -245,13 +245,13 @@ public class OzoneGetConf extends Configured implements Tool {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Handler for {@link Command#KEYSPACEMANAGER}.
|
* Handler for {@link Command#OZONEMANAGER}.
|
||||||
*/
|
*/
|
||||||
static class KeySpaceManagersCommandHandler extends CommandHandler {
|
static class OzoneManagersCommandHandler extends CommandHandler {
|
||||||
@Override
|
@Override
|
||||||
public int doWorkInternal(OzoneGetConf tool, String[] args)
|
public int doWorkInternal(OzoneGetConf tool, String[] args)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
tool.printOut(KsmUtils.getKsmAddress(tool.getConf()).getHostName());
|
tool.printOut(OmUtils.getOmAddress(tool.getConf()).getHostName());
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,81 +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>
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* <p>
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
|
|
||||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
|
||||||
* License for the specific language governing permissions and limitations under
|
|
||||||
* the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.hadoop.ozone.ksm;
|
|
||||||
|
|
||||||
import org.apache.hadoop.ozone.OzoneAcl;
|
|
||||||
/**
|
|
||||||
* KSM Constants.
|
|
||||||
*/
|
|
||||||
public final class KSMConfigKeys {
|
|
||||||
/**
|
|
||||||
* Never constructed.
|
|
||||||
*/
|
|
||||||
private KSMConfigKeys() {
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
public static final String OZONE_KSM_HANDLER_COUNT_KEY =
|
|
||||||
"ozone.ksm.handler.count.key";
|
|
||||||
public static final int OZONE_KSM_HANDLER_COUNT_DEFAULT = 20;
|
|
||||||
|
|
||||||
public static final String OZONE_KSM_ADDRESS_KEY =
|
|
||||||
"ozone.ksm.address";
|
|
||||||
public static final String OZONE_KSM_BIND_HOST_DEFAULT =
|
|
||||||
"0.0.0.0";
|
|
||||||
public static final int OZONE_KSM_PORT_DEFAULT = 9862;
|
|
||||||
|
|
||||||
public static final String OZONE_KSM_HTTP_ENABLED_KEY =
|
|
||||||
"ozone.ksm.http.enabled";
|
|
||||||
public static final String OZONE_KSM_HTTP_BIND_HOST_KEY =
|
|
||||||
"ozone.ksm.http-bind-host";
|
|
||||||
public static final String OZONE_KSM_HTTPS_BIND_HOST_KEY =
|
|
||||||
"ozone.ksm.https-bind-host";
|
|
||||||
public static final String OZONE_KSM_HTTP_ADDRESS_KEY =
|
|
||||||
"ozone.ksm.http-address";
|
|
||||||
public static final String OZONE_KSM_HTTPS_ADDRESS_KEY =
|
|
||||||
"ozone.ksm.https-address";
|
|
||||||
public static final String OZONE_KSM_KEYTAB_FILE =
|
|
||||||
"ozone.ksm.keytab.file";
|
|
||||||
public static final String OZONE_KSM_HTTP_BIND_HOST_DEFAULT = "0.0.0.0";
|
|
||||||
public static final int OZONE_KSM_HTTP_BIND_PORT_DEFAULT = 9874;
|
|
||||||
public static final int OZONE_KSM_HTTPS_BIND_PORT_DEFAULT = 9875;
|
|
||||||
|
|
||||||
// LevelDB cache file uses an off-heap cache in LevelDB of 128 MB.
|
|
||||||
public static final String OZONE_KSM_DB_CACHE_SIZE_MB =
|
|
||||||
"ozone.ksm.db.cache.size.mb";
|
|
||||||
public static final int OZONE_KSM_DB_CACHE_SIZE_DEFAULT = 128;
|
|
||||||
|
|
||||||
public static final String OZONE_KSM_USER_MAX_VOLUME =
|
|
||||||
"ozone.ksm.user.max.volume";
|
|
||||||
public static final int OZONE_KSM_USER_MAX_VOLUME_DEFAULT = 1024;
|
|
||||||
|
|
||||||
// KSM Default user/group permissions
|
|
||||||
public static final String OZONE_KSM_USER_RIGHTS =
|
|
||||||
"ozone.ksm.user.rights";
|
|
||||||
public static final OzoneAcl.OzoneACLRights OZONE_KSM_USER_RIGHTS_DEFAULT =
|
|
||||||
OzoneAcl.OzoneACLRights.READ_WRITE;
|
|
||||||
|
|
||||||
public static final String OZONE_KSM_GROUP_RIGHTS =
|
|
||||||
"ozone.ksm.group.rights";
|
|
||||||
public static final OzoneAcl.OzoneACLRights OZONE_KSM_GROUP_RIGHTS_DEFAULT =
|
|
||||||
OzoneAcl.OzoneACLRights.READ_WRITE;
|
|
||||||
|
|
||||||
public static final String OZONE_KEY_DELETING_LIMIT_PER_TASK =
|
|
||||||
"ozone.key.deleting.limit.per.task";
|
|
||||||
public static final int OZONE_KEY_DELETING_LIMIT_PER_TASK_DEFAULT = 1000;
|
|
||||||
}
|
|
|
@ -0,0 +1,81 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with this
|
||||||
|
* work for additional information regarding copyright ownership. The ASF
|
||||||
|
* licenses this file to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance with the License.
|
||||||
|
* You may obtain a copy of the License at
|
||||||
|
* <p>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
|
||||||
|
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||||
|
* License for the specific language governing permissions and limitations under
|
||||||
|
* the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.ozone.om;
|
||||||
|
|
||||||
|
import org.apache.hadoop.ozone.OzoneAcl;
|
||||||
|
/**
|
||||||
|
* Ozone Manager Constants.
|
||||||
|
*/
|
||||||
|
public final class OMConfigKeys {
|
||||||
|
/**
|
||||||
|
* Never constructed.
|
||||||
|
*/
|
||||||
|
private OMConfigKeys() {
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
public static final String OZONE_OM_HANDLER_COUNT_KEY =
|
||||||
|
"ozone.om.handler.count.key";
|
||||||
|
public static final int OZONE_OM_HANDLER_COUNT_DEFAULT = 20;
|
||||||
|
|
||||||
|
public static final String OZONE_OM_ADDRESS_KEY =
|
||||||
|
"ozone.om.address";
|
||||||
|
public static final String OZONE_OM_BIND_HOST_DEFAULT =
|
||||||
|
"0.0.0.0";
|
||||||
|
public static final int OZONE_OM_PORT_DEFAULT = 9862;
|
||||||
|
|
||||||
|
public static final String OZONE_OM_HTTP_ENABLED_KEY =
|
||||||
|
"ozone.om.http.enabled";
|
||||||
|
public static final String OZONE_OM_HTTP_BIND_HOST_KEY =
|
||||||
|
"ozone.om.http-bind-host";
|
||||||
|
public static final String OZONE_OM_HTTPS_BIND_HOST_KEY =
|
||||||
|
"ozone.om.https-bind-host";
|
||||||
|
public static final String OZONE_OM_HTTP_ADDRESS_KEY =
|
||||||
|
"ozone.om.http-address";
|
||||||
|
public static final String OZONE_OM_HTTPS_ADDRESS_KEY =
|
||||||
|
"ozone.om.https-address";
|
||||||
|
public static final String OZONE_OM_KEYTAB_FILE =
|
||||||
|
"ozone.om.keytab.file";
|
||||||
|
public static final String OZONE_OM_HTTP_BIND_HOST_DEFAULT = "0.0.0.0";
|
||||||
|
public static final int OZONE_OM_HTTP_BIND_PORT_DEFAULT = 9874;
|
||||||
|
public static final int OZONE_OM_HTTPS_BIND_PORT_DEFAULT = 9875;
|
||||||
|
|
||||||
|
// LevelDB cache file uses an off-heap cache in LevelDB of 128 MB.
|
||||||
|
public static final String OZONE_OM_DB_CACHE_SIZE_MB =
|
||||||
|
"ozone.om.db.cache.size.mb";
|
||||||
|
public static final int OZONE_OM_DB_CACHE_SIZE_DEFAULT = 128;
|
||||||
|
|
||||||
|
public static final String OZONE_OM_USER_MAX_VOLUME =
|
||||||
|
"ozone.om.user.max.volume";
|
||||||
|
public static final int OZONE_OM_USER_MAX_VOLUME_DEFAULT = 1024;
|
||||||
|
|
||||||
|
// OM Default user/group permissions
|
||||||
|
public static final String OZONE_OM_USER_RIGHTS =
|
||||||
|
"ozone.om.user.rights";
|
||||||
|
public static final OzoneAcl.OzoneACLRights OZONE_OM_USER_RIGHTS_DEFAULT =
|
||||||
|
OzoneAcl.OzoneACLRights.READ_WRITE;
|
||||||
|
|
||||||
|
public static final String OZONE_OM_GROUP_RIGHTS =
|
||||||
|
"ozone.om.group.rights";
|
||||||
|
public static final OzoneAcl.OzoneACLRights OZONE_OM_GROUP_RIGHTS_DEFAULT =
|
||||||
|
OzoneAcl.OzoneACLRights.READ_WRITE;
|
||||||
|
|
||||||
|
public static final String OZONE_KEY_DELETING_LIMIT_PER_TASK =
|
||||||
|
"ozone.key.deleting.limit.per.task";
|
||||||
|
public static final int OZONE_KEY_DELETING_LIMIT_PER_TASK_DEFAULT = 1000;
|
||||||
|
}
|
|
@ -15,7 +15,7 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm.helpers;
|
package org.apache.hadoop.ozone.om.helpers;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
@ -25,13 +25,13 @@ import org.apache.hadoop.fs.StorageType;
|
||||||
import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
|
import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
|
||||||
import org.apache.hadoop.ozone.OzoneAcl;
|
import org.apache.hadoop.ozone.OzoneAcl;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.BucketArgs;
|
.OzoneManagerProtocolProtos.BucketArgs;
|
||||||
import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
|
import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A class that encapsulates Bucket Arguments.
|
* A class that encapsulates Bucket Arguments.
|
||||||
*/
|
*/
|
||||||
public final class KsmBucketArgs {
|
public final class OmBucketArgs {
|
||||||
/**
|
/**
|
||||||
* Name of the volume in which the bucket belongs to.
|
* Name of the volume in which the bucket belongs to.
|
||||||
*/
|
*/
|
||||||
|
@ -67,7 +67,7 @@ public final class KsmBucketArgs {
|
||||||
* @param isVersionEnabled - Bucket version flag.
|
* @param isVersionEnabled - Bucket version flag.
|
||||||
* @param storageType - Storage type to be used.
|
* @param storageType - Storage type to be used.
|
||||||
*/
|
*/
|
||||||
private KsmBucketArgs(String volumeName, String bucketName,
|
private OmBucketArgs(String volumeName, String bucketName,
|
||||||
List<OzoneAcl> addAcls, List<OzoneAcl> removeAcls,
|
List<OzoneAcl> addAcls, List<OzoneAcl> removeAcls,
|
||||||
Boolean isVersionEnabled, StorageType storageType) {
|
Boolean isVersionEnabled, StorageType storageType) {
|
||||||
this.volumeName = volumeName;
|
this.volumeName = volumeName;
|
||||||
|
@ -127,7 +127,7 @@ public final class KsmBucketArgs {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns new builder class that builds a KsmBucketArgs.
|
* Returns new builder class that builds a OmBucketArgs.
|
||||||
*
|
*
|
||||||
* @return Builder
|
* @return Builder
|
||||||
*/
|
*/
|
||||||
|
@ -136,7 +136,7 @@ public final class KsmBucketArgs {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Builder for KsmBucketArgs.
|
* Builder for OmBucketArgs.
|
||||||
*/
|
*/
|
||||||
public static class Builder {
|
public static class Builder {
|
||||||
private String volumeName;
|
private String volumeName;
|
||||||
|
@ -177,19 +177,19 @@ public final class KsmBucketArgs {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructs the KsmBucketArgs.
|
* Constructs the OmBucketArgs.
|
||||||
* @return instance of KsmBucketArgs.
|
* @return instance of OmBucketArgs.
|
||||||
*/
|
*/
|
||||||
public KsmBucketArgs build() {
|
public OmBucketArgs build() {
|
||||||
Preconditions.checkNotNull(volumeName);
|
Preconditions.checkNotNull(volumeName);
|
||||||
Preconditions.checkNotNull(bucketName);
|
Preconditions.checkNotNull(bucketName);
|
||||||
return new KsmBucketArgs(volumeName, bucketName, addAcls,
|
return new OmBucketArgs(volumeName, bucketName, addAcls,
|
||||||
removeAcls, isVersionEnabled, storageType);
|
removeAcls, isVersionEnabled, storageType);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates BucketArgs protobuf from KsmBucketArgs.
|
* Creates BucketArgs protobuf from OmBucketArgs.
|
||||||
*/
|
*/
|
||||||
public BucketArgs getProtobuf() {
|
public BucketArgs getProtobuf() {
|
||||||
BucketArgs.Builder builder = BucketArgs.newBuilder();
|
BucketArgs.Builder builder = BucketArgs.newBuilder();
|
||||||
|
@ -197,11 +197,11 @@ public final class KsmBucketArgs {
|
||||||
.setBucketName(bucketName);
|
.setBucketName(bucketName);
|
||||||
if(addAcls != null && !addAcls.isEmpty()) {
|
if(addAcls != null && !addAcls.isEmpty()) {
|
||||||
builder.addAllAddAcls(addAcls.stream().map(
|
builder.addAllAddAcls(addAcls.stream().map(
|
||||||
KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
|
OMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
|
||||||
}
|
}
|
||||||
if(removeAcls != null && !removeAcls.isEmpty()) {
|
if(removeAcls != null && !removeAcls.isEmpty()) {
|
||||||
builder.addAllRemoveAcls(removeAcls.stream().map(
|
builder.addAllRemoveAcls(removeAcls.stream().map(
|
||||||
KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
|
OMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
|
||||||
}
|
}
|
||||||
if(isVersionEnabled != null) {
|
if(isVersionEnabled != null) {
|
||||||
builder.setIsVersionEnabled(isVersionEnabled);
|
builder.setIsVersionEnabled(isVersionEnabled);
|
||||||
|
@ -214,17 +214,17 @@ public final class KsmBucketArgs {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Parses BucketInfo protobuf and creates KsmBucketArgs.
|
* Parses BucketInfo protobuf and creates OmBucketArgs.
|
||||||
* @param bucketArgs
|
* @param bucketArgs
|
||||||
* @return instance of KsmBucketArgs
|
* @return instance of OmBucketArgs
|
||||||
*/
|
*/
|
||||||
public static KsmBucketArgs getFromProtobuf(BucketArgs bucketArgs) {
|
public static OmBucketArgs getFromProtobuf(BucketArgs bucketArgs) {
|
||||||
return new KsmBucketArgs(bucketArgs.getVolumeName(),
|
return new OmBucketArgs(bucketArgs.getVolumeName(),
|
||||||
bucketArgs.getBucketName(),
|
bucketArgs.getBucketName(),
|
||||||
bucketArgs.getAddAclsList().stream().map(
|
bucketArgs.getAddAclsList().stream().map(
|
||||||
KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
|
OMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
|
||||||
bucketArgs.getRemoveAclsList().stream().map(
|
bucketArgs.getRemoveAclsList().stream().map(
|
||||||
KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
|
OMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
|
||||||
bucketArgs.hasIsVersionEnabled() ?
|
bucketArgs.hasIsVersionEnabled() ?
|
||||||
bucketArgs.getIsVersionEnabled() : null,
|
bucketArgs.getIsVersionEnabled() : null,
|
||||||
bucketArgs.hasStorageType() ? PBHelperClient.convertStorageType(
|
bucketArgs.hasStorageType() ? PBHelperClient.convertStorageType(
|
|
@ -15,15 +15,15 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm.helpers;
|
package org.apache.hadoop.ozone.om.helpers;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.hadoop.fs.StorageType;
|
import org.apache.hadoop.fs.StorageType;
|
||||||
import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
|
import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
|
||||||
import org.apache.hadoop.ozone.OzoneAcl;
|
import org.apache.hadoop.ozone.OzoneAcl;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.BucketInfo;
|
.OzoneManagerProtocolProtos.BucketInfo;
|
||||||
import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
|
import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
|
||||||
|
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -32,7 +32,7 @@ import java.util.stream.Collectors;
|
||||||
/**
|
/**
|
||||||
* A class that encapsulates Bucket Info.
|
* A class that encapsulates Bucket Info.
|
||||||
*/
|
*/
|
||||||
public final class KsmBucketInfo {
|
public final class OmBucketInfo {
|
||||||
/**
|
/**
|
||||||
* Name of the volume in which the bucket belongs to.
|
* Name of the volume in which the bucket belongs to.
|
||||||
*/
|
*/
|
||||||
|
@ -68,7 +68,7 @@ public final class KsmBucketInfo {
|
||||||
* @param storageType - Storage type to be used.
|
* @param storageType - Storage type to be used.
|
||||||
* @param creationTime - Bucket creation time.
|
* @param creationTime - Bucket creation time.
|
||||||
*/
|
*/
|
||||||
private KsmBucketInfo(String volumeName, String bucketName,
|
private OmBucketInfo(String volumeName, String bucketName,
|
||||||
List<OzoneAcl> acls, boolean isVersionEnabled,
|
List<OzoneAcl> acls, boolean isVersionEnabled,
|
||||||
StorageType storageType, long creationTime) {
|
StorageType storageType, long creationTime) {
|
||||||
this.volumeName = volumeName;
|
this.volumeName = volumeName;
|
||||||
|
@ -129,7 +129,7 @@ public final class KsmBucketInfo {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns new builder class that builds a KsmBucketInfo.
|
* Returns new builder class that builds a OmBucketInfo.
|
||||||
*
|
*
|
||||||
* @return Builder
|
* @return Builder
|
||||||
*/
|
*/
|
||||||
|
@ -138,7 +138,7 @@ public final class KsmBucketInfo {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Builder for KsmBucketInfo.
|
* Builder for OmBucketInfo.
|
||||||
*/
|
*/
|
||||||
public static class Builder {
|
public static class Builder {
|
||||||
private String volumeName;
|
private String volumeName;
|
||||||
|
@ -186,30 +186,30 @@ public final class KsmBucketInfo {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructs the KsmBucketInfo.
|
* Constructs the OmBucketInfo.
|
||||||
* @return instance of KsmBucketInfo.
|
* @return instance of OmBucketInfo.
|
||||||
*/
|
*/
|
||||||
public KsmBucketInfo build() {
|
public OmBucketInfo build() {
|
||||||
Preconditions.checkNotNull(volumeName);
|
Preconditions.checkNotNull(volumeName);
|
||||||
Preconditions.checkNotNull(bucketName);
|
Preconditions.checkNotNull(bucketName);
|
||||||
Preconditions.checkNotNull(acls);
|
Preconditions.checkNotNull(acls);
|
||||||
Preconditions.checkNotNull(isVersionEnabled);
|
Preconditions.checkNotNull(isVersionEnabled);
|
||||||
Preconditions.checkNotNull(storageType);
|
Preconditions.checkNotNull(storageType);
|
||||||
|
|
||||||
return new KsmBucketInfo(volumeName, bucketName, acls,
|
return new OmBucketInfo(volumeName, bucketName, acls,
|
||||||
isVersionEnabled, storageType, creationTime);
|
isVersionEnabled, storageType, creationTime);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates BucketInfo protobuf from KsmBucketInfo.
|
* Creates BucketInfo protobuf from OmBucketInfo.
|
||||||
*/
|
*/
|
||||||
public BucketInfo getProtobuf() {
|
public BucketInfo getProtobuf() {
|
||||||
return BucketInfo.newBuilder()
|
return BucketInfo.newBuilder()
|
||||||
.setVolumeName(volumeName)
|
.setVolumeName(volumeName)
|
||||||
.setBucketName(bucketName)
|
.setBucketName(bucketName)
|
||||||
.addAllAcls(acls.stream().map(
|
.addAllAcls(acls.stream().map(
|
||||||
KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()))
|
OMPBHelper::convertOzoneAcl).collect(Collectors.toList()))
|
||||||
.setIsVersionEnabled(isVersionEnabled)
|
.setIsVersionEnabled(isVersionEnabled)
|
||||||
.setStorageType(PBHelperClient.convertStorageType(
|
.setStorageType(PBHelperClient.convertStorageType(
|
||||||
storageType))
|
storageType))
|
||||||
|
@ -218,16 +218,16 @@ public final class KsmBucketInfo {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Parses BucketInfo protobuf and creates KsmBucketInfo.
|
* Parses BucketInfo protobuf and creates OmBucketInfo.
|
||||||
* @param bucketInfo
|
* @param bucketInfo
|
||||||
* @return instance of KsmBucketInfo
|
* @return instance of OmBucketInfo
|
||||||
*/
|
*/
|
||||||
public static KsmBucketInfo getFromProtobuf(BucketInfo bucketInfo) {
|
public static OmBucketInfo getFromProtobuf(BucketInfo bucketInfo) {
|
||||||
return new KsmBucketInfo(
|
return new OmBucketInfo(
|
||||||
bucketInfo.getVolumeName(),
|
bucketInfo.getVolumeName(),
|
||||||
bucketInfo.getBucketName(),
|
bucketInfo.getBucketName(),
|
||||||
bucketInfo.getAclsList().stream().map(
|
bucketInfo.getAclsList().stream().map(
|
||||||
KSMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
|
OMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
|
||||||
bucketInfo.getIsVersionEnabled(),
|
bucketInfo.getIsVersionEnabled(),
|
||||||
PBHelperClient.convertStorageType(
|
PBHelperClient.convertStorageType(
|
||||||
bucketInfo.getStorageType()), bucketInfo.getCreationTime());
|
bucketInfo.getStorageType()), bucketInfo.getCreationTime());
|
|
@ -15,7 +15,7 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm.helpers;
|
package org.apache.hadoop.ozone.om.helpers;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
||||||
|
|
||||||
|
@ -23,7 +23,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
||||||
* Args for key. Client use this to specify key's attributes on key creation
|
* Args for key. Client use this to specify key's attributes on key creation
|
||||||
* (putKey()).
|
* (putKey()).
|
||||||
*/
|
*/
|
||||||
public final class KsmKeyArgs {
|
public final class OmKeyArgs {
|
||||||
private final String volumeName;
|
private final String volumeName;
|
||||||
private final String bucketName;
|
private final String bucketName;
|
||||||
private final String keyName;
|
private final String keyName;
|
||||||
|
@ -31,7 +31,7 @@ public final class KsmKeyArgs {
|
||||||
private final ReplicationType type;
|
private final ReplicationType type;
|
||||||
private final ReplicationFactor factor;
|
private final ReplicationFactor factor;
|
||||||
|
|
||||||
private KsmKeyArgs(String volumeName, String bucketName, String keyName,
|
private OmKeyArgs(String volumeName, String bucketName, String keyName,
|
||||||
long dataSize, ReplicationType type, ReplicationFactor factor) {
|
long dataSize, ReplicationType type, ReplicationFactor factor) {
|
||||||
this.volumeName = volumeName;
|
this.volumeName = volumeName;
|
||||||
this.bucketName = bucketName;
|
this.bucketName = bucketName;
|
||||||
|
@ -70,7 +70,7 @@ public final class KsmKeyArgs {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Builder class of KsmKeyArgs.
|
* Builder class of OmKeyArgs.
|
||||||
*/
|
*/
|
||||||
public static class Builder {
|
public static class Builder {
|
||||||
private String volumeName;
|
private String volumeName;
|
||||||
|
@ -111,8 +111,8 @@ public final class KsmKeyArgs {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public KsmKeyArgs build() {
|
public OmKeyArgs build() {
|
||||||
return new KsmKeyArgs(volumeName, bucketName, keyName, dataSize,
|
return new OmKeyArgs(volumeName, bucketName, keyName, dataSize,
|
||||||
type, factor);
|
type, factor);
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -15,11 +15,11 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm.helpers;
|
package org.apache.hadoop.ozone.om.helpers;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.KeyInfo;
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
|
||||||
import org.apache.hadoop.util.Time;
|
import org.apache.hadoop.util.Time;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -28,23 +28,23 @@ import java.util.stream.Collectors;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Args for key block. The block instance for the key requested in putKey.
|
* Args for key block. The block instance for the key requested in putKey.
|
||||||
* This is returned from KSM to client, and client use class to talk to
|
* This is returned from OM to client, and client use class to talk to
|
||||||
* datanode. Also, this is the metadata written to ksm.db on server side.
|
* datanode. Also, this is the metadata written to om.db on server side.
|
||||||
*/
|
*/
|
||||||
public final class KsmKeyInfo {
|
public final class OmKeyInfo {
|
||||||
private final String volumeName;
|
private final String volumeName;
|
||||||
private final String bucketName;
|
private final String bucketName;
|
||||||
// name of key client specified
|
// name of key client specified
|
||||||
private String keyName;
|
private String keyName;
|
||||||
private long dataSize;
|
private long dataSize;
|
||||||
private List<KsmKeyLocationInfoGroup> keyLocationVersions;
|
private List<OmKeyLocationInfoGroup> keyLocationVersions;
|
||||||
private final long creationTime;
|
private final long creationTime;
|
||||||
private long modificationTime;
|
private long modificationTime;
|
||||||
private HddsProtos.ReplicationType type;
|
private HddsProtos.ReplicationType type;
|
||||||
private HddsProtos.ReplicationFactor factor;
|
private HddsProtos.ReplicationFactor factor;
|
||||||
|
|
||||||
private KsmKeyInfo(String volumeName, String bucketName, String keyName,
|
private OmKeyInfo(String volumeName, String bucketName, String keyName,
|
||||||
List<KsmKeyLocationInfoGroup> versions, long dataSize,
|
List<OmKeyLocationInfoGroup> versions, long dataSize,
|
||||||
long creationTime, long modificationTime, HddsProtos.ReplicationType type,
|
long creationTime, long modificationTime, HddsProtos.ReplicationType type,
|
||||||
HddsProtos.ReplicationFactor factor) {
|
HddsProtos.ReplicationFactor factor) {
|
||||||
this.volumeName = volumeName;
|
this.volumeName = volumeName;
|
||||||
|
@ -52,12 +52,12 @@ public final class KsmKeyInfo {
|
||||||
this.keyName = keyName;
|
this.keyName = keyName;
|
||||||
this.dataSize = dataSize;
|
this.dataSize = dataSize;
|
||||||
// it is important that the versions are ordered from old to new.
|
// it is important that the versions are ordered from old to new.
|
||||||
// Do this sanity check when versions got loaded on creating KsmKeyInfo.
|
// Do this sanity check when versions got loaded on creating OmKeyInfo.
|
||||||
// TODO : this is not necessary, here only because versioning is still a
|
// TODO : this is not necessary, here only because versioning is still a
|
||||||
// work in-progress, remove this following check when versioning is
|
// work in-progress, remove this following check when versioning is
|
||||||
// complete and prove correctly functioning
|
// complete and prove correctly functioning
|
||||||
long currentVersion = -1;
|
long currentVersion = -1;
|
||||||
for (KsmKeyLocationInfoGroup version : versions) {
|
for (OmKeyLocationInfoGroup version : versions) {
|
||||||
Preconditions.checkArgument(
|
Preconditions.checkArgument(
|
||||||
currentVersion + 1 == version.getVersion());
|
currentVersion + 1 == version.getVersion());
|
||||||
currentVersion = version.getVersion();
|
currentVersion = version.getVersion();
|
||||||
|
@ -101,13 +101,13 @@ public final class KsmKeyInfo {
|
||||||
this.dataSize = size;
|
this.dataSize = size;
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized KsmKeyLocationInfoGroup getLatestVersionLocations()
|
public synchronized OmKeyLocationInfoGroup getLatestVersionLocations()
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return keyLocationVersions.size() == 0? null :
|
return keyLocationVersions.size() == 0? null :
|
||||||
keyLocationVersions.get(keyLocationVersions.size() - 1);
|
keyLocationVersions.get(keyLocationVersions.size() - 1);
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<KsmKeyLocationInfoGroup> getKeyLocationVersions() {
|
public List<OmKeyLocationInfoGroup> getKeyLocationVersions() {
|
||||||
return keyLocationVersions;
|
return keyLocationVersions;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -123,11 +123,11 @@ public final class KsmKeyInfo {
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public synchronized void appendNewBlocks(
|
public synchronized void appendNewBlocks(
|
||||||
List<KsmKeyLocationInfo> newLocationList) throws IOException {
|
List<OmKeyLocationInfo> newLocationList) throws IOException {
|
||||||
if (keyLocationVersions.size() == 0) {
|
if (keyLocationVersions.size() == 0) {
|
||||||
throw new IOException("Appending new block, but no version exist");
|
throw new IOException("Appending new block, but no version exist");
|
||||||
}
|
}
|
||||||
KsmKeyLocationInfoGroup currentLatestVersion =
|
OmKeyLocationInfoGroup currentLatestVersion =
|
||||||
keyLocationVersions.get(keyLocationVersions.size() - 1);
|
keyLocationVersions.get(keyLocationVersions.size() - 1);
|
||||||
currentLatestVersion.appendNewBlocks(newLocationList);
|
currentLatestVersion.appendNewBlocks(newLocationList);
|
||||||
setModificationTime(Time.now());
|
setModificationTime(Time.now());
|
||||||
|
@ -141,18 +141,18 @@ public final class KsmKeyInfo {
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public synchronized long addNewVersion(
|
public synchronized long addNewVersion(
|
||||||
List<KsmKeyLocationInfo> newLocationList) throws IOException {
|
List<OmKeyLocationInfo> newLocationList) throws IOException {
|
||||||
long latestVersionNum;
|
long latestVersionNum;
|
||||||
if (keyLocationVersions.size() == 0) {
|
if (keyLocationVersions.size() == 0) {
|
||||||
// no version exist, these blocks are the very first version.
|
// no version exist, these blocks are the very first version.
|
||||||
keyLocationVersions.add(new KsmKeyLocationInfoGroup(0, newLocationList));
|
keyLocationVersions.add(new OmKeyLocationInfoGroup(0, newLocationList));
|
||||||
latestVersionNum = 0;
|
latestVersionNum = 0;
|
||||||
} else {
|
} else {
|
||||||
// it is important that the new version are always at the tail of the list
|
// it is important that the new version are always at the tail of the list
|
||||||
KsmKeyLocationInfoGroup currentLatestVersion =
|
OmKeyLocationInfoGroup currentLatestVersion =
|
||||||
keyLocationVersions.get(keyLocationVersions.size() - 1);
|
keyLocationVersions.get(keyLocationVersions.size() - 1);
|
||||||
// the new version is created based on the current latest version
|
// the new version is created based on the current latest version
|
||||||
KsmKeyLocationInfoGroup newVersion =
|
OmKeyLocationInfoGroup newVersion =
|
||||||
currentLatestVersion.generateNextVersion(newLocationList);
|
currentLatestVersion.generateNextVersion(newLocationList);
|
||||||
keyLocationVersions.add(newVersion);
|
keyLocationVersions.add(newVersion);
|
||||||
latestVersionNum = newVersion.getVersion();
|
latestVersionNum = newVersion.getVersion();
|
||||||
|
@ -174,14 +174,14 @@ public final class KsmKeyInfo {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Builder of KsmKeyInfo.
|
* Builder of OmKeyInfo.
|
||||||
*/
|
*/
|
||||||
public static class Builder {
|
public static class Builder {
|
||||||
private String volumeName;
|
private String volumeName;
|
||||||
private String bucketName;
|
private String bucketName;
|
||||||
private String keyName;
|
private String keyName;
|
||||||
private long dataSize;
|
private long dataSize;
|
||||||
private List<KsmKeyLocationInfoGroup> ksmKeyLocationInfoGroups;
|
private List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups;
|
||||||
private long creationTime;
|
private long creationTime;
|
||||||
private long modificationTime;
|
private long modificationTime;
|
||||||
private HddsProtos.ReplicationType type;
|
private HddsProtos.ReplicationType type;
|
||||||
|
@ -202,9 +202,9 @@ public final class KsmKeyInfo {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Builder setKsmKeyLocationInfos(
|
public Builder setOmKeyLocationInfos(
|
||||||
List<KsmKeyLocationInfoGroup> ksmKeyLocationInfoList) {
|
List<OmKeyLocationInfoGroup> omKeyLocationInfoList) {
|
||||||
this.ksmKeyLocationInfoGroups = ksmKeyLocationInfoList;
|
this.omKeyLocationInfoGroups = omKeyLocationInfoList;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -233,9 +233,9 @@ public final class KsmKeyInfo {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public KsmKeyInfo build() {
|
public OmKeyInfo build() {
|
||||||
return new KsmKeyInfo(
|
return new OmKeyInfo(
|
||||||
volumeName, bucketName, keyName, ksmKeyLocationInfoGroups,
|
volumeName, bucketName, keyName, omKeyLocationInfoGroups,
|
||||||
dataSize, creationTime, modificationTime, type, factor);
|
dataSize, creationTime, modificationTime, type, factor);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -251,7 +251,7 @@ public final class KsmKeyInfo {
|
||||||
.setFactor(factor)
|
.setFactor(factor)
|
||||||
.setType(type)
|
.setType(type)
|
||||||
.addAllKeyLocationList(keyLocationVersions.stream()
|
.addAllKeyLocationList(keyLocationVersions.stream()
|
||||||
.map(KsmKeyLocationInfoGroup::getProtobuf)
|
.map(OmKeyLocationInfoGroup::getProtobuf)
|
||||||
.collect(Collectors.toList()))
|
.collect(Collectors.toList()))
|
||||||
.setLatestVersion(latestVersion)
|
.setLatestVersion(latestVersion)
|
||||||
.setCreationTime(creationTime)
|
.setCreationTime(creationTime)
|
||||||
|
@ -259,13 +259,13 @@ public final class KsmKeyInfo {
|
||||||
.build();
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
public static KsmKeyInfo getFromProtobuf(KeyInfo keyInfo) {
|
public static OmKeyInfo getFromProtobuf(KeyInfo keyInfo) {
|
||||||
return new KsmKeyInfo(
|
return new OmKeyInfo(
|
||||||
keyInfo.getVolumeName(),
|
keyInfo.getVolumeName(),
|
||||||
keyInfo.getBucketName(),
|
keyInfo.getBucketName(),
|
||||||
keyInfo.getKeyName(),
|
keyInfo.getKeyName(),
|
||||||
keyInfo.getKeyLocationListList().stream()
|
keyInfo.getKeyLocationListList().stream()
|
||||||
.map(KsmKeyLocationInfoGroup::getFromProtobuf)
|
.map(OmKeyLocationInfoGroup::getFromProtobuf)
|
||||||
.collect(Collectors.toList()),
|
.collect(Collectors.toList()),
|
||||||
keyInfo.getDataSize(),
|
keyInfo.getDataSize(),
|
||||||
keyInfo.getCreationTime(),
|
keyInfo.getCreationTime(),
|
|
@ -14,16 +14,16 @@
|
||||||
* License for the specific language governing permissions and limitations under
|
* License for the specific language governing permissions and limitations under
|
||||||
* the License.
|
* the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm.helpers;
|
package org.apache.hadoop.ozone.om.helpers;
|
||||||
|
|
||||||
import org.apache.hadoop.hdds.client.BlockID;
|
import org.apache.hadoop.hdds.client.BlockID;
|
||||||
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.KeyLocation;
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyLocation;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* One key can be too huge to fit in one container. In which case it gets split
|
* One key can be too huge to fit in one container. In which case it gets split
|
||||||
* into a number of subkeys. This class represents one such subkey instance.
|
* into a number of subkeys. This class represents one such subkey instance.
|
||||||
*/
|
*/
|
||||||
public final class KsmKeyLocationInfo {
|
public final class OmKeyLocationInfo {
|
||||||
private final BlockID blockID;
|
private final BlockID blockID;
|
||||||
private final boolean shouldCreateContainer;
|
private final boolean shouldCreateContainer;
|
||||||
// the id of this subkey in all the subkeys.
|
// the id of this subkey in all the subkeys.
|
||||||
|
@ -32,7 +32,7 @@ public final class KsmKeyLocationInfo {
|
||||||
// the version number indicating when this block was added
|
// the version number indicating when this block was added
|
||||||
private long createVersion;
|
private long createVersion;
|
||||||
|
|
||||||
private KsmKeyLocationInfo(BlockID blockID, boolean shouldCreateContainer,
|
private OmKeyLocationInfo(BlockID blockID, boolean shouldCreateContainer,
|
||||||
long length, long offset) {
|
long length, long offset) {
|
||||||
this.blockID = blockID;
|
this.blockID = blockID;
|
||||||
this.shouldCreateContainer = shouldCreateContainer;
|
this.shouldCreateContainer = shouldCreateContainer;
|
||||||
|
@ -73,7 +73,7 @@ public final class KsmKeyLocationInfo {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Builder of KsmKeyLocationInfo.
|
* Builder of OmKeyLocationInfo.
|
||||||
*/
|
*/
|
||||||
public static class Builder {
|
public static class Builder {
|
||||||
private BlockID blockID;
|
private BlockID blockID;
|
||||||
|
@ -101,8 +101,8 @@ public final class KsmKeyLocationInfo {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public KsmKeyLocationInfo build() {
|
public OmKeyLocationInfo build() {
|
||||||
return new KsmKeyLocationInfo(blockID,
|
return new OmKeyLocationInfo(blockID,
|
||||||
shouldCreateContainer, length, offset);
|
shouldCreateContainer, length, offset);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -117,8 +117,8 @@ public final class KsmKeyLocationInfo {
|
||||||
.build();
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
public static KsmKeyLocationInfo getFromProtobuf(KeyLocation keyLocation) {
|
public static OmKeyLocationInfo getFromProtobuf(KeyLocation keyLocation) {
|
||||||
KsmKeyLocationInfo info = new KsmKeyLocationInfo(
|
OmKeyLocationInfo info = new OmKeyLocationInfo(
|
||||||
BlockID.getFromProtobuf(keyLocation.getBlockID()),
|
BlockID.getFromProtobuf(keyLocation.getBlockID()),
|
||||||
keyLocation.getShouldCreateContainer(),
|
keyLocation.getShouldCreateContainer(),
|
||||||
keyLocation.getLength(),
|
keyLocation.getLength(),
|
|
@ -14,9 +14,9 @@
|
||||||
* License for the specific language governing permissions and limitations under
|
* License for the specific language governing permissions and limitations under
|
||||||
* the License.
|
* the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm.helpers;
|
package org.apache.hadoop.ozone.om.helpers;
|
||||||
|
|
||||||
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.KeyLocationList;
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyLocationList;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
@ -27,12 +27,12 @@ import java.util.stream.Collectors;
|
||||||
* A list of key locations. This class represents one single version of the
|
* A list of key locations. This class represents one single version of the
|
||||||
* blocks of a key.
|
* blocks of a key.
|
||||||
*/
|
*/
|
||||||
public class KsmKeyLocationInfoGroup {
|
public class OmKeyLocationInfoGroup {
|
||||||
private final long version;
|
private final long version;
|
||||||
private final List<KsmKeyLocationInfo> locationList;
|
private final List<OmKeyLocationInfo> locationList;
|
||||||
|
|
||||||
public KsmKeyLocationInfoGroup(long version,
|
public OmKeyLocationInfoGroup(long version,
|
||||||
List<KsmKeyLocationInfo> locations) {
|
List<OmKeyLocationInfo> locations) {
|
||||||
this.version = version;
|
this.version = version;
|
||||||
this.locationList = locations;
|
this.locationList = locations;
|
||||||
}
|
}
|
||||||
|
@ -42,8 +42,8 @@ public class KsmKeyLocationInfoGroup {
|
||||||
*
|
*
|
||||||
* @return the list of blocks that are created in the latest version.
|
* @return the list of blocks that are created in the latest version.
|
||||||
*/
|
*/
|
||||||
public List<KsmKeyLocationInfo> getBlocksLatestVersionOnly() {
|
public List<OmKeyLocationInfo> getBlocksLatestVersionOnly() {
|
||||||
List<KsmKeyLocationInfo> list = new ArrayList<>();
|
List<OmKeyLocationInfo> list = new ArrayList<>();
|
||||||
locationList.stream().filter(x -> x.getCreateVersion() == version)
|
locationList.stream().filter(x -> x.getCreateVersion() == version)
|
||||||
.forEach(list::add);
|
.forEach(list::add);
|
||||||
return list;
|
return list;
|
||||||
|
@ -53,7 +53,7 @@ public class KsmKeyLocationInfoGroup {
|
||||||
return version;
|
return version;
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<KsmKeyLocationInfo> getLocationList() {
|
public List<OmKeyLocationInfo> getLocationList() {
|
||||||
return locationList;
|
return locationList;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -61,17 +61,17 @@ public class KsmKeyLocationInfoGroup {
|
||||||
return KeyLocationList.newBuilder()
|
return KeyLocationList.newBuilder()
|
||||||
.setVersion(version)
|
.setVersion(version)
|
||||||
.addAllKeyLocations(
|
.addAllKeyLocations(
|
||||||
locationList.stream().map(KsmKeyLocationInfo::getProtobuf)
|
locationList.stream().map(OmKeyLocationInfo::getProtobuf)
|
||||||
.collect(Collectors.toList()))
|
.collect(Collectors.toList()))
|
||||||
.build();
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
public static KsmKeyLocationInfoGroup getFromProtobuf(
|
public static OmKeyLocationInfoGroup getFromProtobuf(
|
||||||
KeyLocationList keyLocationList) {
|
KeyLocationList keyLocationList) {
|
||||||
return new KsmKeyLocationInfoGroup(
|
return new OmKeyLocationInfoGroup(
|
||||||
keyLocationList.getVersion(),
|
keyLocationList.getVersion(),
|
||||||
keyLocationList.getKeyLocationsList().stream()
|
keyLocationList.getKeyLocationsList().stream()
|
||||||
.map(KsmKeyLocationInfo::getFromProtobuf)
|
.map(OmKeyLocationInfo::getFromProtobuf)
|
||||||
.collect(Collectors.toList()));
|
.collect(Collectors.toList()));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -82,25 +82,25 @@ public class KsmKeyLocationInfoGroup {
|
||||||
* @param newLocationList a list of new location to be added.
|
* @param newLocationList a list of new location to be added.
|
||||||
* @return
|
* @return
|
||||||
*/
|
*/
|
||||||
KsmKeyLocationInfoGroup generateNextVersion(
|
OmKeyLocationInfoGroup generateNextVersion(
|
||||||
List<KsmKeyLocationInfo> newLocationList) throws IOException {
|
List<OmKeyLocationInfo> newLocationList) throws IOException {
|
||||||
// TODO : revisit if we can do this method more efficiently
|
// TODO : revisit if we can do this method more efficiently
|
||||||
// one potential inefficiency here is that later version always include
|
// one potential inefficiency here is that later version always include
|
||||||
// older ones. e.g. v1 has B1, then v2, v3...will all have B1 and only add
|
// older ones. e.g. v1 has B1, then v2, v3...will all have B1 and only add
|
||||||
// more
|
// more
|
||||||
List<KsmKeyLocationInfo> newList = new ArrayList<>();
|
List<OmKeyLocationInfo> newList = new ArrayList<>();
|
||||||
newList.addAll(locationList);
|
newList.addAll(locationList);
|
||||||
for (KsmKeyLocationInfo newInfo : newLocationList) {
|
for (OmKeyLocationInfo newInfo : newLocationList) {
|
||||||
// all these new blocks will have addVersion of current version + 1
|
// all these new blocks will have addVersion of current version + 1
|
||||||
newInfo.setCreateVersion(version + 1);
|
newInfo.setCreateVersion(version + 1);
|
||||||
newList.add(newInfo);
|
newList.add(newInfo);
|
||||||
}
|
}
|
||||||
return new KsmKeyLocationInfoGroup(version + 1, newList);
|
return new OmKeyLocationInfoGroup(version + 1, newList);
|
||||||
}
|
}
|
||||||
|
|
||||||
void appendNewBlocks(List<KsmKeyLocationInfo> newLocationList)
|
void appendNewBlocks(List<OmKeyLocationInfo> newLocationList)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
for (KsmKeyLocationInfo info : newLocationList) {
|
for (OmKeyLocationInfo info : newLocationList) {
|
||||||
info.setCreateVersion(version);
|
info.setCreateVersion(version);
|
||||||
locationList.add(info);
|
locationList.add(info);
|
||||||
}
|
}
|
||||||
|
@ -110,7 +110,7 @@ public class KsmKeyLocationInfoGroup {
|
||||||
public String toString() {
|
public String toString() {
|
||||||
StringBuilder sb = new StringBuilder();
|
StringBuilder sb = new StringBuilder();
|
||||||
sb.append("version:").append(version).append(" ");
|
sb.append("version:").append(version).append(" ");
|
||||||
for (KsmKeyLocationInfo kli : locationList) {
|
for (OmKeyLocationInfo kli : locationList) {
|
||||||
sb.append(kli.getLocalID()).append(" || ");
|
sb.append(kli.getLocalID()).append(" || ");
|
||||||
}
|
}
|
||||||
return sb.toString();
|
return sb.toString();
|
|
@ -16,14 +16,14 @@
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package org.apache.hadoop.ozone.ksm.helpers;
|
package org.apache.hadoop.ozone.om.helpers;
|
||||||
|
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.OzoneAclInfo;
|
.OzoneManagerProtocolProtos.OzoneAclInfo;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.OzoneAclInfo.OzoneAclRights;
|
.OzoneManagerProtocolProtos.OzoneAclInfo.OzoneAclRights;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.OzoneAclInfo.OzoneAclType;
|
.OzoneManagerProtocolProtos.OzoneAclInfo.OzoneAclType;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
|
@ -34,11 +34,11 @@ import java.util.HashMap;
|
||||||
/**
|
/**
|
||||||
* This helper class keeps a map of all user and their permissions.
|
* This helper class keeps a map of all user and their permissions.
|
||||||
*/
|
*/
|
||||||
public class KsmOzoneAclMap {
|
public class OmOzoneAclMap {
|
||||||
// per Acl Type user:rights map
|
// per Acl Type user:rights map
|
||||||
private ArrayList<Map<String, OzoneAclRights>> aclMaps;
|
private ArrayList<Map<String, OzoneAclRights>> aclMaps;
|
||||||
|
|
||||||
KsmOzoneAclMap() {
|
OmOzoneAclMap() {
|
||||||
aclMaps = new ArrayList<>();
|
aclMaps = new ArrayList<>();
|
||||||
for (OzoneAclType aclType : OzoneAclType.values()) {
|
for (OzoneAclType aclType : OzoneAclType.values()) {
|
||||||
aclMaps.add(aclType.ordinal(), new HashMap<>());
|
aclMaps.add(aclType.ordinal(), new HashMap<>());
|
||||||
|
@ -99,9 +99,9 @@ public class KsmOzoneAclMap {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Create map from list of OzoneAclInfos
|
// Create map from list of OzoneAclInfos
|
||||||
public static KsmOzoneAclMap ozoneAclGetFromProtobuf(
|
public static OmOzoneAclMap ozoneAclGetFromProtobuf(
|
||||||
List<OzoneAclInfo> aclList) {
|
List<OzoneAclInfo> aclList) {
|
||||||
KsmOzoneAclMap aclMap = new KsmOzoneAclMap();
|
OmOzoneAclMap aclMap = new OmOzoneAclMap();
|
||||||
for (OzoneAclInfo acl : aclList) {
|
for (OzoneAclInfo acl : aclList) {
|
||||||
aclMap.addAcl(acl);
|
aclMap.addAcl(acl);
|
||||||
}
|
}
|
|
@ -15,13 +15,13 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm.helpers;
|
package org.apache.hadoop.ozone.om.helpers;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.OzoneAclInfo;
|
.OzoneManagerProtocolProtos.OzoneAclInfo;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.VolumeInfo;
|
.OzoneManagerProtocolProtos.VolumeInfo;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.KeyValue;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.KeyValue;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -33,16 +33,16 @@ import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A class that encapsulates the KsmVolumeArgs Args.
|
* A class that encapsulates the OmVolumeArgs Args.
|
||||||
*/
|
*/
|
||||||
public final class KsmVolumeArgs {
|
public final class OmVolumeArgs {
|
||||||
private final String adminName;
|
private final String adminName;
|
||||||
private final String ownerName;
|
private final String ownerName;
|
||||||
private final String volume;
|
private final String volume;
|
||||||
private final long creationTime;
|
private final long creationTime;
|
||||||
private final long quotaInBytes;
|
private final long quotaInBytes;
|
||||||
private final Map<String, String> keyValueMap;
|
private final Map<String, String> keyValueMap;
|
||||||
private final KsmOzoneAclMap aclMap;
|
private final OmOzoneAclMap aclMap;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Private constructor, constructed via builder.
|
* Private constructor, constructed via builder.
|
||||||
|
@ -54,9 +54,9 @@ public final class KsmVolumeArgs {
|
||||||
* @param aclMap - User to access rights map.
|
* @param aclMap - User to access rights map.
|
||||||
* @param creationTime - Volume creation time.
|
* @param creationTime - Volume creation time.
|
||||||
*/
|
*/
|
||||||
private KsmVolumeArgs(String adminName, String ownerName, String volume,
|
private OmVolumeArgs(String adminName, String ownerName, String volume,
|
||||||
long quotaInBytes, Map<String, String> keyValueMap,
|
long quotaInBytes, Map<String, String> keyValueMap,
|
||||||
KsmOzoneAclMap aclMap, long creationTime) {
|
OmOzoneAclMap aclMap, long creationTime) {
|
||||||
this.adminName = adminName;
|
this.adminName = adminName;
|
||||||
this.ownerName = ownerName;
|
this.ownerName = ownerName;
|
||||||
this.volume = volume;
|
this.volume = volume;
|
||||||
|
@ -110,11 +110,11 @@ public final class KsmVolumeArgs {
|
||||||
return keyValueMap;
|
return keyValueMap;
|
||||||
}
|
}
|
||||||
|
|
||||||
public KsmOzoneAclMap getAclMap() {
|
public OmOzoneAclMap getAclMap() {
|
||||||
return aclMap;
|
return aclMap;
|
||||||
}
|
}
|
||||||
/**
|
/**
|
||||||
* Returns new builder class that builds a KsmVolumeArgs.
|
* Returns new builder class that builds a OmVolumeArgs.
|
||||||
*
|
*
|
||||||
* @return Builder
|
* @return Builder
|
||||||
*/
|
*/
|
||||||
|
@ -123,7 +123,7 @@ public final class KsmVolumeArgs {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Builder for KsmVolumeArgs.
|
* Builder for OmVolumeArgs.
|
||||||
*/
|
*/
|
||||||
public static class Builder {
|
public static class Builder {
|
||||||
private String adminName;
|
private String adminName;
|
||||||
|
@ -132,14 +132,14 @@ public final class KsmVolumeArgs {
|
||||||
private long creationTime;
|
private long creationTime;
|
||||||
private long quotaInBytes;
|
private long quotaInBytes;
|
||||||
private Map<String, String> keyValueMap;
|
private Map<String, String> keyValueMap;
|
||||||
private KsmOzoneAclMap aclMap;
|
private OmOzoneAclMap aclMap;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructs a builder.
|
* Constructs a builder.
|
||||||
*/
|
*/
|
||||||
Builder() {
|
Builder() {
|
||||||
keyValueMap = new HashMap<>();
|
keyValueMap = new HashMap<>();
|
||||||
aclMap = new KsmOzoneAclMap();
|
aclMap = new OmOzoneAclMap();
|
||||||
}
|
}
|
||||||
|
|
||||||
public Builder setAdminName(String admin) {
|
public Builder setAdminName(String admin) {
|
||||||
|
@ -181,11 +181,11 @@ public final class KsmVolumeArgs {
|
||||||
* Constructs a CreateVolumeArgument.
|
* Constructs a CreateVolumeArgument.
|
||||||
* @return CreateVolumeArgs.
|
* @return CreateVolumeArgs.
|
||||||
*/
|
*/
|
||||||
public KsmVolumeArgs build() {
|
public OmVolumeArgs build() {
|
||||||
Preconditions.checkNotNull(adminName);
|
Preconditions.checkNotNull(adminName);
|
||||||
Preconditions.checkNotNull(ownerName);
|
Preconditions.checkNotNull(ownerName);
|
||||||
Preconditions.checkNotNull(volume);
|
Preconditions.checkNotNull(volume);
|
||||||
return new KsmVolumeArgs(adminName, ownerName, volume, quotaInBytes,
|
return new OmVolumeArgs(adminName, ownerName, volume, quotaInBytes,
|
||||||
keyValueMap, aclMap, creationTime);
|
keyValueMap, aclMap, creationTime);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -209,14 +209,14 @@ public final class KsmVolumeArgs {
|
||||||
.build();
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
public static KsmVolumeArgs getFromProtobuf(VolumeInfo volInfo) {
|
public static OmVolumeArgs getFromProtobuf(VolumeInfo volInfo) {
|
||||||
Map<String, String> kvMap = volInfo.getMetadataList().stream()
|
Map<String, String> kvMap = volInfo.getMetadataList().stream()
|
||||||
.collect(Collectors.toMap(KeyValue::getKey,
|
.collect(Collectors.toMap(KeyValue::getKey,
|
||||||
KeyValue::getValue));
|
KeyValue::getValue));
|
||||||
KsmOzoneAclMap aclMap =
|
OmOzoneAclMap aclMap =
|
||||||
KsmOzoneAclMap.ozoneAclGetFromProtobuf(volInfo.getVolumeAclsList());
|
OmOzoneAclMap.ozoneAclGetFromProtobuf(volInfo.getVolumeAclsList());
|
||||||
|
|
||||||
return new KsmVolumeArgs(volInfo.getAdminName(), volInfo.getOwnerName(),
|
return new OmVolumeArgs(volInfo.getAdminName(), volInfo.getOwnerName(),
|
||||||
volInfo.getVolume(), volInfo.getQuotaInBytes(), kvMap, aclMap,
|
volInfo.getVolume(), volInfo.getQuotaInBytes(), kvMap, aclMap,
|
||||||
volInfo.getCreationTime());
|
volInfo.getCreationTime());
|
||||||
}
|
}
|
|
@ -15,7 +15,7 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm.helpers;
|
package org.apache.hadoop.ozone.om.helpers;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class represents a open key "session". A session here means a key is
|
* This class represents a open key "session". A session here means a key is
|
||||||
|
@ -24,13 +24,13 @@ package org.apache.hadoop.ozone.ksm.helpers;
|
||||||
*/
|
*/
|
||||||
public class OpenKeySession {
|
public class OpenKeySession {
|
||||||
private final int id;
|
private final int id;
|
||||||
private final KsmKeyInfo keyInfo;
|
private final OmKeyInfo keyInfo;
|
||||||
// the version of the key when it is being opened in this session.
|
// the version of the key when it is being opened in this session.
|
||||||
// a block that has a create version equals to open version means it will
|
// a block that has a create version equals to open version means it will
|
||||||
// be committed only when this open session is closed.
|
// be committed only when this open session is closed.
|
||||||
private long openVersion;
|
private long openVersion;
|
||||||
|
|
||||||
public OpenKeySession(int id, KsmKeyInfo info, long version) {
|
public OpenKeySession(int id, OmKeyInfo info, long version) {
|
||||||
this.id = id;
|
this.id = id;
|
||||||
this.keyInfo = info;
|
this.keyInfo = info;
|
||||||
this.openVersion = version;
|
this.openVersion = version;
|
||||||
|
@ -40,7 +40,7 @@ public class OpenKeySession {
|
||||||
return this.openVersion;
|
return this.openVersion;
|
||||||
}
|
}
|
||||||
|
|
||||||
public KsmKeyInfo getKeyInfo() {
|
public OmKeyInfo getKeyInfo() {
|
||||||
return keyInfo;
|
return keyInfo;
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,7 +16,7 @@
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package org.apache.hadoop.ozone.ksm.helpers;
|
package org.apache.hadoop.ozone.om.helpers;
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonIgnore;
|
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||||
|
@ -25,8 +25,8 @@ import com.fasterxml.jackson.databind.ObjectReader;
|
||||||
import com.fasterxml.jackson.databind.ObjectWriter;
|
import com.fasterxml.jackson.databind.ObjectWriter;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.hadoop.ozone.client.rest.response.BucketInfo;
|
import org.apache.hadoop.ozone.client.rest.response.BucketInfo;
|
||||||
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos;
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
||||||
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||||
.ServicePort;
|
.ServicePort;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType;
|
||||||
|
|
||||||
|
@ -121,14 +121,14 @@ public final class ServiceInfo {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Converts {@link ServiceInfo} to KeySpaceManagerProtocolProtos.ServiceInfo.
|
* Converts {@link ServiceInfo} to OzoneManagerProtocolProtos.ServiceInfo.
|
||||||
*
|
*
|
||||||
* @return KeySpaceManagerProtocolProtos.ServiceInfo
|
* @return OzoneManagerProtocolProtos.ServiceInfo
|
||||||
*/
|
*/
|
||||||
@JsonIgnore
|
@JsonIgnore
|
||||||
public KeySpaceManagerProtocolProtos.ServiceInfo getProtobuf() {
|
public OzoneManagerProtocolProtos.ServiceInfo getProtobuf() {
|
||||||
KeySpaceManagerProtocolProtos.ServiceInfo.Builder builder =
|
OzoneManagerProtocolProtos.ServiceInfo.Builder builder =
|
||||||
KeySpaceManagerProtocolProtos.ServiceInfo.newBuilder();
|
OzoneManagerProtocolProtos.ServiceInfo.newBuilder();
|
||||||
builder.setNodeType(nodeType)
|
builder.setNodeType(nodeType)
|
||||||
.setHostname(hostname)
|
.setHostname(hostname)
|
||||||
.addAllServicePorts(
|
.addAllServicePorts(
|
||||||
|
@ -143,13 +143,13 @@ public final class ServiceInfo {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Converts KeySpaceManagerProtocolProtos.ServiceInfo to {@link ServiceInfo}.
|
* Converts OzoneManagerProtocolProtos.ServiceInfo to {@link ServiceInfo}.
|
||||||
*
|
*
|
||||||
* @return {@link ServiceInfo}
|
* @return {@link ServiceInfo}
|
||||||
*/
|
*/
|
||||||
@JsonIgnore
|
@JsonIgnore
|
||||||
public static ServiceInfo getFromProtobuf(
|
public static ServiceInfo getFromProtobuf(
|
||||||
KeySpaceManagerProtocolProtos.ServiceInfo serviceInfo) {
|
OzoneManagerProtocolProtos.ServiceInfo serviceInfo) {
|
||||||
return new ServiceInfo(serviceInfo.getNodeType(),
|
return new ServiceInfo(serviceInfo.getNodeType(),
|
||||||
serviceInfo.getHostname(),
|
serviceInfo.getHostname(),
|
||||||
serviceInfo.getServicePortsList());
|
serviceInfo.getServicePortsList());
|
|
@ -15,7 +15,7 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm.helpers;
|
package org.apache.hadoop.ozone.om.helpers;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
|
|
@ -15,4 +15,4 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm.helpers;
|
package org.apache.hadoop.ozone.om.helpers;
|
|
@ -15,7 +15,7 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm;
|
package org.apache.hadoop.ozone.om;
|
||||||
/**
|
/**
|
||||||
This package contains client side protocol library to communicate with KSM.
|
This package contains client side protocol library to communicate with OM.
|
||||||
*/
|
*/
|
|
@ -15,32 +15,32 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm.protocol;
|
package org.apache.hadoop.ozone.om.protocol;
|
||||||
|
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession;
|
import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
|
import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.OzoneAclInfo;
|
.OzoneManagerProtocolProtos.OzoneAclInfo;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Protocol to talk to KSM.
|
* Protocol to talk to OM.
|
||||||
*/
|
*/
|
||||||
public interface KeySpaceManagerProtocol {
|
public interface OzoneManagerProtocol {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a volume.
|
* Creates a volume.
|
||||||
* @param args - Arguments to create Volume.
|
* @param args - Arguments to create Volume.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
void createVolume(KsmVolumeArgs args) throws IOException;
|
void createVolume(OmVolumeArgs args) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Changes the owner of a volume.
|
* Changes the owner of a volume.
|
||||||
|
@ -75,7 +75,7 @@ public interface KeySpaceManagerProtocol {
|
||||||
* @return VolumeArgs or exception is thrown.
|
* @return VolumeArgs or exception is thrown.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
KsmVolumeArgs getVolumeInfo(String volume) throws IOException;
|
OmVolumeArgs getVolumeInfo(String volume) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Deletes an existing empty volume.
|
* Deletes an existing empty volume.
|
||||||
|
@ -93,7 +93,7 @@ public interface KeySpaceManagerProtocol {
|
||||||
* @return List of Volumes.
|
* @return List of Volumes.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
List<KsmVolumeArgs> listVolumeByUser(String userName, String prefix, String
|
List<OmVolumeArgs> listVolumeByUser(String userName, String prefix, String
|
||||||
prevKey, int maxKeys) throws IOException;
|
prevKey, int maxKeys) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -104,7 +104,7 @@ public interface KeySpaceManagerProtocol {
|
||||||
* @return List of Volumes.
|
* @return List of Volumes.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
List<KsmVolumeArgs> listAllVolumes(String prefix, String
|
List<OmVolumeArgs> listAllVolumes(String prefix, String
|
||||||
prevKey, int maxKeys) throws IOException;
|
prevKey, int maxKeys) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -112,16 +112,16 @@ public interface KeySpaceManagerProtocol {
|
||||||
* @param bucketInfo - BucketInfo to create Bucket.
|
* @param bucketInfo - BucketInfo to create Bucket.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
void createBucket(KsmBucketInfo bucketInfo) throws IOException;
|
void createBucket(OmBucketInfo bucketInfo) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets the bucket information.
|
* Gets the bucket information.
|
||||||
* @param volumeName - Volume name.
|
* @param volumeName - Volume name.
|
||||||
* @param bucketName - Bucket name.
|
* @param bucketName - Bucket name.
|
||||||
* @return KsmBucketInfo or exception is thrown.
|
* @return OmBucketInfo or exception is thrown.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
KsmBucketInfo getBucketInfo(String volumeName, String bucketName)
|
OmBucketInfo getBucketInfo(String volumeName, String bucketName)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -129,7 +129,7 @@ public interface KeySpaceManagerProtocol {
|
||||||
* @param args - BucketArgs.
|
* @param args - BucketArgs.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
void setBucketProperty(KsmBucketArgs args) throws IOException;
|
void setBucketProperty(OmBucketArgs args) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Open the given key and return an open key session.
|
* Open the given key and return an open key session.
|
||||||
|
@ -138,7 +138,7 @@ public interface KeySpaceManagerProtocol {
|
||||||
* @return OpenKeySession instance that client uses to talk to container.
|
* @return OpenKeySession instance that client uses to talk to container.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
OpenKeySession openKey(KsmKeyArgs args) throws IOException;
|
OpenKeySession openKey(OmKeyArgs args) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Commit a key. This will make the change from the client visible. The client
|
* Commit a key. This will make the change from the client visible. The client
|
||||||
|
@ -148,7 +148,7 @@ public interface KeySpaceManagerProtocol {
|
||||||
* @param clientID the client identification
|
* @param clientID the client identification
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
void commitKey(KsmKeyArgs args, int clientID) throws IOException;
|
void commitKey(OmKeyArgs args, int clientID) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Allocate a new block, it is assumed that the client is having an open key
|
* Allocate a new block, it is assumed that the client is having an open key
|
||||||
|
@ -159,24 +159,24 @@ public interface KeySpaceManagerProtocol {
|
||||||
* @return an allocated block
|
* @return an allocated block
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
KsmKeyLocationInfo allocateBlock(KsmKeyArgs args, int clientID)
|
OmKeyLocationInfo allocateBlock(OmKeyArgs args, int clientID)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Look up for the container of an existing key.
|
* Look up for the container of an existing key.
|
||||||
*
|
*
|
||||||
* @param args the args of the key.
|
* @param args the args of the key.
|
||||||
* @return KsmKeyInfo instance that client uses to talk to container.
|
* @return OmKeyInfo instance that client uses to talk to container.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
KsmKeyInfo lookupKey(KsmKeyArgs args) throws IOException;
|
OmKeyInfo lookupKey(OmKeyArgs args) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Rename an existing key within a bucket
|
* Rename an existing key within a bucket
|
||||||
* @param args the args of the key.
|
* @param args the args of the key.
|
||||||
* @param toKeyName New name to be used for the Key
|
* @param toKeyName New name to be used for the Key
|
||||||
*/
|
*/
|
||||||
void renameKey(KsmKeyArgs args, String toKeyName) throws IOException;
|
void renameKey(OmKeyArgs args, String toKeyName) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Deletes an existing key.
|
* Deletes an existing key.
|
||||||
|
@ -184,7 +184,7 @@ public interface KeySpaceManagerProtocol {
|
||||||
* @param args the args of the key.
|
* @param args the args of the key.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
void deleteKey(KsmKeyArgs args) throws IOException;
|
void deleteKey(OmKeyArgs args) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Deletes an existing empty bucket from volume.
|
* Deletes an existing empty bucket from volume.
|
||||||
|
@ -195,7 +195,7 @@ public interface KeySpaceManagerProtocol {
|
||||||
void deleteBucket(String volume, String bucket) throws IOException;
|
void deleteBucket(String volume, String bucket) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a list of buckets represented by {@link KsmBucketInfo}
|
* Returns a list of buckets represented by {@link OmBucketInfo}
|
||||||
* in the given volume. Argument volumeName is required, others
|
* in the given volume. Argument volumeName is required, others
|
||||||
* are optional.
|
* are optional.
|
||||||
*
|
*
|
||||||
|
@ -213,12 +213,12 @@ public interface KeySpaceManagerProtocol {
|
||||||
* @return a list of buckets.
|
* @return a list of buckets.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
List<KsmBucketInfo> listBuckets(String volumeName,
|
List<OmBucketInfo> listBuckets(String volumeName,
|
||||||
String startBucketName, String bucketPrefix, int maxNumOfBuckets)
|
String startBucketName, String bucketPrefix, int maxNumOfBuckets)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a list of keys represented by {@link KsmKeyInfo}
|
* Returns a list of keys represented by {@link OmKeyInfo}
|
||||||
* in the given bucket. Argument volumeName, bucketName is required,
|
* in the given bucket. Argument volumeName, bucketName is required,
|
||||||
* others are optional.
|
* others are optional.
|
||||||
*
|
*
|
||||||
|
@ -238,7 +238,7 @@ public interface KeySpaceManagerProtocol {
|
||||||
* @return a list of keys.
|
* @return a list of keys.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
List<KsmKeyInfo> listKeys(String volumeName,
|
List<OmKeyInfo> listKeys(String volumeName,
|
||||||
String bucketName, String startKeyName, String keyPrefix, int maxKeys)
|
String bucketName, String startKeyName, String keyPrefix, int maxKeys)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
|
@ -16,4 +16,4 @@
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package org.apache.hadoop.ozone.ksm.protocol;
|
package org.apache.hadoop.ozone.om.protocol;
|
|
@ -15,7 +15,7 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm.protocolPB;
|
package org.apache.hadoop.ozone.om.protocolPB;
|
||||||
|
|
||||||
import com.google.common.base.Strings;
|
import com.google.common.base.Strings;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
@ -24,95 +24,95 @@ import com.google.protobuf.ServiceException;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.ipc.ProtobufHelper;
|
import org.apache.hadoop.ipc.ProtobufHelper;
|
||||||
import org.apache.hadoop.ipc.ProtocolTranslator;
|
import org.apache.hadoop.ipc.ProtocolTranslator;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession;
|
import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
|
import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.protocol.KeySpaceManagerProtocol;
|
import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.AllocateBlockRequest;
|
.OzoneManagerProtocolProtos.AllocateBlockRequest;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.AllocateBlockResponse;
|
.OzoneManagerProtocolProtos.AllocateBlockResponse;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.CommitKeyRequest;
|
.OzoneManagerProtocolProtos.CommitKeyRequest;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.CommitKeyResponse;
|
.OzoneManagerProtocolProtos.CommitKeyResponse;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.BucketArgs;
|
.OzoneManagerProtocolProtos.BucketArgs;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.BucketInfo;
|
.OzoneManagerProtocolProtos.BucketInfo;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.CreateBucketRequest;
|
.OzoneManagerProtocolProtos.CreateBucketRequest;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.CreateBucketResponse;
|
.OzoneManagerProtocolProtos.CreateBucketResponse;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.InfoBucketRequest;
|
.OzoneManagerProtocolProtos.InfoBucketRequest;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.InfoBucketResponse;
|
.OzoneManagerProtocolProtos.InfoBucketResponse;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.SetBucketPropertyRequest;
|
.OzoneManagerProtocolProtos.SetBucketPropertyRequest;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.SetBucketPropertyResponse;
|
.OzoneManagerProtocolProtos.SetBucketPropertyResponse;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.DeleteBucketRequest;
|
.OzoneManagerProtocolProtos.DeleteBucketRequest;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.DeleteBucketResponse;
|
.OzoneManagerProtocolProtos.DeleteBucketResponse;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.CreateVolumeRequest;
|
.OzoneManagerProtocolProtos.CreateVolumeRequest;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.CreateVolumeResponse;
|
.OzoneManagerProtocolProtos.CreateVolumeResponse;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.LocateKeyRequest;
|
.OzoneManagerProtocolProtos.LocateKeyRequest;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.LocateKeyResponse;
|
.OzoneManagerProtocolProtos.LocateKeyResponse;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.RenameKeyRequest;
|
.OzoneManagerProtocolProtos.RenameKeyRequest;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.RenameKeyResponse;
|
.OzoneManagerProtocolProtos.RenameKeyResponse;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.KeyArgs;
|
.OzoneManagerProtocolProtos.KeyArgs;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.SetVolumePropertyRequest;
|
.OzoneManagerProtocolProtos.SetVolumePropertyRequest;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.SetVolumePropertyResponse;
|
.OzoneManagerProtocolProtos.SetVolumePropertyResponse;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.DeleteVolumeRequest;
|
.OzoneManagerProtocolProtos.DeleteVolumeRequest;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.DeleteVolumeResponse;
|
.OzoneManagerProtocolProtos.DeleteVolumeResponse;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.InfoVolumeRequest;
|
.OzoneManagerProtocolProtos.InfoVolumeRequest;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.InfoVolumeResponse;
|
.OzoneManagerProtocolProtos.InfoVolumeResponse;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.CheckVolumeAccessRequest;
|
.OzoneManagerProtocolProtos.CheckVolumeAccessRequest;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.CheckVolumeAccessResponse;
|
.OzoneManagerProtocolProtos.CheckVolumeAccessResponse;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.ListBucketsRequest;
|
.OzoneManagerProtocolProtos.ListBucketsRequest;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.ListBucketsResponse;
|
.OzoneManagerProtocolProtos.ListBucketsResponse;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.ListKeysRequest;
|
.OzoneManagerProtocolProtos.ListKeysRequest;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.ListKeysResponse;
|
.OzoneManagerProtocolProtos.ListKeysResponse;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.VolumeInfo;
|
.OzoneManagerProtocolProtos.VolumeInfo;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.Status;
|
.OzoneManagerProtocolProtos.Status;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.OzoneAclInfo;
|
.OzoneManagerProtocolProtos.OzoneAclInfo;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.ListVolumeRequest;
|
.OzoneManagerProtocolProtos.ListVolumeRequest;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.ListVolumeResponse;
|
.OzoneManagerProtocolProtos.ListVolumeResponse;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.ServiceListRequest;
|
.OzoneManagerProtocolProtos.ServiceListRequest;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.ServiceListResponse;
|
.OzoneManagerProtocolProtos.ServiceListResponse;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -121,26 +121,26 @@ import java.util.ArrayList;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The client side implementation of KeySpaceManagerProtocol.
|
* The client side implementation of OzoneManagerProtocol.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
public final class OzoneManagerProtocolClientSideTranslatorPB
|
||||||
implements KeySpaceManagerProtocol, ProtocolTranslator, Closeable {
|
implements OzoneManagerProtocol, ProtocolTranslator, Closeable {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* RpcController is not used and hence is set to null.
|
* RpcController is not used and hence is set to null.
|
||||||
*/
|
*/
|
||||||
private static final RpcController NULL_RPC_CONTROLLER = null;
|
private static final RpcController NULL_RPC_CONTROLLER = null;
|
||||||
|
|
||||||
private final KeySpaceManagerProtocolPB rpcProxy;
|
private final OzoneManagerProtocolPB rpcProxy;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructor for KeySpaceManger Client.
|
* Constructor for KeySpaceManger Client.
|
||||||
* @param rpcProxy
|
* @param rpcProxy
|
||||||
*/
|
*/
|
||||||
public KeySpaceManagerProtocolClientSideTranslatorPB(
|
public OzoneManagerProtocolClientSideTranslatorPB(
|
||||||
KeySpaceManagerProtocolPB rpcProxy) {
|
OzoneManagerProtocolPB rpcProxy) {
|
||||||
this.rpcProxy = rpcProxy;
|
this.rpcProxy = rpcProxy;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -169,7 +169,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void createVolume(KsmVolumeArgs args) throws IOException {
|
public void createVolume(OmVolumeArgs args) throws IOException {
|
||||||
CreateVolumeRequest.Builder req =
|
CreateVolumeRequest.Builder req =
|
||||||
CreateVolumeRequest.newBuilder();
|
CreateVolumeRequest.newBuilder();
|
||||||
VolumeInfo volumeInfo = args.getProtobuf();
|
VolumeInfo volumeInfo = args.getProtobuf();
|
||||||
|
@ -273,11 +273,11 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
* Gets the volume information.
|
* Gets the volume information.
|
||||||
*
|
*
|
||||||
* @param volume - Volume name.
|
* @param volume - Volume name.
|
||||||
* @return KsmVolumeArgs or exception is thrown.
|
* @return OmVolumeArgs or exception is thrown.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public KsmVolumeArgs getVolumeInfo(String volume) throws IOException {
|
public OmVolumeArgs getVolumeInfo(String volume) throws IOException {
|
||||||
InfoVolumeRequest.Builder req = InfoVolumeRequest.newBuilder();
|
InfoVolumeRequest.Builder req = InfoVolumeRequest.newBuilder();
|
||||||
req.setVolumeName(volume);
|
req.setVolumeName(volume);
|
||||||
final InfoVolumeResponse resp;
|
final InfoVolumeResponse resp;
|
||||||
|
@ -290,7 +290,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
throw new
|
throw new
|
||||||
IOException("Info Volume failed, error:" + resp.getStatus());
|
IOException("Info Volume failed, error:" + resp.getStatus());
|
||||||
}
|
}
|
||||||
return KsmVolumeArgs.getFromProtobuf(resp.getVolumeInfo());
|
return OmVolumeArgs.getFromProtobuf(resp.getVolumeInfo());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -327,7 +327,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public List<KsmVolumeArgs> listVolumeByUser(String userName, String prefix,
|
public List<OmVolumeArgs> listVolumeByUser(String userName, String prefix,
|
||||||
String prevKey, int maxKeys)
|
String prevKey, int maxKeys)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
ListVolumeRequest.Builder builder = ListVolumeRequest.newBuilder();
|
ListVolumeRequest.Builder builder = ListVolumeRequest.newBuilder();
|
||||||
|
@ -354,7 +354,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public List<KsmVolumeArgs> listAllVolumes(String prefix, String prevKey,
|
public List<OmVolumeArgs> listAllVolumes(String prefix, String prevKey,
|
||||||
int maxKeys) throws IOException {
|
int maxKeys) throws IOException {
|
||||||
ListVolumeRequest.Builder builder = ListVolumeRequest.newBuilder();
|
ListVolumeRequest.Builder builder = ListVolumeRequest.newBuilder();
|
||||||
if (!Strings.isNullOrEmpty(prefix)) {
|
if (!Strings.isNullOrEmpty(prefix)) {
|
||||||
|
@ -368,7 +368,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
return listVolume(builder.build());
|
return listVolume(builder.build());
|
||||||
}
|
}
|
||||||
|
|
||||||
private List<KsmVolumeArgs> listVolume(ListVolumeRequest request)
|
private List<OmVolumeArgs> listVolume(ListVolumeRequest request)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
final ListVolumeResponse resp;
|
final ListVolumeResponse resp;
|
||||||
try {
|
try {
|
||||||
|
@ -382,14 +382,14 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
+ resp.getStatus());
|
+ resp.getStatus());
|
||||||
}
|
}
|
||||||
|
|
||||||
List<KsmVolumeArgs> result = Lists.newArrayList();
|
List<OmVolumeArgs> result = Lists.newArrayList();
|
||||||
for (VolumeInfo volInfo : resp.getVolumeInfoList()) {
|
for (VolumeInfo volInfo : resp.getVolumeInfoList()) {
|
||||||
KsmVolumeArgs volArgs = KsmVolumeArgs.getFromProtobuf(volInfo);
|
OmVolumeArgs volArgs = OmVolumeArgs.getFromProtobuf(volInfo);
|
||||||
result.add(volArgs);
|
result.add(volArgs);
|
||||||
}
|
}
|
||||||
|
|
||||||
return resp.getVolumeInfoList().stream()
|
return resp.getVolumeInfoList().stream()
|
||||||
.map(item -> KsmVolumeArgs.getFromProtobuf(item))
|
.map(item -> OmVolumeArgs.getFromProtobuf(item))
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -400,7 +400,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void createBucket(KsmBucketInfo bucketInfo) throws IOException {
|
public void createBucket(OmBucketInfo bucketInfo) throws IOException {
|
||||||
CreateBucketRequest.Builder req =
|
CreateBucketRequest.Builder req =
|
||||||
CreateBucketRequest.newBuilder();
|
CreateBucketRequest.newBuilder();
|
||||||
BucketInfo bucketInfoProtobuf = bucketInfo.getProtobuf();
|
BucketInfo bucketInfoProtobuf = bucketInfo.getProtobuf();
|
||||||
|
@ -424,11 +424,11 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
*
|
*
|
||||||
* @param volume - Volume name.
|
* @param volume - Volume name.
|
||||||
* @param bucket - Bucket name.
|
* @param bucket - Bucket name.
|
||||||
* @return KsmBucketInfo or exception is thrown.
|
* @return OmBucketInfo or exception is thrown.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public KsmBucketInfo getBucketInfo(String volume, String bucket)
|
public OmBucketInfo getBucketInfo(String volume, String bucket)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
InfoBucketRequest.Builder req =
|
InfoBucketRequest.Builder req =
|
||||||
InfoBucketRequest.newBuilder();
|
InfoBucketRequest.newBuilder();
|
||||||
|
@ -443,7 +443,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
throw ProtobufHelper.getRemoteException(e);
|
throw ProtobufHelper.getRemoteException(e);
|
||||||
}
|
}
|
||||||
if (resp.getStatus() == Status.OK) {
|
if (resp.getStatus() == Status.OK) {
|
||||||
return KsmBucketInfo.getFromProtobuf(resp.getBucketInfo());
|
return OmBucketInfo.getFromProtobuf(resp.getBucketInfo());
|
||||||
} else {
|
} else {
|
||||||
throw new IOException("Info Bucket failed, error: "
|
throw new IOException("Info Bucket failed, error: "
|
||||||
+ resp.getStatus());
|
+ resp.getStatus());
|
||||||
|
@ -456,7 +456,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void setBucketProperty(KsmBucketArgs args)
|
public void setBucketProperty(OmBucketArgs args)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
SetBucketPropertyRequest.Builder req =
|
SetBucketPropertyRequest.Builder req =
|
||||||
SetBucketPropertyRequest.newBuilder();
|
SetBucketPropertyRequest.newBuilder();
|
||||||
|
@ -486,9 +486,9 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public List<KsmBucketInfo> listBuckets(String volumeName,
|
public List<OmBucketInfo> listBuckets(String volumeName,
|
||||||
String startKey, String prefix, int count) throws IOException {
|
String startKey, String prefix, int count) throws IOException {
|
||||||
List<KsmBucketInfo> buckets = new ArrayList<>();
|
List<OmBucketInfo> buckets = new ArrayList<>();
|
||||||
ListBucketsRequest.Builder reqBuilder = ListBucketsRequest.newBuilder();
|
ListBucketsRequest.Builder reqBuilder = ListBucketsRequest.newBuilder();
|
||||||
reqBuilder.setVolumeName(volumeName);
|
reqBuilder.setVolumeName(volumeName);
|
||||||
reqBuilder.setCount(count);
|
reqBuilder.setCount(count);
|
||||||
|
@ -509,7 +509,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
if (resp.getStatus() == Status.OK) {
|
if (resp.getStatus() == Status.OK) {
|
||||||
buckets.addAll(
|
buckets.addAll(
|
||||||
resp.getBucketInfoList().stream()
|
resp.getBucketInfoList().stream()
|
||||||
.map(KsmBucketInfo::getFromProtobuf)
|
.map(OmBucketInfo::getFromProtobuf)
|
||||||
.collect(Collectors.toList()));
|
.collect(Collectors.toList()));
|
||||||
return buckets;
|
return buckets;
|
||||||
} else {
|
} else {
|
||||||
|
@ -526,7 +526,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public OpenKeySession openKey(KsmKeyArgs args) throws IOException {
|
public OpenKeySession openKey(OmKeyArgs args) throws IOException {
|
||||||
LocateKeyRequest.Builder req = LocateKeyRequest.newBuilder();
|
LocateKeyRequest.Builder req = LocateKeyRequest.newBuilder();
|
||||||
KeyArgs.Builder keyArgs = KeyArgs.newBuilder()
|
KeyArgs.Builder keyArgs = KeyArgs.newBuilder()
|
||||||
.setVolumeName(args.getVolumeName())
|
.setVolumeName(args.getVolumeName())
|
||||||
|
@ -549,11 +549,11 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
throw new IOException("Create key failed, error:" + resp.getStatus());
|
throw new IOException("Create key failed, error:" + resp.getStatus());
|
||||||
}
|
}
|
||||||
return new OpenKeySession(resp.getID(),
|
return new OpenKeySession(resp.getID(),
|
||||||
KsmKeyInfo.getFromProtobuf(resp.getKeyInfo()), resp.getOpenVersion());
|
OmKeyInfo.getFromProtobuf(resp.getKeyInfo()), resp.getOpenVersion());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public KsmKeyLocationInfo allocateBlock(KsmKeyArgs args, int clientID)
|
public OmKeyLocationInfo allocateBlock(OmKeyArgs args, int clientID)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
AllocateBlockRequest.Builder req = AllocateBlockRequest.newBuilder();
|
AllocateBlockRequest.Builder req = AllocateBlockRequest.newBuilder();
|
||||||
KeyArgs keyArgs = KeyArgs.newBuilder()
|
KeyArgs keyArgs = KeyArgs.newBuilder()
|
||||||
|
@ -574,11 +574,11 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
throw new IOException("Allocate block failed, error:" +
|
throw new IOException("Allocate block failed, error:" +
|
||||||
resp.getStatus());
|
resp.getStatus());
|
||||||
}
|
}
|
||||||
return KsmKeyLocationInfo.getFromProtobuf(resp.getKeyLocation());
|
return OmKeyLocationInfo.getFromProtobuf(resp.getKeyLocation());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void commitKey(KsmKeyArgs args, int clientID)
|
public void commitKey(OmKeyArgs args, int clientID)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
CommitKeyRequest.Builder req = CommitKeyRequest.newBuilder();
|
CommitKeyRequest.Builder req = CommitKeyRequest.newBuilder();
|
||||||
KeyArgs keyArgs = KeyArgs.newBuilder()
|
KeyArgs keyArgs = KeyArgs.newBuilder()
|
||||||
|
@ -603,7 +603,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public KsmKeyInfo lookupKey(KsmKeyArgs args) throws IOException {
|
public OmKeyInfo lookupKey(OmKeyArgs args) throws IOException {
|
||||||
LocateKeyRequest.Builder req = LocateKeyRequest.newBuilder();
|
LocateKeyRequest.Builder req = LocateKeyRequest.newBuilder();
|
||||||
KeyArgs keyArgs = KeyArgs.newBuilder()
|
KeyArgs keyArgs = KeyArgs.newBuilder()
|
||||||
.setVolumeName(args.getVolumeName())
|
.setVolumeName(args.getVolumeName())
|
||||||
|
@ -622,11 +622,11 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
throw new IOException("Lookup key failed, error:" +
|
throw new IOException("Lookup key failed, error:" +
|
||||||
resp.getStatus());
|
resp.getStatus());
|
||||||
}
|
}
|
||||||
return KsmKeyInfo.getFromProtobuf(resp.getKeyInfo());
|
return OmKeyInfo.getFromProtobuf(resp.getKeyInfo());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void renameKey(KsmKeyArgs args, String toKeyName) throws IOException {
|
public void renameKey(OmKeyArgs args, String toKeyName) throws IOException {
|
||||||
RenameKeyRequest.Builder req = RenameKeyRequest.newBuilder();
|
RenameKeyRequest.Builder req = RenameKeyRequest.newBuilder();
|
||||||
KeyArgs keyArgs = KeyArgs.newBuilder()
|
KeyArgs keyArgs = KeyArgs.newBuilder()
|
||||||
.setVolumeName(args.getVolumeName())
|
.setVolumeName(args.getVolumeName())
|
||||||
|
@ -655,7 +655,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void deleteKey(KsmKeyArgs args) throws IOException {
|
public void deleteKey(OmKeyArgs args) throws IOException {
|
||||||
LocateKeyRequest.Builder req = LocateKeyRequest.newBuilder();
|
LocateKeyRequest.Builder req = LocateKeyRequest.newBuilder();
|
||||||
KeyArgs keyArgs = KeyArgs.newBuilder()
|
KeyArgs keyArgs = KeyArgs.newBuilder()
|
||||||
.setVolumeName(args.getVolumeName())
|
.setVolumeName(args.getVolumeName())
|
||||||
|
@ -701,9 +701,9 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
* List keys in a bucket.
|
* List keys in a bucket.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public List<KsmKeyInfo> listKeys(String volumeName, String bucketName,
|
public List<OmKeyInfo> listKeys(String volumeName, String bucketName,
|
||||||
String startKey, String prefix, int maxKeys) throws IOException {
|
String startKey, String prefix, int maxKeys) throws IOException {
|
||||||
List<KsmKeyInfo> keys = new ArrayList<>();
|
List<OmKeyInfo> keys = new ArrayList<>();
|
||||||
ListKeysRequest.Builder reqBuilder = ListKeysRequest.newBuilder();
|
ListKeysRequest.Builder reqBuilder = ListKeysRequest.newBuilder();
|
||||||
reqBuilder.setVolumeName(volumeName);
|
reqBuilder.setVolumeName(volumeName);
|
||||||
reqBuilder.setBucketName(bucketName);
|
reqBuilder.setBucketName(bucketName);
|
||||||
|
@ -728,7 +728,7 @@ public final class KeySpaceManagerProtocolClientSideTranslatorPB
|
||||||
if (resp.getStatus() == Status.OK) {
|
if (resp.getStatus() == Status.OK) {
|
||||||
keys.addAll(
|
keys.addAll(
|
||||||
resp.getKeyInfoList().stream()
|
resp.getKeyInfoList().stream()
|
||||||
.map(KsmKeyInfo::getFromProtobuf)
|
.map(OmKeyInfo::getFromProtobuf)
|
||||||
.collect(Collectors.toList()));
|
.collect(Collectors.toList()));
|
||||||
return keys;
|
return keys;
|
||||||
} else {
|
} else {
|
|
@ -15,20 +15,20 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm.protocolPB;
|
package org.apache.hadoop.ozone.om.protocolPB;
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.ipc.ProtocolInfo;
|
import org.apache.hadoop.ipc.ProtocolInfo;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.KeySpaceManagerService;
|
.OzoneManagerProtocolProtos.OzoneManagerService;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Protocol used to communicate with KSM.
|
* Protocol used to communicate with OM.
|
||||||
*/
|
*/
|
||||||
@ProtocolInfo(protocolName =
|
@ProtocolInfo(protocolName =
|
||||||
"org.apache.hadoop.ozone.protocol.KeySpaceManagerProtocol",
|
"org.apache.hadoop.ozone.protocol.OzoneManagerProtocol",
|
||||||
protocolVersion = 1)
|
protocolVersion = 1)
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public interface KeySpaceManagerProtocolPB
|
public interface OzoneManagerProtocolPB
|
||||||
extends KeySpaceManagerService.BlockingInterface {
|
extends OzoneManagerService.BlockingInterface {
|
||||||
}
|
}
|
|
@ -16,4 +16,4 @@
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package org.apache.hadoop.ozone.ksm.protocolPB;
|
package org.apache.hadoop.ozone.om.protocolPB;
|
|
@ -19,18 +19,18 @@ package org.apache.hadoop.ozone.protocolPB;
|
||||||
|
|
||||||
import org.apache.hadoop.ozone.OzoneAcl;
|
import org.apache.hadoop.ozone.OzoneAcl;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.OzoneAclInfo;
|
.OzoneManagerProtocolProtos.OzoneAclInfo;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.OzoneAclInfo.OzoneAclType;
|
.OzoneManagerProtocolProtos.OzoneAclInfo.OzoneAclType;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.OzoneAclInfo.OzoneAclRights;
|
.OzoneManagerProtocolProtos.OzoneAclInfo.OzoneAclRights;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Utilities for converting protobuf classes.
|
* Utilities for converting protobuf classes.
|
||||||
*/
|
*/
|
||||||
public final class KSMPBHelper {
|
public final class OMPBHelper {
|
||||||
|
|
||||||
private KSMPBHelper() {
|
private OMPBHelper() {
|
||||||
/** Hidden constructor */
|
/** Hidden constructor */
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,14 +23,14 @@
|
||||||
*/
|
*/
|
||||||
|
|
||||||
option java_package = "org.apache.hadoop.ozone.protocol.proto";
|
option java_package = "org.apache.hadoop.ozone.protocol.proto";
|
||||||
option java_outer_classname = "KeySpaceManagerProtocolProtos";
|
option java_outer_classname = "OzoneManagerProtocolProtos";
|
||||||
option java_generic_services = true;
|
option java_generic_services = true;
|
||||||
option java_generate_equals_and_hash = true;
|
option java_generate_equals_and_hash = true;
|
||||||
package hadoop.ozone;
|
package hadoop.ozone;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
This is file contains the protocol to communicate with
|
This is file contains the protocol to communicate with
|
||||||
Ozone key space manager. Ozone KSM manages the namespace for ozone.
|
Ozone Manager. Ozone Manager manages the namespace for ozone.
|
||||||
This is similar to Namenode for Ozone.
|
This is similar to Namenode for Ozone.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
|
@ -53,6 +53,12 @@ enum Status {
|
||||||
INVALID_KEY_NAME = 13;
|
INVALID_KEY_NAME = 13;
|
||||||
ACCESS_DENIED = 14;
|
ACCESS_DENIED = 14;
|
||||||
INTERNAL_ERROR = 15;
|
INTERNAL_ERROR = 15;
|
||||||
|
KEY_ALLOCATION_ERROR = 16;
|
||||||
|
KEY_DELETION_ERROR = 17;
|
||||||
|
KEY_RENAME_ERROR = 18;
|
||||||
|
METADATA_ERROR = 19;
|
||||||
|
OM_NOT_INITIALIZED = 20;
|
||||||
|
SCM_VERSION_MISMATCH_ERROR = 21;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -355,9 +361,9 @@ message ServiceInfo {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
The KSM service that takes care of Ozone namespace.
|
The OM service that takes care of Ozone namespace.
|
||||||
*/
|
*/
|
||||||
service KeySpaceManagerService {
|
service OzoneManagerService {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
Creates a Volume.
|
Creates a Volume.
|
|
@ -194,12 +194,12 @@ This path will be created by datanodes if it doesn't exist already. Here is an
|
||||||
</property>
|
</property>
|
||||||
```
|
```
|
||||||
|
|
||||||
1. **ozone.ksm.address** OM server address. This is used by OzoneClient and
|
1. **ozone.om.address** OM server address. This is used by OzoneClient and
|
||||||
Ozone File System.
|
Ozone File System.
|
||||||
```
|
```
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.ksm.address</name>
|
<name>ozone.om.address</name>
|
||||||
<value>ksm.hadoop.apache.org</value>
|
<value>om.hadoop.apache.org</value>
|
||||||
</property>
|
</property>
|
||||||
```
|
```
|
||||||
|
|
||||||
|
@ -213,7 +213,7 @@ Ozone File System.
|
||||||
| ozone.scm.block.client.address | SCM server name and port | Used by services like OM |
|
| ozone.scm.block.client.address | SCM server name and port | Used by services like OM |
|
||||||
| ozone.scm.client.address | SCM server name and port | Used by client side |
|
| ozone.scm.client.address | SCM server name and port | Used by client side |
|
||||||
| ozone.scm.datanode.address | SCM server name and port | Used by datanode to talk to SCM |
|
| ozone.scm.datanode.address | SCM server name and port | Used by datanode to talk to SCM |
|
||||||
| ozone.ksm.address | OM server name | Used by Ozone handler and Ozone file system. |
|
| ozone.om.address | OM server name | Used by Ozone handler and Ozone file system. |
|
||||||
|
|
||||||
|
|
||||||
#### Sample ozone-site.xml
|
#### Sample ozone-site.xml
|
||||||
|
@ -253,7 +253,7 @@ Ozone File System.
|
||||||
</property>
|
</property>
|
||||||
|
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.ksm.address</name>
|
<name>ozone.om.address</name>
|
||||||
<value>127.0.0.1:9874</value>
|
<value>127.0.0.1:9874</value>
|
||||||
</property>
|
</property>
|
||||||
</configuration>
|
</configuration>
|
||||||
|
@ -286,12 +286,12 @@ ozone --daemon start scm
|
||||||
|
|
||||||
Once SCM gets started, OM must be initialized.
|
Once SCM gets started, OM must be initialized.
|
||||||
```
|
```
|
||||||
ozone ksm -createObjectStore
|
ozone om -createObjectStore
|
||||||
```
|
```
|
||||||
|
|
||||||
Start OM.
|
Start OM.
|
||||||
```
|
```
|
||||||
ozone --daemon start ksm
|
ozone --daemon start om
|
||||||
```
|
```
|
||||||
|
|
||||||
If you would like to start HDFS and Ozone together, you can do that by running
|
If you would like to start HDFS and Ozone together, you can do that by running
|
||||||
|
@ -349,7 +349,7 @@ log4j.additivity.org.apache.hadoop.ozone=false
|
||||||
```
|
```
|
||||||
|
|
||||||
On the SCM/OM side, you will be able to see
|
On the SCM/OM side, you will be able to see
|
||||||
1. `hadoop-hdfs-ksm-hostname.log`
|
1. `hadoop-hdfs-om-hostname.log`
|
||||||
1. `hadoop-hdfs-scm-hostname.log`
|
1. `hadoop-hdfs-scm-hostname.log`
|
||||||
|
|
||||||
## Reporting Bugs
|
## Reporting Bugs
|
||||||
|
|
|
@ -131,10 +131,10 @@ Following are the counters for containers:
|
||||||
|
|
||||||
### Key Space Metrics
|
### Key Space Metrics
|
||||||
|
|
||||||
The metrics for various key space manager operations in HDFS Ozone.
|
The metrics for various Ozone Manager operations in HDFS Ozone.
|
||||||
|
|
||||||
key space manager (KSM) is a service that similar to the Namenode in HDFS.
|
The Ozone Manager (OM) is a service that similar to the Namenode in HDFS.
|
||||||
In the current design of KSM, it maintains metadata of all volumes, buckets and keys.
|
In the current design of OM, it maintains metadata of all volumes, buckets and keys.
|
||||||
These metrics are only available when ozone is enabled.
|
These metrics are only available when ozone is enabled.
|
||||||
|
|
||||||
Following is the set of counters maintained for each key space operation.
|
Following is the set of counters maintained for each key space operation.
|
||||||
|
@ -142,12 +142,12 @@ Following is the set of counters maintained for each key space operation.
|
||||||
*Total number of operation* - We maintain an array which counts how
|
*Total number of operation* - We maintain an array which counts how
|
||||||
many times a specific operation has been performed.
|
many times a specific operation has been performed.
|
||||||
Eg.`NumVolumeCreate` tells us how many times create volume has been
|
Eg.`NumVolumeCreate` tells us how many times create volume has been
|
||||||
invoked in KSM.
|
invoked in OM.
|
||||||
|
|
||||||
*Total number of failed operation* - This type operation is opposite to the above
|
*Total number of failed operation* - This type operation is opposite to the above
|
||||||
operation.
|
operation.
|
||||||
Eg.`NumVolumeCreateFails` tells us how many times create volume has been invoked
|
Eg.`NumVolumeCreateFails` tells us how many times create volume has been invoked
|
||||||
failed in KSM.
|
failed in OM.
|
||||||
|
|
||||||
Following are the counters for each of key space operations.
|
Following are the counters for each of key space operations.
|
||||||
|
|
||||||
|
|
|
@ -56,14 +56,14 @@ This is like DFSClient in HDFS. This acts as the standard client to talk to
|
||||||
Ozone. All other components that we have discussed so far rely on Ozone client
|
Ozone. All other components that we have discussed so far rely on Ozone client
|
||||||
(TODO: Add Ozone client documentation).
|
(TODO: Add Ozone client documentation).
|
||||||
|
|
||||||
## Key Space Manager
|
## Ozone Manager
|
||||||
|
|
||||||
Key Space Manager(KSM) takes care of the Ozone's namespace.
|
Ozone Manager (OM) takes care of the Ozone's namespace.
|
||||||
All ozone entities like volumes, buckets and keys are managed by KSM
|
All ozone entities like volumes, buckets and keys are managed by OM
|
||||||
(TODO: Add KSM documentation). In Short, KSM is the metadata manager for Ozone.
|
(TODO: Add OM documentation). In short, OM is the metadata manager for Ozone.
|
||||||
KSM talks to blockManager(SCM) to get blocks and passes it on to the Ozone
|
OM talks to blockManager(SCM) to get blocks and passes it on to the Ozone
|
||||||
client. Ozone client writes data to these blocks.
|
client. Ozone client writes data to these blocks.
|
||||||
KSM will eventually be replicated via Apache Ratis for High Availability.
|
OM will eventually be replicated via Apache Ratis for High Availability.
|
||||||
|
|
||||||
## Storage Container Manager
|
## Storage Container Manager
|
||||||
Storage Container Manager (SCM) is the block and cluster manager for Ozone.
|
Storage Container Manager (SCM) is the block and cluster manager for Ozone.
|
||||||
|
|
|
@ -166,7 +166,7 @@
|
||||||
<path d="M307.5,148.5 L433.5,148.5" id="Line" stroke="#000000" fill="#000000" stroke-linecap="square"></path>
|
<path d="M307.5,148.5 L433.5,148.5" id="Line" stroke="#000000" fill="#000000" stroke-linecap="square"></path>
|
||||||
<path id="Line-decoration-1" d="M433.5,148.5 L422.7,145.5 L422.7,151.5 L433.5,148.5 Z" stroke="#000000" fill="#000000" stroke-linecap="square"></path>
|
<path id="Line-decoration-1" d="M433.5,148.5 L422.7,145.5 L422.7,151.5 L433.5,148.5 Z" stroke="#000000" fill="#000000" stroke-linecap="square"></path>
|
||||||
<path d="M4,232 L699,232" id="Line" stroke="#000000" stroke-width="2" stroke-linecap="square" stroke-dasharray="5,2,5"></path>
|
<path d="M4,232 L699,232" id="Line" stroke="#000000" stroke-width="2" stroke-linecap="square" stroke-dasharray="5,2,5"></path>
|
||||||
<g id="KSM" transform="translate(432.000000, 132.000000)">
|
<g id="OM" transform="translate(432.000000, 132.000000)">
|
||||||
<g id="Rectangle-3">
|
<g id="Rectangle-3">
|
||||||
<use fill="#C6D4F9" fill-rule="evenodd" xlink:href="#path-19"></use>
|
<use fill="#C6D4F9" fill-rule="evenodd" xlink:href="#path-19"></use>
|
||||||
<rect stroke="#000000" stroke-width="1" x="0.5" y="0.5" width="225" height="35" rx="8"></rect>
|
<rect stroke="#000000" stroke-width="1" x="0.5" y="0.5" width="225" height="35" rx="8"></rect>
|
||||||
|
|
Before Width: | Height: | Size: 16 KiB After Width: | Height: | Size: 16 KiB |
|
@ -21,7 +21,7 @@ import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
||||||
import org.apache.hadoop.ozone.client.OzoneClient;
|
import org.apache.hadoop.ozone.client.OzoneClient;
|
||||||
import org.apache.hadoop.ozone.ksm.KeySpaceManager;
|
import org.apache.hadoop.ozone.om.OzoneManager;
|
||||||
import org.apache.hadoop.hdds.scm.protocolPB
|
import org.apache.hadoop.hdds.scm.protocolPB
|
||||||
.StorageContainerLocationProtocolClientSideTranslatorPB;
|
.StorageContainerLocationProtocolClientSideTranslatorPB;
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
|
@ -82,12 +82,12 @@ public interface MiniOzoneCluster {
|
||||||
StorageContainerManager getStorageContainerManager();
|
StorageContainerManager getStorageContainerManager();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns {@link KeySpaceManager} associated with this
|
* Returns {@link OzoneManager} associated with this
|
||||||
* {@link MiniOzoneCluster} instance.
|
* {@link MiniOzoneCluster} instance.
|
||||||
*
|
*
|
||||||
* @return {@link KeySpaceManager} instance
|
* @return {@link OzoneManager} instance
|
||||||
*/
|
*/
|
||||||
KeySpaceManager getKeySpaceManager();
|
OzoneManager getOzoneManager();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the list of {@link HddsDatanodeService} which are part of this
|
* Returns the list of {@link HddsDatanodeService} which are part of this
|
||||||
|
@ -141,11 +141,11 @@ public interface MiniOzoneCluster {
|
||||||
void restartStorageContainerManager() throws IOException;
|
void restartStorageContainerManager() throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Restarts KeySpaceManager instance.
|
* Restarts OzoneManager instance.
|
||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
void restartKeySpaceManager() throws IOException;
|
void restartOzoneManager() throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Restart a particular HddsDatanode.
|
* Restart a particular HddsDatanode.
|
||||||
|
@ -184,13 +184,13 @@ public interface MiniOzoneCluster {
|
||||||
protected Optional<Integer> hbInterval = Optional.empty();
|
protected Optional<Integer> hbInterval = Optional.empty();
|
||||||
protected Optional<Integer> hbProcessorInterval = Optional.empty();
|
protected Optional<Integer> hbProcessorInterval = Optional.empty();
|
||||||
protected Optional<String> scmId = Optional.empty();
|
protected Optional<String> scmId = Optional.empty();
|
||||||
protected Optional<String> ksmId = Optional.empty();
|
protected Optional<String> omId = Optional.empty();
|
||||||
|
|
||||||
protected Boolean ozoneEnabled = true;
|
protected Boolean ozoneEnabled = true;
|
||||||
protected Boolean randomContainerPort = true;
|
protected Boolean randomContainerPort = true;
|
||||||
|
|
||||||
// Use relative smaller number of handlers for testing
|
// Use relative smaller number of handlers for testing
|
||||||
protected int numOfKsmHandlers = 20;
|
protected int numOfOmHandlers = 20;
|
||||||
protected int numOfScmHandlers = 20;
|
protected int numOfScmHandlers = 20;
|
||||||
protected int numOfDatanodes = 1;
|
protected int numOfDatanodes = 1;
|
||||||
|
|
||||||
|
@ -226,14 +226,14 @@ public interface MiniOzoneCluster {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Sets the KSM id.
|
* Sets the OM id.
|
||||||
*
|
*
|
||||||
* @param id KSM Id
|
* @param id OM Id
|
||||||
*
|
*
|
||||||
* @return MiniOzoneCluster.Builder
|
* @return MiniOzoneCluster.Builder
|
||||||
*/
|
*/
|
||||||
public Builder setKsmId(String id) {
|
public Builder setOmId(String id) {
|
||||||
ksmId = Optional.of(id);
|
omId = Optional.of(id);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -34,10 +34,10 @@ import org.apache.hadoop.net.NetUtils;
|
||||||
import org.apache.hadoop.ozone.client.OzoneClient;
|
import org.apache.hadoop.ozone.client.OzoneClient;
|
||||||
import org.apache.hadoop.ozone.client.OzoneClientFactory;
|
import org.apache.hadoop.ozone.client.OzoneClientFactory;
|
||||||
import org.apache.hadoop.ozone.client.rest.OzoneException;
|
import org.apache.hadoop.ozone.client.rest.OzoneException;
|
||||||
import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
|
import org.apache.hadoop.ozone.om.OMConfigKeys;
|
||||||
import org.apache.hadoop.ozone.ksm.KeySpaceManager;
|
import org.apache.hadoop.ozone.om.OzoneManager;
|
||||||
import org.apache.hadoop.hdds.scm.server.SCMStorage;
|
import org.apache.hadoop.hdds.scm.server.SCMStorage;
|
||||||
import org.apache.hadoop.ozone.ksm.KSMStorage;
|
import org.apache.hadoop.ozone.om.OMStorage;
|
||||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||||
import org.apache.hadoop.hdds.scm.protocolPB
|
import org.apache.hadoop.hdds.scm.protocolPB
|
||||||
.StorageContainerLocationProtocolClientSideTranslatorPB;
|
.StorageContainerLocationProtocolClientSideTranslatorPB;
|
||||||
|
@ -73,7 +73,7 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* MiniOzoneCluster creates a complete in-process Ozone cluster suitable for
|
* MiniOzoneCluster creates a complete in-process Ozone cluster suitable for
|
||||||
* running tests. The cluster consists of a KeySpaceManager,
|
* running tests. The cluster consists of a OzoneManager,
|
||||||
* StorageContainerManager and multiple DataNodes.
|
* StorageContainerManager and multiple DataNodes.
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
|
@ -84,7 +84,7 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
|
||||||
|
|
||||||
private final OzoneConfiguration conf;
|
private final OzoneConfiguration conf;
|
||||||
private final StorageContainerManager scm;
|
private final StorageContainerManager scm;
|
||||||
private final KeySpaceManager ksm;
|
private final OzoneManager ozoneManager;
|
||||||
private final List<HddsDatanodeService> hddsDatanodes;
|
private final List<HddsDatanodeService> hddsDatanodes;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -93,11 +93,11 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
|
||||||
* @throws IOException if there is an I/O error
|
* @throws IOException if there is an I/O error
|
||||||
*/
|
*/
|
||||||
private MiniOzoneClusterImpl(OzoneConfiguration conf,
|
private MiniOzoneClusterImpl(OzoneConfiguration conf,
|
||||||
KeySpaceManager ksm,
|
OzoneManager ozoneManager,
|
||||||
StorageContainerManager scm,
|
StorageContainerManager scm,
|
||||||
List<HddsDatanodeService> hddsDatanodes) {
|
List<HddsDatanodeService> hddsDatanodes) {
|
||||||
this.conf = conf;
|
this.conf = conf;
|
||||||
this.ksm = ksm;
|
this.ozoneManager = ozoneManager;
|
||||||
this.scm = scm;
|
this.scm = scm;
|
||||||
this.hddsDatanodes = hddsDatanodes;
|
this.hddsDatanodes = hddsDatanodes;
|
||||||
}
|
}
|
||||||
|
@ -147,8 +147,8 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public KeySpaceManager getKeySpaceManager() {
|
public OzoneManager getOzoneManager() {
|
||||||
return this.ksm;
|
return this.ozoneManager;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -209,9 +209,9 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void restartKeySpaceManager() throws IOException {
|
public void restartOzoneManager() throws IOException {
|
||||||
ksm.stop();
|
ozoneManager.stop();
|
||||||
ksm.start();
|
ozoneManager.start();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -247,10 +247,10 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
|
||||||
scm.getClientProtocolServer().getScmInfo().getClusterId()));
|
scm.getClientProtocolServer().getScmInfo().getClusterId()));
|
||||||
FileUtils.deleteDirectory(baseDir);
|
FileUtils.deleteDirectory(baseDir);
|
||||||
|
|
||||||
if (ksm != null) {
|
if (ozoneManager != null) {
|
||||||
LOG.info("Shutting down the keySpaceManager");
|
LOG.info("Shutting down the OzoneManager");
|
||||||
ksm.stop();
|
ozoneManager.stop();
|
||||||
ksm.join();
|
ozoneManager.join();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (scm != null) {
|
if (scm != null) {
|
||||||
|
@ -291,11 +291,11 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
|
||||||
initializeConfiguration();
|
initializeConfiguration();
|
||||||
StorageContainerManager scm = createSCM();
|
StorageContainerManager scm = createSCM();
|
||||||
scm.start();
|
scm.start();
|
||||||
KeySpaceManager ksm = createKSM();
|
OzoneManager om = createOM();
|
||||||
ksm.start();
|
om.start();
|
||||||
List<HddsDatanodeService> hddsDatanodes = createHddsDatanodes(scm);
|
List<HddsDatanodeService> hddsDatanodes = createHddsDatanodes(scm);
|
||||||
hddsDatanodes.forEach((datanode) -> datanode.start(null));
|
hddsDatanodes.forEach((datanode) -> datanode.start(null));
|
||||||
return new MiniOzoneClusterImpl(conf, ksm, scm, hddsDatanodes);
|
return new MiniOzoneClusterImpl(conf, om, scm, hddsDatanodes);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -331,20 +331,20 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a new KeySpaceManager instance.
|
* Creates a new OzoneManager instance.
|
||||||
*
|
*
|
||||||
* @return {@link KeySpaceManager}
|
* @return {@link OzoneManager}
|
||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
private KeySpaceManager createKSM() throws IOException {
|
private OzoneManager createOM() throws IOException {
|
||||||
configureKSM();
|
configureOM();
|
||||||
KSMStorage ksmStore = new KSMStorage(conf);
|
OMStorage omStore = new OMStorage(conf);
|
||||||
ksmStore.setClusterId(clusterId);
|
omStore.setClusterId(clusterId);
|
||||||
ksmStore.setScmId(scmId.get());
|
omStore.setScmId(scmId.get());
|
||||||
ksmStore.setKsmId(ksmId.orElse(UUID.randomUUID().toString()));
|
omStore.setOmId(omId.orElse(UUID.randomUUID().toString()));
|
||||||
ksmStore.initialize();
|
omStore.initialize();
|
||||||
return KeySpaceManager.createKSM(null, conf);
|
return OzoneManager.createOm(null, conf);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -415,10 +415,10 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
private void configureKSM() {
|
private void configureOM() {
|
||||||
conf.set(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY, "127.0.0.1:0");
|
conf.set(OMConfigKeys.OZONE_OM_ADDRESS_KEY, "127.0.0.1:0");
|
||||||
conf.set(KSMConfigKeys.OZONE_KSM_HTTP_ADDRESS_KEY, "127.0.0.1:0");
|
conf.set(OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY, "127.0.0.1:0");
|
||||||
conf.setInt(KSMConfigKeys.OZONE_KSM_HANDLER_COUNT_KEY, numOfKsmHandlers);
|
conf.setInt(OMConfigKeys.OZONE_OM_HANDLER_COUNT_KEY, numOfOmHandlers);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void configureHddsDatanodes() {
|
private void configureHddsDatanodes() {
|
||||||
|
|
|
@ -18,7 +18,7 @@
|
||||||
package org.apache.hadoop.ozone;
|
package org.apache.hadoop.ozone;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.TestConfigurationFieldsBase;
|
import org.apache.hadoop.conf.TestConfigurationFieldsBase;
|
||||||
import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
|
import org.apache.hadoop.ozone.om.OMConfigKeys;
|
||||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -31,7 +31,7 @@ public class TestOzoneConfigurationFields extends TestConfigurationFieldsBase {
|
||||||
xmlFilename = new String("ozone-default.xml");
|
xmlFilename = new String("ozone-default.xml");
|
||||||
configurationClasses =
|
configurationClasses =
|
||||||
new Class[] {OzoneConfigKeys.class, ScmConfigKeys.class,
|
new Class[] {OzoneConfigKeys.class, ScmConfigKeys.class,
|
||||||
KSMConfigKeys.class};
|
OMConfigKeys.class};
|
||||||
errorIfMissingConfigProps = true;
|
errorIfMissingConfigProps = true;
|
||||||
errorIfMissingXmlProps = true;
|
errorIfMissingXmlProps = true;
|
||||||
xmlPropsToSkipCompare.add("hadoop.tags.custom");
|
xmlPropsToSkipCompare.add("hadoop.tags.custom");
|
||||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.hadoop.hdds.scm.server.SCMClientProtocolServer;
|
||||||
import org.apache.hadoop.hdds.scm.server.SCMStorage;
|
import org.apache.hadoop.hdds.scm.server.SCMStorage;
|
||||||
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
||||||
import org.apache.hadoop.ozone.container.ContainerTestHelper;
|
import org.apache.hadoop.ozone.container.ContainerTestHelper;
|
||||||
|
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
||||||
import org.apache.hadoop.ozone.protocol.commands.DeleteBlocksCommand;
|
import org.apache.hadoop.ozone.protocol.commands.DeleteBlocksCommand;
|
||||||
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
|
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
|
@ -60,8 +61,7 @@ import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
|
|
||||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||||
|
|
||||||
import org.junit.rules.Timeout;
|
import org.junit.rules.Timeout;
|
||||||
|
@ -211,7 +211,7 @@ public class TestStorageContainerManager {
|
||||||
// Create {numKeys} random names keys.
|
// Create {numKeys} random names keys.
|
||||||
TestStorageContainerManagerHelper helper =
|
TestStorageContainerManagerHelper helper =
|
||||||
new TestStorageContainerManagerHelper(cluster, conf);
|
new TestStorageContainerManagerHelper(cluster, conf);
|
||||||
Map<String, KsmKeyInfo> keyLocations = helper.createKeys(numKeys, 4096);
|
Map<String, OmKeyInfo> keyLocations = helper.createKeys(numKeys, 4096);
|
||||||
|
|
||||||
Map<Long, List<Long>> containerBlocks = createDeleteTXLog(delLog,
|
Map<Long, List<Long>> containerBlocks = createDeleteTXLog(delLog,
|
||||||
keyLocations, helper);
|
keyLocations, helper);
|
||||||
|
@ -293,7 +293,7 @@ public class TestStorageContainerManager {
|
||||||
// Create {numKeys} random names keys.
|
// Create {numKeys} random names keys.
|
||||||
TestStorageContainerManagerHelper helper =
|
TestStorageContainerManagerHelper helper =
|
||||||
new TestStorageContainerManagerHelper(cluster, conf);
|
new TestStorageContainerManagerHelper(cluster, conf);
|
||||||
Map<String, KsmKeyInfo> keyLocations = helper.createKeys(numKeys, 4096);
|
Map<String, OmKeyInfo> keyLocations = helper.createKeys(numKeys, 4096);
|
||||||
|
|
||||||
createDeleteTXLog(delLog, keyLocations, helper);
|
createDeleteTXLog(delLog, keyLocations, helper);
|
||||||
// Verify a few TX gets created in the TX log.
|
// Verify a few TX gets created in the TX log.
|
||||||
|
@ -320,13 +320,13 @@ public class TestStorageContainerManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
private Map<Long, List<Long>> createDeleteTXLog(DeletedBlockLog delLog,
|
private Map<Long, List<Long>> createDeleteTXLog(DeletedBlockLog delLog,
|
||||||
Map<String, KsmKeyInfo> keyLocations,
|
Map<String, OmKeyInfo> keyLocations,
|
||||||
TestStorageContainerManagerHelper helper) throws IOException {
|
TestStorageContainerManagerHelper helper) throws IOException {
|
||||||
// These keys will be written into a bunch of containers,
|
// These keys will be written into a bunch of containers,
|
||||||
// gets a set of container names, verify container containerBlocks
|
// gets a set of container names, verify container containerBlocks
|
||||||
// on datanodes.
|
// on datanodes.
|
||||||
Set<Long> containerNames = new HashSet<>();
|
Set<Long> containerNames = new HashSet<>();
|
||||||
for (Map.Entry<String, KsmKeyInfo> entry : keyLocations.entrySet()) {
|
for (Map.Entry<String, OmKeyInfo> entry : keyLocations.entrySet()) {
|
||||||
entry.getValue().getLatestVersionLocations().getLocationList()
|
entry.getValue().getLatestVersionLocations().getLocationList()
|
||||||
.forEach(loc -> containerNames.add(loc.getContainerID()));
|
.forEach(loc -> containerNames.add(loc.getContainerID()));
|
||||||
}
|
}
|
||||||
|
@ -334,7 +334,7 @@ public class TestStorageContainerManager {
|
||||||
// Total number of containerBlocks of these containers should be equal to
|
// Total number of containerBlocks of these containers should be equal to
|
||||||
// total number of containerBlocks via creation call.
|
// total number of containerBlocks via creation call.
|
||||||
int totalCreatedBlocks = 0;
|
int totalCreatedBlocks = 0;
|
||||||
for (KsmKeyInfo info : keyLocations.values()) {
|
for (OmKeyInfo info : keyLocations.values()) {
|
||||||
totalCreatedBlocks += info.getKeyLocationVersions().size();
|
totalCreatedBlocks += info.getKeyLocationVersions().size();
|
||||||
}
|
}
|
||||||
Assert.assertTrue(totalCreatedBlocks > 0);
|
Assert.assertTrue(totalCreatedBlocks > 0);
|
||||||
|
@ -343,8 +343,8 @@ public class TestStorageContainerManager {
|
||||||
|
|
||||||
// Create a deletion TX for each key.
|
// Create a deletion TX for each key.
|
||||||
Map<Long, List<Long>> containerBlocks = Maps.newHashMap();
|
Map<Long, List<Long>> containerBlocks = Maps.newHashMap();
|
||||||
for (KsmKeyInfo info : keyLocations.values()) {
|
for (OmKeyInfo info : keyLocations.values()) {
|
||||||
List<KsmKeyLocationInfo> list =
|
List<OmKeyLocationInfo> list =
|
||||||
info.getLatestVersionLocations().getLocationList();
|
info.getLatestVersionLocations().getLocationList();
|
||||||
list.forEach(location -> {
|
list.forEach(location -> {
|
||||||
if (containerBlocks.containsKey(location.getContainerID())) {
|
if (containerBlocks.containsKey(location.getContainerID())) {
|
||||||
|
|
|
@ -30,8 +30,8 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
|
import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
|
||||||
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
|
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
||||||
import org.apache.hadoop.ozone.web.handlers.BucketArgs;
|
import org.apache.hadoop.ozone.web.handlers.BucketArgs;
|
||||||
import org.apache.hadoop.ozone.web.handlers.KeyArgs;
|
import org.apache.hadoop.ozone.web.handlers.KeyArgs;
|
||||||
import org.apache.hadoop.ozone.web.handlers.UserArgs;
|
import org.apache.hadoop.ozone.web.handlers.UserArgs;
|
||||||
|
@ -67,9 +67,9 @@ public class TestStorageContainerManagerHelper {
|
||||||
storageHandler = new ObjectStoreHandler(conf).getStorageHandler();
|
storageHandler = new ObjectStoreHandler(conf).getStorageHandler();
|
||||||
}
|
}
|
||||||
|
|
||||||
public Map<String, KsmKeyInfo> createKeys(int numOfKeys, int keySize)
|
public Map<String, OmKeyInfo> createKeys(int numOfKeys, int keySize)
|
||||||
throws Exception {
|
throws Exception {
|
||||||
Map<String, KsmKeyInfo> keyLocationMap = Maps.newHashMap();
|
Map<String, OmKeyInfo> keyLocationMap = Maps.newHashMap();
|
||||||
String volume = "volume" + RandomStringUtils.randomNumeric(5);
|
String volume = "volume" + RandomStringUtils.randomNumeric(5);
|
||||||
String bucket = "bucket" + RandomStringUtils.randomNumeric(5);
|
String bucket = "bucket" + RandomStringUtils.randomNumeric(5);
|
||||||
String userName = "user" + RandomStringUtils.randomNumeric(5);
|
String userName = "user" + RandomStringUtils.randomNumeric(5);
|
||||||
|
@ -104,12 +104,12 @@ public class TestStorageContainerManagerHelper {
|
||||||
}
|
}
|
||||||
|
|
||||||
for (String key : keyNames) {
|
for (String key : keyNames) {
|
||||||
KsmKeyArgs arg = new KsmKeyArgs.Builder()
|
OmKeyArgs arg = new OmKeyArgs.Builder()
|
||||||
.setVolumeName(volume)
|
.setVolumeName(volume)
|
||||||
.setBucketName(bucket)
|
.setBucketName(bucket)
|
||||||
.setKeyName(key)
|
.setKeyName(key)
|
||||||
.build();
|
.build();
|
||||||
KsmKeyInfo location = cluster.getKeySpaceManager()
|
OmKeyInfo location = cluster.getOzoneManager()
|
||||||
.lookupKey(arg);
|
.lookupKey(arg);
|
||||||
keyLocationMap.put(key, location);
|
keyLocationMap.put(key, location);
|
||||||
}
|
}
|
||||||
|
|
|
@ -77,10 +77,10 @@ public class TestOzoneRestClient {
|
||||||
OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
|
OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
|
||||||
cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
|
cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
|
||||||
cluster.waitForClusterToBeReady();
|
cluster.waitForClusterToBeReady();
|
||||||
InetSocketAddress ksmHttpAddress = cluster.getKeySpaceManager()
|
InetSocketAddress omHttpAddress = cluster.getOzoneManager()
|
||||||
.getHttpServer().getHttpAddress();
|
.getHttpServer().getHttpAddress();
|
||||||
ozClient = OzoneClientFactory.getRestClient(ksmHttpAddress.getHostName(),
|
ozClient = OzoneClientFactory.getRestClient(omHttpAddress.getHostName(),
|
||||||
ksmHttpAddress.getPort(), conf);
|
omHttpAddress.getPort(), conf);
|
||||||
store = ozClient.getObjectStore();
|
store = ozClient.getObjectStore();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -39,10 +39,10 @@ import org.apache.hadoop.hdds.client.ReplicationType;
|
||||||
import org.apache.hadoop.ozone.client.VolumeArgs;
|
import org.apache.hadoop.ozone.client.VolumeArgs;
|
||||||
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
|
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
|
||||||
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
||||||
import org.apache.hadoop.ozone.ksm.KeySpaceManager;
|
import org.apache.hadoop.ozone.om.OzoneManager;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.ozone.client.rest.OzoneException;
|
import org.apache.hadoop.ozone.client.rest.OzoneException;
|
||||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||||
|
@ -73,7 +73,7 @@ public class TestOzoneRpcClient {
|
||||||
private static MiniOzoneCluster cluster = null;
|
private static MiniOzoneCluster cluster = null;
|
||||||
private static OzoneClient ozClient = null;
|
private static OzoneClient ozClient = null;
|
||||||
private static ObjectStore store = null;
|
private static ObjectStore store = null;
|
||||||
private static KeySpaceManager keySpaceManager;
|
private static OzoneManager ozoneManager;
|
||||||
private static StorageContainerLocationProtocolClientSideTranslatorPB
|
private static StorageContainerLocationProtocolClientSideTranslatorPB
|
||||||
storageContainerLocationClient;
|
storageContainerLocationClient;
|
||||||
|
|
||||||
|
@ -97,7 +97,7 @@ public class TestOzoneRpcClient {
|
||||||
store = ozClient.getObjectStore();
|
store = ozClient.getObjectStore();
|
||||||
storageContainerLocationClient =
|
storageContainerLocationClient =
|
||||||
cluster.getStorageContainerLocationClient();
|
cluster.getStorageContainerLocationClient();
|
||||||
keySpaceManager = cluster.getKeySpaceManager();
|
ozoneManager = cluster.getOzoneManager();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -376,7 +376,7 @@ public class TestOzoneRpcClient {
|
||||||
private boolean verifyRatisReplication(String volumeName, String bucketName,
|
private boolean verifyRatisReplication(String volumeName, String bucketName,
|
||||||
String keyName, ReplicationType type, ReplicationFactor factor)
|
String keyName, ReplicationType type, ReplicationFactor factor)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
|
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
|
||||||
.setVolumeName(volumeName)
|
.setVolumeName(volumeName)
|
||||||
.setBucketName(bucketName)
|
.setBucketName(bucketName)
|
||||||
.setKeyName(keyName)
|
.setKeyName(keyName)
|
||||||
|
@ -385,8 +385,8 @@ public class TestOzoneRpcClient {
|
||||||
HddsProtos.ReplicationType.valueOf(type.toString());
|
HddsProtos.ReplicationType.valueOf(type.toString());
|
||||||
HddsProtos.ReplicationFactor replicationFactor =
|
HddsProtos.ReplicationFactor replicationFactor =
|
||||||
HddsProtos.ReplicationFactor.valueOf(factor.getValue());
|
HddsProtos.ReplicationFactor.valueOf(factor.getValue());
|
||||||
KsmKeyInfo keyInfo = keySpaceManager.lookupKey(keyArgs);
|
OmKeyInfo keyInfo = ozoneManager.lookupKey(keyArgs);
|
||||||
for (KsmKeyLocationInfo info:
|
for (OmKeyLocationInfo info:
|
||||||
keyInfo.getLatestVersionLocations().getLocationList()) {
|
keyInfo.getLatestVersionLocations().getLocationList()) {
|
||||||
ContainerInfo container =
|
ContainerInfo container =
|
||||||
storageContainerLocationClient.getContainer(info.getContainerID());
|
storageContainerLocationClient.getContainer(info.getContainerID());
|
||||||
|
|
|
@ -37,10 +37,10 @@ import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
|
import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
|
||||||
import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl;
|
import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl;
|
||||||
import org.apache.hadoop.ozone.ksm.KeySpaceManager;
|
import org.apache.hadoop.ozone.om.OzoneManager;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfoGroup;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
|
||||||
import org.apache.hadoop.ozone.ozShell.TestOzoneShell;
|
import org.apache.hadoop.ozone.ozShell.TestOzoneShell;
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
import org.apache.hadoop.utils.MetadataStore;
|
import org.apache.hadoop.utils.MetadataStore;
|
||||||
|
@ -61,7 +61,7 @@ public class TestBlockDeletion {
|
||||||
private static ObjectStore store;
|
private static ObjectStore store;
|
||||||
private static ContainerManagerImpl dnContainerManager = null;
|
private static ContainerManagerImpl dnContainerManager = null;
|
||||||
private static StorageContainerManager scm = null;
|
private static StorageContainerManager scm = null;
|
||||||
private static KeySpaceManager ksm = null;
|
private static OzoneManager om = null;
|
||||||
private static Set<Long> containerIdsWithDeletedBlocks;
|
private static Set<Long> containerIdsWithDeletedBlocks;
|
||||||
|
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
|
@ -88,7 +88,7 @@ public class TestBlockDeletion {
|
||||||
dnContainerManager =
|
dnContainerManager =
|
||||||
(ContainerManagerImpl) cluster.getHddsDatanodes().get(0)
|
(ContainerManagerImpl) cluster.getHddsDatanodes().get(0)
|
||||||
.getDatanodeStateMachine().getContainer().getContainerManager();
|
.getDatanodeStateMachine().getContainer().getContainerManager();
|
||||||
ksm = cluster.getKeySpaceManager();
|
om = cluster.getOzoneManager();
|
||||||
scm = cluster.getStorageContainerManager();
|
scm = cluster.getStorageContainerManager();
|
||||||
containerIdsWithDeletedBlocks = new HashSet<>();
|
containerIdsWithDeletedBlocks = new HashSet<>();
|
||||||
}
|
}
|
||||||
|
@ -112,23 +112,23 @@ public class TestBlockDeletion {
|
||||||
out.write(value.getBytes());
|
out.write(value.getBytes());
|
||||||
out.close();
|
out.close();
|
||||||
|
|
||||||
KsmKeyArgs keyArgs = new KsmKeyArgs.Builder().setVolumeName(volumeName)
|
OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName)
|
||||||
.setBucketName(bucketName).setKeyName(keyName).setDataSize(0)
|
.setBucketName(bucketName).setKeyName(keyName).setDataSize(0)
|
||||||
.setType(HddsProtos.ReplicationType.STAND_ALONE)
|
.setType(HddsProtos.ReplicationType.STAND_ALONE)
|
||||||
.setFactor(HddsProtos.ReplicationFactor.ONE).build();
|
.setFactor(HddsProtos.ReplicationFactor.ONE).build();
|
||||||
List<KsmKeyLocationInfoGroup> ksmKeyLocationInfoGroupList =
|
List<OmKeyLocationInfoGroup> omKeyLocationInfoGroupList =
|
||||||
ksm.lookupKey(keyArgs).getKeyLocationVersions();
|
om.lookupKey(keyArgs).getKeyLocationVersions();
|
||||||
|
|
||||||
// verify key blocks were created in DN.
|
// verify key blocks were created in DN.
|
||||||
Assert.assertTrue(verifyBlocksCreated(ksmKeyLocationInfoGroupList));
|
Assert.assertTrue(verifyBlocksCreated(omKeyLocationInfoGroupList));
|
||||||
// No containers with deleted blocks
|
// No containers with deleted blocks
|
||||||
Assert.assertTrue(containerIdsWithDeletedBlocks.isEmpty());
|
Assert.assertTrue(containerIdsWithDeletedBlocks.isEmpty());
|
||||||
// Delete transactionIds for the containers should be 0
|
// Delete transactionIds for the containers should be 0
|
||||||
matchContainerTransactionIds();
|
matchContainerTransactionIds();
|
||||||
ksm.deleteKey(keyArgs);
|
om.deleteKey(keyArgs);
|
||||||
Thread.sleep(5000);
|
Thread.sleep(5000);
|
||||||
// The blocks should be deleted in the DN.
|
// The blocks should be deleted in the DN.
|
||||||
Assert.assertTrue(verifyBlocksDeleted(ksmKeyLocationInfoGroupList));
|
Assert.assertTrue(verifyBlocksDeleted(omKeyLocationInfoGroupList));
|
||||||
|
|
||||||
// Few containers with deleted blocks
|
// Few containers with deleted blocks
|
||||||
Assert.assertTrue(!containerIdsWithDeletedBlocks.isEmpty());
|
Assert.assertTrue(!containerIdsWithDeletedBlocks.isEmpty());
|
||||||
|
@ -155,7 +155,7 @@ public class TestBlockDeletion {
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean verifyBlocksCreated(
|
private boolean verifyBlocksCreated(
|
||||||
List<KsmKeyLocationInfoGroup> ksmKeyLocationInfoGroups)
|
List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return performOperationOnKeyContainers((blockID) -> {
|
return performOperationOnKeyContainers((blockID) -> {
|
||||||
try {
|
try {
|
||||||
|
@ -166,11 +166,11 @@ public class TestBlockDeletion {
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
e.printStackTrace();
|
e.printStackTrace();
|
||||||
}
|
}
|
||||||
}, ksmKeyLocationInfoGroups);
|
}, omKeyLocationInfoGroups);
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean verifyBlocksDeleted(
|
private boolean verifyBlocksDeleted(
|
||||||
List<KsmKeyLocationInfoGroup> ksmKeyLocationInfoGroups)
|
List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return performOperationOnKeyContainers((blockID) -> {
|
return performOperationOnKeyContainers((blockID) -> {
|
||||||
try {
|
try {
|
||||||
|
@ -186,19 +186,20 @@ public class TestBlockDeletion {
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
e.printStackTrace();
|
e.printStackTrace();
|
||||||
}
|
}
|
||||||
}, ksmKeyLocationInfoGroups);
|
}, omKeyLocationInfoGroups);
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean performOperationOnKeyContainers(Consumer<BlockID> consumer,
|
private boolean performOperationOnKeyContainers(Consumer<BlockID> consumer,
|
||||||
List<KsmKeyLocationInfoGroup> ksmKeyLocationInfoGroups)
|
List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
|
||||||
try {
|
try {
|
||||||
for (KsmKeyLocationInfoGroup ksmKeyLocationInfoGroup : ksmKeyLocationInfoGroups) {
|
for (OmKeyLocationInfoGroup omKeyLocationInfoGroup :
|
||||||
List<KsmKeyLocationInfo> ksmKeyLocationInfos =
|
omKeyLocationInfoGroups) {
|
||||||
ksmKeyLocationInfoGroup.getLocationList();
|
List<OmKeyLocationInfo> omKeyLocationInfos =
|
||||||
for (KsmKeyLocationInfo ksmKeyLocationInfo : ksmKeyLocationInfos) {
|
omKeyLocationInfoGroup.getLocationList();
|
||||||
BlockID blockID = ksmKeyLocationInfo.getBlockID();
|
for (OmKeyLocationInfo omKeyLocationInfo : omKeyLocationInfos) {
|
||||||
|
BlockID blockID = omKeyLocationInfo.getBlockID();
|
||||||
consumer.accept(blockID);
|
consumer.accept(blockID);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -34,8 +34,8 @@ import org.apache.hadoop.ozone.client.OzoneClientFactory;
|
||||||
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
||||||
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
|
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
||||||
import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
|
import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
import org.junit.AfterClass;
|
import org.junit.AfterClass;
|
||||||
|
@ -45,7 +45,6 @@ import org.junit.Test;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Random;
|
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
|
|
||||||
public class TestCloseContainerByPipeline {
|
public class TestCloseContainerByPipeline {
|
||||||
|
@ -98,17 +97,17 @@ public class TestCloseContainerByPipeline {
|
||||||
key.close();
|
key.close();
|
||||||
|
|
||||||
//get the name of a valid container
|
//get the name of a valid container
|
||||||
KsmKeyArgs keyArgs =
|
OmKeyArgs keyArgs =
|
||||||
new KsmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
|
new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
|
||||||
.setType(HddsProtos.ReplicationType.STAND_ALONE)
|
.setType(HddsProtos.ReplicationType.STAND_ALONE)
|
||||||
.setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
|
.setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
|
||||||
.setKeyName("testCloseContainer").build();
|
.setKeyName("testCloseContainer").build();
|
||||||
|
|
||||||
KsmKeyLocationInfo ksmKeyLocationInfo =
|
OmKeyLocationInfo omKeyLocationInfo =
|
||||||
cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions()
|
cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
|
||||||
.get(0).getBlocksLatestVersionOnly().get(0);
|
.get(0).getBlocksLatestVersionOnly().get(0);
|
||||||
|
|
||||||
long containerID = ksmKeyLocationInfo.getContainerID();
|
long containerID = omKeyLocationInfo.getContainerID();
|
||||||
List<DatanodeDetails> datanodes = cluster.getStorageContainerManager()
|
List<DatanodeDetails> datanodes = cluster.getStorageContainerManager()
|
||||||
.getScmContainerManager().getContainerWithPipeline(containerID)
|
.getScmContainerManager().getContainerWithPipeline(containerID)
|
||||||
.getPipeline().getMachines();
|
.getPipeline().getMachines();
|
||||||
|
@ -153,17 +152,17 @@ public class TestCloseContainerByPipeline {
|
||||||
key.close();
|
key.close();
|
||||||
|
|
||||||
//get the name of a valid container
|
//get the name of a valid container
|
||||||
KsmKeyArgs keyArgs =
|
OmKeyArgs keyArgs =
|
||||||
new KsmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
|
new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
|
||||||
.setType(HddsProtos.ReplicationType.STAND_ALONE)
|
.setType(HddsProtos.ReplicationType.STAND_ALONE)
|
||||||
.setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
|
.setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
|
||||||
.setKeyName("standalone").build();
|
.setKeyName("standalone").build();
|
||||||
|
|
||||||
KsmKeyLocationInfo ksmKeyLocationInfo =
|
OmKeyLocationInfo omKeyLocationInfo =
|
||||||
cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions()
|
cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
|
||||||
.get(0).getBlocksLatestVersionOnly().get(0);
|
.get(0).getBlocksLatestVersionOnly().get(0);
|
||||||
|
|
||||||
long containerID = ksmKeyLocationInfo.getContainerID();
|
long containerID = omKeyLocationInfo.getContainerID();
|
||||||
List<DatanodeDetails> datanodes = cluster.getStorageContainerManager()
|
List<DatanodeDetails> datanodes = cluster.getStorageContainerManager()
|
||||||
.getScmContainerManager().getContainerWithPipeline(containerID)
|
.getScmContainerManager().getContainerWithPipeline(containerID)
|
||||||
.getPipeline().getMachines();
|
.getPipeline().getMachines();
|
||||||
|
@ -207,16 +206,16 @@ public class TestCloseContainerByPipeline {
|
||||||
key.close();
|
key.close();
|
||||||
|
|
||||||
//get the name of a valid container
|
//get the name of a valid container
|
||||||
KsmKeyArgs keyArgs = new KsmKeyArgs.Builder().setVolumeName("test").
|
OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName("test").
|
||||||
setBucketName("test").setType(HddsProtos.ReplicationType.RATIS)
|
setBucketName("test").setType(HddsProtos.ReplicationType.RATIS)
|
||||||
.setFactor(HddsProtos.ReplicationFactor.THREE).setDataSize(1024)
|
.setFactor(HddsProtos.ReplicationFactor.THREE).setDataSize(1024)
|
||||||
.setKeyName("ratis").build();
|
.setKeyName("ratis").build();
|
||||||
|
|
||||||
KsmKeyLocationInfo ksmKeyLocationInfo =
|
OmKeyLocationInfo omKeyLocationInfo =
|
||||||
cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions()
|
cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
|
||||||
.get(0).getBlocksLatestVersionOnly().get(0);
|
.get(0).getBlocksLatestVersionOnly().get(0);
|
||||||
|
|
||||||
long containerID = ksmKeyLocationInfo.getContainerID();
|
long containerID = omKeyLocationInfo.getContainerID();
|
||||||
List<DatanodeDetails> datanodes = cluster.getStorageContainerManager()
|
List<DatanodeDetails> datanodes = cluster.getStorageContainerManager()
|
||||||
.getScmContainerManager().getContainerWithPipeline(containerID)
|
.getScmContainerManager().getContainerWithPipeline(containerID)
|
||||||
.getPipeline().getMachines();
|
.getPipeline().getMachines();
|
||||||
|
|
|
@ -28,8 +28,8 @@ import org.apache.hadoop.hdds.client.ReplicationType;
|
||||||
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
||||||
import org.apache.hadoop.ozone.client.rest.OzoneException;
|
import org.apache.hadoop.ozone.client.rest.OzoneException;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
||||||
import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
|
import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB;
|
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB;
|
||||||
|
@ -69,17 +69,17 @@ public class TestCloseContainerHandler {
|
||||||
key.close();
|
key.close();
|
||||||
|
|
||||||
//get the name of a valid container
|
//get the name of a valid container
|
||||||
KsmKeyArgs keyArgs =
|
OmKeyArgs keyArgs =
|
||||||
new KsmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
|
new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
|
||||||
.setType(HddsProtos.ReplicationType.STAND_ALONE)
|
.setType(HddsProtos.ReplicationType.STAND_ALONE)
|
||||||
.setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
|
.setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
|
||||||
.setKeyName("test").build();
|
.setKeyName("test").build();
|
||||||
|
|
||||||
KsmKeyLocationInfo ksmKeyLocationInfo =
|
OmKeyLocationInfo omKeyLocationInfo =
|
||||||
cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions()
|
cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
|
||||||
.get(0).getBlocksLatestVersionOnly().get(0);
|
.get(0).getBlocksLatestVersionOnly().get(0);
|
||||||
|
|
||||||
long containerID = ksmKeyLocationInfo.getContainerID();
|
long containerID = omKeyLocationInfo.getContainerID();
|
||||||
|
|
||||||
Assert.assertFalse(isContainerClosed(cluster, containerID));
|
Assert.assertFalse(isContainerClosed(cluster, containerID));
|
||||||
|
|
||||||
|
|
|
@ -14,7 +14,7 @@
|
||||||
* License for the specific language governing permissions and limitations under
|
* License for the specific language governing permissions and limitations under
|
||||||
* the License.
|
* the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm;
|
package org.apache.hadoop.ozone.om;
|
||||||
|
|
||||||
import org.apache.commons.lang3.RandomStringUtils;
|
import org.apache.commons.lang3.RandomStringUtils;
|
||||||
|
|
||||||
|
@ -30,8 +30,8 @@ import org.apache.hadoop.ozone.client.*;
|
||||||
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
||||||
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
|
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
||||||
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
|
||||||
import org.junit.AfterClass;
|
import org.junit.AfterClass;
|
||||||
|
@ -104,7 +104,7 @@ public class TestContainerReportWithKeys {
|
||||||
key.write(dataString.getBytes());
|
key.write(dataString.getBytes());
|
||||||
key.close();
|
key.close();
|
||||||
|
|
||||||
KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
|
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
|
||||||
.setVolumeName(volumeName)
|
.setVolumeName(volumeName)
|
||||||
.setBucketName(bucketName)
|
.setBucketName(bucketName)
|
||||||
.setKeyName(keyName)
|
.setKeyName(keyName)
|
||||||
|
@ -113,8 +113,8 @@ public class TestContainerReportWithKeys {
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
|
|
||||||
KsmKeyLocationInfo keyInfo =
|
OmKeyLocationInfo keyInfo =
|
||||||
cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions()
|
cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
|
||||||
.get(0).getBlocksLatestVersionOnly().get(0);
|
.get(0).getBlocksLatestVersionOnly().get(0);
|
||||||
|
|
||||||
ContainerData cd = getContainerData(keyInfo.getContainerID());
|
ContainerData cd = getContainerData(keyInfo.getContainerID());
|
|
@ -14,7 +14,7 @@
|
||||||
* License for the specific language governing permissions and limitations under
|
* License for the specific language governing permissions and limitations under
|
||||||
* the License.
|
* the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm;
|
package org.apache.hadoop.ozone.om;
|
||||||
|
|
||||||
import org.apache.commons.lang3.RandomStringUtils;
|
import org.apache.commons.lang3.RandomStringUtils;
|
||||||
import org.apache.hadoop.fs.StorageType;
|
import org.apache.hadoop.fs.StorageType;
|
|
@ -14,7 +14,7 @@
|
||||||
* License for the specific language governing permissions and limitations under
|
* License for the specific language governing permissions and limitations under
|
||||||
* the License.
|
* the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm;
|
package org.apache.hadoop.ozone.om;
|
||||||
|
|
||||||
import org.apache.commons.lang3.RandomStringUtils;
|
import org.apache.commons.lang3.RandomStringUtils;
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
|
@ -24,11 +24,11 @@ import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
|
||||||
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
||||||
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||||
import org.apache.hadoop.ozone.OzoneConsts;
|
import org.apache.hadoop.ozone.OzoneConsts;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfoGroup;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession;
|
import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
|
||||||
import org.apache.hadoop.ozone.web.handlers.BucketArgs;
|
import org.apache.hadoop.ozone.web.handlers.BucketArgs;
|
||||||
import org.apache.hadoop.ozone.web.handlers.KeyArgs;
|
import org.apache.hadoop.ozone.web.handlers.KeyArgs;
|
||||||
import org.apache.hadoop.ozone.web.handlers.UserArgs;
|
import org.apache.hadoop.ozone.web.handlers.UserArgs;
|
||||||
|
@ -51,13 +51,13 @@ import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class tests the versioning of blocks from KSM side.
|
* This class tests the versioning of blocks from OM side.
|
||||||
*/
|
*/
|
||||||
public class TestKsmBlockVersioning {
|
public class TestOmBlockVersioning {
|
||||||
private static MiniOzoneCluster cluster = null;
|
private static MiniOzoneCluster cluster = null;
|
||||||
private static UserArgs userArgs;
|
private static UserArgs userArgs;
|
||||||
private static OzoneConfiguration conf;
|
private static OzoneConfiguration conf;
|
||||||
private static KeySpaceManager keySpaceManager;
|
private static OzoneManager ozoneManager;
|
||||||
private static StorageHandler storageHandler;
|
private static StorageHandler storageHandler;
|
||||||
|
|
||||||
@Rule
|
@Rule
|
||||||
|
@ -81,7 +81,7 @@ public class TestKsmBlockVersioning {
|
||||||
storageHandler = new ObjectStoreHandler(conf).getStorageHandler();
|
storageHandler = new ObjectStoreHandler(conf).getStorageHandler();
|
||||||
userArgs = new UserArgs(null, OzoneUtils.getRequestID(),
|
userArgs = new UserArgs(null, OzoneUtils.getRequestID(),
|
||||||
null, null, null, null);
|
null, null, null, null);
|
||||||
keySpaceManager = cluster.getKeySpaceManager();
|
ozoneManager = cluster.getOzoneManager();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -113,7 +113,7 @@ public class TestKsmBlockVersioning {
|
||||||
bucketArgs.setStorageType(StorageType.DISK);
|
bucketArgs.setStorageType(StorageType.DISK);
|
||||||
storageHandler.createBucket(bucketArgs);
|
storageHandler.createBucket(bucketArgs);
|
||||||
|
|
||||||
KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
|
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
|
||||||
.setVolumeName(volumeName)
|
.setVolumeName(volumeName)
|
||||||
.setBucketName(bucketName)
|
.setBucketName(bucketName)
|
||||||
.setKeyName(keyName)
|
.setKeyName(keyName)
|
||||||
|
@ -121,49 +121,49 @@ public class TestKsmBlockVersioning {
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
// 1st update, version 0
|
// 1st update, version 0
|
||||||
OpenKeySession openKey = keySpaceManager.openKey(keyArgs);
|
OpenKeySession openKey = ozoneManager.openKey(keyArgs);
|
||||||
keySpaceManager.commitKey(keyArgs, openKey.getId());
|
ozoneManager.commitKey(keyArgs, openKey.getId());
|
||||||
|
|
||||||
KsmKeyInfo keyInfo = keySpaceManager.lookupKey(keyArgs);
|
OmKeyInfo keyInfo = ozoneManager.lookupKey(keyArgs);
|
||||||
KsmKeyLocationInfoGroup highestVersion =
|
OmKeyLocationInfoGroup highestVersion =
|
||||||
checkVersions(keyInfo.getKeyLocationVersions());
|
checkVersions(keyInfo.getKeyLocationVersions());
|
||||||
assertEquals(0, highestVersion.getVersion());
|
assertEquals(0, highestVersion.getVersion());
|
||||||
assertEquals(1, highestVersion.getLocationList().size());
|
assertEquals(1, highestVersion.getLocationList().size());
|
||||||
|
|
||||||
// 2nd update, version 1
|
// 2nd update, version 1
|
||||||
openKey = keySpaceManager.openKey(keyArgs);
|
openKey = ozoneManager.openKey(keyArgs);
|
||||||
//KsmKeyLocationInfo locationInfo =
|
//OmKeyLocationInfo locationInfo =
|
||||||
// keySpaceManager.allocateBlock(keyArgs, openKey.getId());
|
// ozoneManager.allocateBlock(keyArgs, openKey.getId());
|
||||||
keySpaceManager.commitKey(keyArgs, openKey.getId());
|
ozoneManager.commitKey(keyArgs, openKey.getId());
|
||||||
|
|
||||||
keyInfo = keySpaceManager.lookupKey(keyArgs);
|
keyInfo = ozoneManager.lookupKey(keyArgs);
|
||||||
highestVersion = checkVersions(keyInfo.getKeyLocationVersions());
|
highestVersion = checkVersions(keyInfo.getKeyLocationVersions());
|
||||||
assertEquals(1, highestVersion.getVersion());
|
assertEquals(1, highestVersion.getVersion());
|
||||||
assertEquals(2, highestVersion.getLocationList().size());
|
assertEquals(2, highestVersion.getLocationList().size());
|
||||||
|
|
||||||
// 3rd update, version 2
|
// 3rd update, version 2
|
||||||
openKey = keySpaceManager.openKey(keyArgs);
|
openKey = ozoneManager.openKey(keyArgs);
|
||||||
// this block will be appended to the latest version of version 2.
|
// this block will be appended to the latest version of version 2.
|
||||||
keySpaceManager.allocateBlock(keyArgs, openKey.getId());
|
ozoneManager.allocateBlock(keyArgs, openKey.getId());
|
||||||
keySpaceManager.commitKey(keyArgs, openKey.getId());
|
ozoneManager.commitKey(keyArgs, openKey.getId());
|
||||||
|
|
||||||
keyInfo = keySpaceManager.lookupKey(keyArgs);
|
keyInfo = ozoneManager.lookupKey(keyArgs);
|
||||||
highestVersion = checkVersions(keyInfo.getKeyLocationVersions());
|
highestVersion = checkVersions(keyInfo.getKeyLocationVersions());
|
||||||
assertEquals(2, highestVersion.getVersion());
|
assertEquals(2, highestVersion.getVersion());
|
||||||
assertEquals(4, highestVersion.getLocationList().size());
|
assertEquals(4, highestVersion.getLocationList().size());
|
||||||
}
|
}
|
||||||
|
|
||||||
private KsmKeyLocationInfoGroup checkVersions(
|
private OmKeyLocationInfoGroup checkVersions(
|
||||||
List<KsmKeyLocationInfoGroup> versions) {
|
List<OmKeyLocationInfoGroup> versions) {
|
||||||
KsmKeyLocationInfoGroup currentVersion = null;
|
OmKeyLocationInfoGroup currentVersion = null;
|
||||||
for (KsmKeyLocationInfoGroup version : versions) {
|
for (OmKeyLocationInfoGroup version : versions) {
|
||||||
if (currentVersion != null) {
|
if (currentVersion != null) {
|
||||||
assertEquals(currentVersion.getVersion() + 1, version.getVersion());
|
assertEquals(currentVersion.getVersion() + 1, version.getVersion());
|
||||||
for (KsmKeyLocationInfo info : currentVersion.getLocationList()) {
|
for (OmKeyLocationInfo info : currentVersion.getLocationList()) {
|
||||||
boolean found = false;
|
boolean found = false;
|
||||||
// all the blocks from the previous version must present in the next
|
// all the blocks from the previous version must present in the next
|
||||||
// version
|
// version
|
||||||
for (KsmKeyLocationInfo info2 : version.getLocationList()) {
|
for (OmKeyLocationInfo info2 : version.getLocationList()) {
|
||||||
if (info.getLocalID() == info2.getLocalID()) {
|
if (info.getLocalID() == info2.getLocalID()) {
|
||||||
found = true;
|
found = true;
|
||||||
break;
|
break;
|
||||||
|
@ -197,7 +197,7 @@ public class TestKsmBlockVersioning {
|
||||||
bucketArgs.setStorageType(StorageType.DISK);
|
bucketArgs.setStorageType(StorageType.DISK);
|
||||||
storageHandler.createBucket(bucketArgs);
|
storageHandler.createBucket(bucketArgs);
|
||||||
|
|
||||||
KsmKeyArgs ksmKeyArgs = new KsmKeyArgs.Builder()
|
OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
|
||||||
.setVolumeName(volumeName)
|
.setVolumeName(volumeName)
|
||||||
.setBucketName(bucketName)
|
.setBucketName(bucketName)
|
||||||
.setKeyName(keyName)
|
.setKeyName(keyName)
|
||||||
|
@ -214,7 +214,7 @@ public class TestKsmBlockVersioning {
|
||||||
try (InputStream in = storageHandler.newKeyReader(keyArgs)) {
|
try (InputStream in = storageHandler.newKeyReader(keyArgs)) {
|
||||||
in.read(data);
|
in.read(data);
|
||||||
}
|
}
|
||||||
KsmKeyInfo keyInfo = keySpaceManager.lookupKey(ksmKeyArgs);
|
OmKeyInfo keyInfo = ozoneManager.lookupKey(omKeyArgs);
|
||||||
assertEquals(dataString, DFSUtil.bytes2String(data));
|
assertEquals(dataString, DFSUtil.bytes2String(data));
|
||||||
assertEquals(0, keyInfo.getLatestVersionLocations().getVersion());
|
assertEquals(0, keyInfo.getLatestVersionLocations().getVersion());
|
||||||
assertEquals(1,
|
assertEquals(1,
|
||||||
|
@ -230,7 +230,7 @@ public class TestKsmBlockVersioning {
|
||||||
try (InputStream in = storageHandler.newKeyReader(keyArgs)) {
|
try (InputStream in = storageHandler.newKeyReader(keyArgs)) {
|
||||||
in.read(data);
|
in.read(data);
|
||||||
}
|
}
|
||||||
keyInfo = keySpaceManager.lookupKey(ksmKeyArgs);
|
keyInfo = ozoneManager.lookupKey(omKeyArgs);
|
||||||
assertEquals(dataString, DFSUtil.bytes2String(data));
|
assertEquals(dataString, DFSUtil.bytes2String(data));
|
||||||
assertEquals(1, keyInfo.getLatestVersionLocations().getVersion());
|
assertEquals(1, keyInfo.getLatestVersionLocations().getVersion());
|
||||||
assertEquals(2,
|
assertEquals(2,
|
||||||
|
@ -244,7 +244,7 @@ public class TestKsmBlockVersioning {
|
||||||
try (InputStream in = storageHandler.newKeyReader(keyArgs)) {
|
try (InputStream in = storageHandler.newKeyReader(keyArgs)) {
|
||||||
in.read(data);
|
in.read(data);
|
||||||
}
|
}
|
||||||
keyInfo = keySpaceManager.lookupKey(ksmKeyArgs);
|
keyInfo = ozoneManager.lookupKey(omKeyArgs);
|
||||||
assertEquals(dataString, DFSUtil.bytes2String(data));
|
assertEquals(dataString, DFSUtil.bytes2String(data));
|
||||||
assertEquals(2, keyInfo.getLatestVersionLocations().getVersion());
|
assertEquals(2, keyInfo.getLatestVersionLocations().getVersion());
|
||||||
assertEquals(3,
|
assertEquals(3,
|
|
@ -14,7 +14,7 @@
|
||||||
* License for the specific language governing permissions and limitations under
|
* License for the specific language governing permissions and limitations under
|
||||||
* the License.
|
* the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm;
|
package org.apache.hadoop.ozone.om;
|
||||||
|
|
||||||
import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
|
import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
|
||||||
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
|
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
|
||||||
|
@ -26,18 +26,18 @@ import org.apache.hadoop.ozone.MiniOzoneCluster;
|
||||||
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.ozone.OzoneConsts;
|
import org.apache.hadoop.ozone.OzoneConsts;
|
||||||
import org.apache.hadoop.test.Whitebox;
|
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.mockito.Mockito;
|
import org.mockito.Mockito;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test for KSM metrics.
|
* Test for OM metrics.
|
||||||
*/
|
*/
|
||||||
public class TestKSMMetrcis {
|
@SuppressWarnings("deprecation")
|
||||||
|
public class TestOmMetrics {
|
||||||
private MiniOzoneCluster cluster;
|
private MiniOzoneCluster cluster;
|
||||||
private KeySpaceManager ksmManager;
|
private OzoneManager ozoneManager;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The exception used for testing failure metrics.
|
* The exception used for testing failure metrics.
|
||||||
|
@ -56,7 +56,7 @@ public class TestKSMMetrcis {
|
||||||
OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
|
OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
|
||||||
cluster = MiniOzoneCluster.newBuilder(conf).build();
|
cluster = MiniOzoneCluster.newBuilder(conf).build();
|
||||||
cluster.waitForClusterToBeReady();
|
cluster.waitForClusterToBeReady();
|
||||||
ksmManager = cluster.getKeySpaceManager();
|
ozoneManager = cluster.getOzoneManager();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -71,8 +71,9 @@ public class TestKSMMetrcis {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testVolumeOps() throws IOException {
|
public void testVolumeOps() throws IOException {
|
||||||
VolumeManager volumeManager = (VolumeManager) Whitebox
|
VolumeManager volumeManager =
|
||||||
.getInternalState(ksmManager, "volumeManager");
|
(VolumeManager) org.apache.hadoop.test.Whitebox
|
||||||
|
.getInternalState(ozoneManager, "volumeManager");
|
||||||
VolumeManager mockVm = Mockito.spy(volumeManager);
|
VolumeManager mockVm = Mockito.spy(volumeManager);
|
||||||
|
|
||||||
Mockito.doNothing().when(mockVm).createVolume(null);
|
Mockito.doNothing().when(mockVm).createVolume(null);
|
||||||
|
@ -82,17 +83,18 @@ public class TestKSMMetrcis {
|
||||||
Mockito.doNothing().when(mockVm).setOwner(null, null);
|
Mockito.doNothing().when(mockVm).setOwner(null, null);
|
||||||
Mockito.doReturn(null).when(mockVm).listVolumes(null, null, null, 0);
|
Mockito.doReturn(null).when(mockVm).listVolumes(null, null, null, 0);
|
||||||
|
|
||||||
Whitebox.setInternalState(ksmManager, "volumeManager", mockVm);
|
org.apache.hadoop.test.Whitebox.setInternalState(
|
||||||
|
ozoneManager, "volumeManager", mockVm);
|
||||||
doVolumeOps();
|
doVolumeOps();
|
||||||
|
|
||||||
MetricsRecordBuilder ksmMetrics = getMetrics("KSMMetrics");
|
MetricsRecordBuilder omMetrics = getMetrics("OMMetrics");
|
||||||
assertCounter("NumVolumeOps", 6L, ksmMetrics);
|
assertCounter("NumVolumeOps", 6L, omMetrics);
|
||||||
assertCounter("NumVolumeCreates", 1L, ksmMetrics);
|
assertCounter("NumVolumeCreates", 1L, omMetrics);
|
||||||
assertCounter("NumVolumeUpdates", 1L, ksmMetrics);
|
assertCounter("NumVolumeUpdates", 1L, omMetrics);
|
||||||
assertCounter("NumVolumeInfos", 1L, ksmMetrics);
|
assertCounter("NumVolumeInfos", 1L, omMetrics);
|
||||||
assertCounter("NumVolumeCheckAccesses", 1L, ksmMetrics);
|
assertCounter("NumVolumeCheckAccesses", 1L, omMetrics);
|
||||||
assertCounter("NumVolumeDeletes", 1L, ksmMetrics);
|
assertCounter("NumVolumeDeletes", 1L, omMetrics);
|
||||||
assertCounter("NumVolumeLists", 1L, ksmMetrics);
|
assertCounter("NumVolumeLists", 1L, omMetrics);
|
||||||
|
|
||||||
// inject exception to test for Failure Metrics
|
// inject exception to test for Failure Metrics
|
||||||
Mockito.doThrow(exception).when(mockVm).createVolume(null);
|
Mockito.doThrow(exception).when(mockVm).createVolume(null);
|
||||||
|
@ -102,30 +104,31 @@ public class TestKSMMetrcis {
|
||||||
Mockito.doThrow(exception).when(mockVm).setOwner(null, null);
|
Mockito.doThrow(exception).when(mockVm).setOwner(null, null);
|
||||||
Mockito.doThrow(exception).when(mockVm).listVolumes(null, null, null, 0);
|
Mockito.doThrow(exception).when(mockVm).listVolumes(null, null, null, 0);
|
||||||
|
|
||||||
Whitebox.setInternalState(ksmManager, "volumeManager", mockVm);
|
org.apache.hadoop.test.Whitebox.setInternalState(ozoneManager, "volumeManager", mockVm);
|
||||||
doVolumeOps();
|
doVolumeOps();
|
||||||
|
|
||||||
ksmMetrics = getMetrics("KSMMetrics");
|
omMetrics = getMetrics("OMMetrics");
|
||||||
assertCounter("NumVolumeOps", 12L, ksmMetrics);
|
assertCounter("NumVolumeOps", 12L, omMetrics);
|
||||||
assertCounter("NumVolumeCreates", 2L, ksmMetrics);
|
assertCounter("NumVolumeCreates", 2L, omMetrics);
|
||||||
assertCounter("NumVolumeUpdates", 2L, ksmMetrics);
|
assertCounter("NumVolumeUpdates", 2L, omMetrics);
|
||||||
assertCounter("NumVolumeInfos", 2L, ksmMetrics);
|
assertCounter("NumVolumeInfos", 2L, omMetrics);
|
||||||
assertCounter("NumVolumeCheckAccesses", 2L, ksmMetrics);
|
assertCounter("NumVolumeCheckAccesses", 2L, omMetrics);
|
||||||
assertCounter("NumVolumeDeletes", 2L, ksmMetrics);
|
assertCounter("NumVolumeDeletes", 2L, omMetrics);
|
||||||
assertCounter("NumVolumeLists", 2L, ksmMetrics);
|
assertCounter("NumVolumeLists", 2L, omMetrics);
|
||||||
|
|
||||||
assertCounter("NumVolumeCreateFails", 1L, ksmMetrics);
|
assertCounter("NumVolumeCreateFails", 1L, omMetrics);
|
||||||
assertCounter("NumVolumeUpdateFails", 1L, ksmMetrics);
|
assertCounter("NumVolumeUpdateFails", 1L, omMetrics);
|
||||||
assertCounter("NumVolumeInfoFails", 1L, ksmMetrics);
|
assertCounter("NumVolumeInfoFails", 1L, omMetrics);
|
||||||
assertCounter("NumVolumeCheckAccessFails", 1L, ksmMetrics);
|
assertCounter("NumVolumeCheckAccessFails", 1L, omMetrics);
|
||||||
assertCounter("NumVolumeDeleteFails", 1L, ksmMetrics);
|
assertCounter("NumVolumeDeleteFails", 1L, omMetrics);
|
||||||
assertCounter("NumVolumeListFails", 1L, ksmMetrics);
|
assertCounter("NumVolumeListFails", 1L, omMetrics);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testBucketOps() throws IOException {
|
public void testBucketOps() throws IOException {
|
||||||
BucketManager bucketManager = (BucketManager) Whitebox
|
BucketManager bucketManager =
|
||||||
.getInternalState(ksmManager, "bucketManager");
|
(BucketManager) org.apache.hadoop.test.Whitebox
|
||||||
|
.getInternalState(ozoneManager, "bucketManager");
|
||||||
BucketManager mockBm = Mockito.spy(bucketManager);
|
BucketManager mockBm = Mockito.spy(bucketManager);
|
||||||
|
|
||||||
Mockito.doNothing().when(mockBm).createBucket(null);
|
Mockito.doNothing().when(mockBm).createBucket(null);
|
||||||
|
@ -134,16 +137,17 @@ public class TestKSMMetrcis {
|
||||||
Mockito.doNothing().when(mockBm).setBucketProperty(null);
|
Mockito.doNothing().when(mockBm).setBucketProperty(null);
|
||||||
Mockito.doReturn(null).when(mockBm).listBuckets(null, null, null, 0);
|
Mockito.doReturn(null).when(mockBm).listBuckets(null, null, null, 0);
|
||||||
|
|
||||||
Whitebox.setInternalState(ksmManager, "bucketManager", mockBm);
|
org.apache.hadoop.test.Whitebox.setInternalState(
|
||||||
|
ozoneManager, "bucketManager", mockBm);
|
||||||
doBucketOps();
|
doBucketOps();
|
||||||
|
|
||||||
MetricsRecordBuilder ksmMetrics = getMetrics("KSMMetrics");
|
MetricsRecordBuilder omMetrics = getMetrics("OMMetrics");
|
||||||
assertCounter("NumBucketOps", 5L, ksmMetrics);
|
assertCounter("NumBucketOps", 5L, omMetrics);
|
||||||
assertCounter("NumBucketCreates", 1L, ksmMetrics);
|
assertCounter("NumBucketCreates", 1L, omMetrics);
|
||||||
assertCounter("NumBucketUpdates", 1L, ksmMetrics);
|
assertCounter("NumBucketUpdates", 1L, omMetrics);
|
||||||
assertCounter("NumBucketInfos", 1L, ksmMetrics);
|
assertCounter("NumBucketInfos", 1L, omMetrics);
|
||||||
assertCounter("NumBucketDeletes", 1L, ksmMetrics);
|
assertCounter("NumBucketDeletes", 1L, omMetrics);
|
||||||
assertCounter("NumBucketLists", 1L, ksmMetrics);
|
assertCounter("NumBucketLists", 1L, omMetrics);
|
||||||
|
|
||||||
// inject exception to test for Failure Metrics
|
// inject exception to test for Failure Metrics
|
||||||
Mockito.doThrow(exception).when(mockBm).createBucket(null);
|
Mockito.doThrow(exception).when(mockBm).createBucket(null);
|
||||||
|
@ -152,28 +156,29 @@ public class TestKSMMetrcis {
|
||||||
Mockito.doThrow(exception).when(mockBm).setBucketProperty(null);
|
Mockito.doThrow(exception).when(mockBm).setBucketProperty(null);
|
||||||
Mockito.doThrow(exception).when(mockBm).listBuckets(null, null, null, 0);
|
Mockito.doThrow(exception).when(mockBm).listBuckets(null, null, null, 0);
|
||||||
|
|
||||||
Whitebox.setInternalState(ksmManager, "bucketManager", mockBm);
|
org.apache.hadoop.test.Whitebox.setInternalState(
|
||||||
|
ozoneManager, "bucketManager", mockBm);
|
||||||
doBucketOps();
|
doBucketOps();
|
||||||
|
|
||||||
ksmMetrics = getMetrics("KSMMetrics");
|
omMetrics = getMetrics("OMMetrics");
|
||||||
assertCounter("NumBucketOps", 10L, ksmMetrics);
|
assertCounter("NumBucketOps", 10L, omMetrics);
|
||||||
assertCounter("NumBucketCreates", 2L, ksmMetrics);
|
assertCounter("NumBucketCreates", 2L, omMetrics);
|
||||||
assertCounter("NumBucketUpdates", 2L, ksmMetrics);
|
assertCounter("NumBucketUpdates", 2L, omMetrics);
|
||||||
assertCounter("NumBucketInfos", 2L, ksmMetrics);
|
assertCounter("NumBucketInfos", 2L, omMetrics);
|
||||||
assertCounter("NumBucketDeletes", 2L, ksmMetrics);
|
assertCounter("NumBucketDeletes", 2L, omMetrics);
|
||||||
assertCounter("NumBucketLists", 2L, ksmMetrics);
|
assertCounter("NumBucketLists", 2L, omMetrics);
|
||||||
|
|
||||||
assertCounter("NumBucketCreateFails", 1L, ksmMetrics);
|
assertCounter("NumBucketCreateFails", 1L, omMetrics);
|
||||||
assertCounter("NumBucketUpdateFails", 1L, ksmMetrics);
|
assertCounter("NumBucketUpdateFails", 1L, omMetrics);
|
||||||
assertCounter("NumBucketInfoFails", 1L, ksmMetrics);
|
assertCounter("NumBucketInfoFails", 1L, omMetrics);
|
||||||
assertCounter("NumBucketDeleteFails", 1L, ksmMetrics);
|
assertCounter("NumBucketDeleteFails", 1L, omMetrics);
|
||||||
assertCounter("NumBucketListFails", 1L, ksmMetrics);
|
assertCounter("NumBucketListFails", 1L, omMetrics);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testKeyOps() throws IOException {
|
public void testKeyOps() throws IOException {
|
||||||
KeyManager bucketManager = (KeyManager) Whitebox
|
KeyManager bucketManager = (KeyManager) org.apache.hadoop.test.Whitebox
|
||||||
.getInternalState(ksmManager, "keyManager");
|
.getInternalState(ozoneManager, "keyManager");
|
||||||
KeyManager mockKm = Mockito.spy(bucketManager);
|
KeyManager mockKm = Mockito.spy(bucketManager);
|
||||||
|
|
||||||
Mockito.doReturn(null).when(mockKm).openKey(null);
|
Mockito.doReturn(null).when(mockKm).openKey(null);
|
||||||
|
@ -181,15 +186,16 @@ public class TestKSMMetrcis {
|
||||||
Mockito.doReturn(null).when(mockKm).lookupKey(null);
|
Mockito.doReturn(null).when(mockKm).lookupKey(null);
|
||||||
Mockito.doReturn(null).when(mockKm).listKeys(null, null, null, null, 0);
|
Mockito.doReturn(null).when(mockKm).listKeys(null, null, null, null, 0);
|
||||||
|
|
||||||
Whitebox.setInternalState(ksmManager, "keyManager", mockKm);
|
org.apache.hadoop.test.Whitebox.setInternalState(
|
||||||
|
ozoneManager, "keyManager", mockKm);
|
||||||
doKeyOps();
|
doKeyOps();
|
||||||
|
|
||||||
MetricsRecordBuilder ksmMetrics = getMetrics("KSMMetrics");
|
MetricsRecordBuilder omMetrics = getMetrics("OMMetrics");
|
||||||
assertCounter("NumKeyOps", 4L, ksmMetrics);
|
assertCounter("NumKeyOps", 4L, omMetrics);
|
||||||
assertCounter("NumKeyAllocate", 1L, ksmMetrics);
|
assertCounter("NumKeyAllocate", 1L, omMetrics);
|
||||||
assertCounter("NumKeyLookup", 1L, ksmMetrics);
|
assertCounter("NumKeyLookup", 1L, omMetrics);
|
||||||
assertCounter("NumKeyDeletes", 1L, ksmMetrics);
|
assertCounter("NumKeyDeletes", 1L, omMetrics);
|
||||||
assertCounter("NumKeyLists", 1L, ksmMetrics);
|
assertCounter("NumKeyLists", 1L, omMetrics);
|
||||||
|
|
||||||
// inject exception to test for Failure Metrics
|
// inject exception to test for Failure Metrics
|
||||||
Mockito.doThrow(exception).when(mockKm).openKey(null);
|
Mockito.doThrow(exception).when(mockKm).openKey(null);
|
||||||
|
@ -198,20 +204,21 @@ public class TestKSMMetrcis {
|
||||||
Mockito.doThrow(exception).when(mockKm).listKeys(
|
Mockito.doThrow(exception).when(mockKm).listKeys(
|
||||||
null, null, null, null, 0);
|
null, null, null, null, 0);
|
||||||
|
|
||||||
Whitebox.setInternalState(ksmManager, "keyManager", mockKm);
|
org.apache.hadoop.test.Whitebox.setInternalState(
|
||||||
|
ozoneManager, "keyManager", mockKm);
|
||||||
doKeyOps();
|
doKeyOps();
|
||||||
|
|
||||||
ksmMetrics = getMetrics("KSMMetrics");
|
omMetrics = getMetrics("OMMetrics");
|
||||||
assertCounter("NumKeyOps", 8L, ksmMetrics);
|
assertCounter("NumKeyOps", 8L, omMetrics);
|
||||||
assertCounter("NumKeyAllocate", 2L, ksmMetrics);
|
assertCounter("NumKeyAllocate", 2L, omMetrics);
|
||||||
assertCounter("NumKeyLookup", 2L, ksmMetrics);
|
assertCounter("NumKeyLookup", 2L, omMetrics);
|
||||||
assertCounter("NumKeyDeletes", 2L, ksmMetrics);
|
assertCounter("NumKeyDeletes", 2L, omMetrics);
|
||||||
assertCounter("NumKeyLists", 2L, ksmMetrics);
|
assertCounter("NumKeyLists", 2L, omMetrics);
|
||||||
|
|
||||||
assertCounter("NumKeyAllocateFails", 1L, ksmMetrics);
|
assertCounter("NumKeyAllocateFails", 1L, omMetrics);
|
||||||
assertCounter("NumKeyLookupFails", 1L, ksmMetrics);
|
assertCounter("NumKeyLookupFails", 1L, omMetrics);
|
||||||
assertCounter("NumKeyDeleteFails", 1L, ksmMetrics);
|
assertCounter("NumKeyDeleteFails", 1L, omMetrics);
|
||||||
assertCounter("NumKeyListFails", 1L, ksmMetrics);
|
assertCounter("NumKeyListFails", 1L, omMetrics);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -219,32 +226,32 @@ public class TestKSMMetrcis {
|
||||||
*/
|
*/
|
||||||
private void doVolumeOps() {
|
private void doVolumeOps() {
|
||||||
try {
|
try {
|
||||||
ksmManager.createVolume(null);
|
ozoneManager.createVolume(null);
|
||||||
} catch (IOException ignored) {
|
} catch (IOException ignored) {
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
ksmManager.deleteVolume(null);
|
ozoneManager.deleteVolume(null);
|
||||||
} catch (IOException ignored) {
|
} catch (IOException ignored) {
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
ksmManager.getVolumeInfo(null);
|
ozoneManager.getVolumeInfo(null);
|
||||||
} catch (IOException ignored) {
|
} catch (IOException ignored) {
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
ksmManager.checkVolumeAccess(null, null);
|
ozoneManager.checkVolumeAccess(null, null);
|
||||||
} catch (IOException ignored) {
|
} catch (IOException ignored) {
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
ksmManager.setOwner(null, null);
|
ozoneManager.setOwner(null, null);
|
||||||
} catch (IOException ignored) {
|
} catch (IOException ignored) {
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
ksmManager.listAllVolumes(null, null, 0);
|
ozoneManager.listAllVolumes(null, null, 0);
|
||||||
} catch (IOException ignored) {
|
} catch (IOException ignored) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -254,27 +261,27 @@ public class TestKSMMetrcis {
|
||||||
*/
|
*/
|
||||||
private void doBucketOps() {
|
private void doBucketOps() {
|
||||||
try {
|
try {
|
||||||
ksmManager.createBucket(null);
|
ozoneManager.createBucket(null);
|
||||||
} catch (IOException ignored) {
|
} catch (IOException ignored) {
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
ksmManager.deleteBucket(null, null);
|
ozoneManager.deleteBucket(null, null);
|
||||||
} catch (IOException ignored) {
|
} catch (IOException ignored) {
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
ksmManager.getBucketInfo(null, null);
|
ozoneManager.getBucketInfo(null, null);
|
||||||
} catch (IOException ignored) {
|
} catch (IOException ignored) {
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
ksmManager.setBucketProperty(null);
|
ozoneManager.setBucketProperty(null);
|
||||||
} catch (IOException ignored) {
|
} catch (IOException ignored) {
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
ksmManager.listBuckets(null, null, null, 0);
|
ozoneManager.listBuckets(null, null, null, 0);
|
||||||
} catch (IOException ignored) {
|
} catch (IOException ignored) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -284,22 +291,22 @@ public class TestKSMMetrcis {
|
||||||
*/
|
*/
|
||||||
private void doKeyOps() {
|
private void doKeyOps() {
|
||||||
try {
|
try {
|
||||||
ksmManager.openKey(null);
|
ozoneManager.openKey(null);
|
||||||
} catch (IOException ignored) {
|
} catch (IOException ignored) {
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
ksmManager.deleteKey(null);
|
ozoneManager.deleteKey(null);
|
||||||
} catch (IOException ignored) {
|
} catch (IOException ignored) {
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
ksmManager.lookupKey(null);
|
ozoneManager.lookupKey(null);
|
||||||
} catch (IOException ignored) {
|
} catch (IOException ignored) {
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
ksmManager.listKeys(null, null, null, null, 0);
|
ozoneManager.listKeys(null, null, null, null, 0);
|
||||||
} catch (IOException ignored) {
|
} catch (IOException ignored) {
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -14,7 +14,7 @@
|
||||||
* License for the specific language governing permissions and limitations under
|
* License for the specific language governing permissions and limitations under
|
||||||
* the License.
|
* the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm;
|
package org.apache.hadoop.ozone.om;
|
||||||
|
|
||||||
import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
|
import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
|
||||||
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
||||||
|
@ -51,17 +51,17 @@ import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
|
|
||||||
import static org.apache.hadoop.ozone.OzoneConsts.KSM_DB_NAME;
|
import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertFalse;
|
import static org.junit.Assert.assertFalse;
|
||||||
import static org.junit.Assert.assertNotNull;
|
import static org.junit.Assert.assertNotNull;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class tests the CLI that transforms ksm.db into SQLite DB files.
|
* This class tests the CLI that transforms om.db into SQLite DB files.
|
||||||
*/
|
*/
|
||||||
@RunWith(Parameterized.class)
|
@RunWith(Parameterized.class)
|
||||||
public class TestKSMSQLCli {
|
public class TestOmSQLCli {
|
||||||
private MiniOzoneCluster cluster = null;
|
private MiniOzoneCluster cluster = null;
|
||||||
private StorageHandler storageHandler;
|
private StorageHandler storageHandler;
|
||||||
private UserArgs userArgs;
|
private UserArgs userArgs;
|
||||||
|
@ -90,7 +90,7 @@ public class TestKSMSQLCli {
|
||||||
|
|
||||||
private String metaStoreType;
|
private String metaStoreType;
|
||||||
|
|
||||||
public TestKSMSQLCli(String type) {
|
public TestOmSQLCli(String type) {
|
||||||
metaStoreType = type;
|
metaStoreType = type;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -152,7 +152,7 @@ public class TestKSMSQLCli {
|
||||||
stream = storageHandler.newKeyWriter(keyArgs3);
|
stream = storageHandler.newKeyWriter(keyArgs3);
|
||||||
stream.close();
|
stream.close();
|
||||||
|
|
||||||
cluster.getKeySpaceManager().stop();
|
cluster.getOzoneManager().stop();
|
||||||
cluster.getStorageContainerManager().stop();
|
cluster.getStorageContainerManager().stop();
|
||||||
conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, metaStoreType);
|
conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, metaStoreType);
|
||||||
cli = new SQLCLI(conf);
|
cli = new SQLCLI(conf);
|
||||||
|
@ -166,12 +166,12 @@ public class TestKSMSQLCli {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testKSMDB() throws Exception {
|
public void testOmDB() throws Exception {
|
||||||
String dbOutPath = GenericTestUtils.getTempPath(
|
String dbOutPath = GenericTestUtils.getTempPath(
|
||||||
UUID.randomUUID() + "/out_sql.db");
|
UUID.randomUUID() + "/out_sql.db");
|
||||||
|
|
||||||
String dbRootPath = conf.get(OzoneConfigKeys.OZONE_METADATA_DIRS);
|
String dbRootPath = conf.get(OzoneConfigKeys.OZONE_METADATA_DIRS);
|
||||||
String dbPath = dbRootPath + "/" + KSM_DB_NAME;
|
String dbPath = dbRootPath + "/" + OM_DB_NAME;
|
||||||
String[] args = {"-p", dbPath, "-o", dbOutPath};
|
String[] args = {"-p", dbPath, "-o", dbOutPath};
|
||||||
|
|
||||||
cli.run(args);
|
cli.run(args);
|
|
@ -14,7 +14,7 @@
|
||||||
* License for the specific language governing permissions and limitations under
|
* License for the specific language governing permissions and limitations under
|
||||||
* the License.
|
* the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm;
|
package org.apache.hadoop.ozone.om;
|
||||||
|
|
||||||
|
|
||||||
import org.apache.commons.lang3.RandomStringUtils;
|
import org.apache.commons.lang3.RandomStringUtils;
|
||||||
|
@ -29,11 +29,11 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.ozone.OzoneConsts;
|
import org.apache.hadoop.ozone.OzoneConsts;
|
||||||
import org.apache.hadoop.ozone.common.BlockGroup;
|
import org.apache.hadoop.ozone.common.BlockGroup;
|
||||||
import org.apache.hadoop.ozone.client.rest.OzoneException;
|
import org.apache.hadoop.ozone.client.rest.OzoneException;
|
||||||
import org.apache.hadoop.ozone.ksm.exceptions.KSMException;
|
import org.apache.hadoop.ozone.om.exceptions.OMException;
|
||||||
import org.apache.hadoop.hdds.scm.server.SCMStorage;
|
import org.apache.hadoop.hdds.scm.server.SCMStorage;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
|
import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.ServicePort;
|
.OzoneManagerProtocolProtos.ServicePort;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.ozone.web.handlers.BucketArgs;
|
import org.apache.hadoop.ozone.web.handlers.BucketArgs;
|
||||||
import org.apache.hadoop.ozone.web.handlers.KeyArgs;
|
import org.apache.hadoop.ozone.web.handlers.KeyArgs;
|
||||||
|
@ -50,7 +50,7 @@ import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||||
import org.apache.hadoop.hdds.scm.ScmInfo;
|
import org.apache.hadoop.hdds.scm.ScmInfo;
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.Status;
|
.OzoneManagerProtocolProtos.Status;
|
||||||
import org.apache.hadoop.ozone.web.handlers.ListArgs;
|
import org.apache.hadoop.ozone.web.handlers.ListArgs;
|
||||||
import org.apache.hadoop.ozone.web.response.ListBuckets;
|
import org.apache.hadoop.ozone.web.response.ListBuckets;
|
||||||
import org.apache.hadoop.ozone.web.response.ListKeys;
|
import org.apache.hadoop.ozone.web.response.ListKeys;
|
||||||
|
@ -74,7 +74,6 @@ import java.nio.file.Path;
|
||||||
import java.nio.file.Paths;
|
import java.nio.file.Paths;
|
||||||
import java.net.InetSocketAddress;
|
import java.net.InetSocketAddress;
|
||||||
import java.text.ParseException;
|
import java.text.ParseException;
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
@ -86,22 +85,22 @@ import java.util.stream.Stream;
|
||||||
|
|
||||||
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS;
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS;
|
||||||
import static org.apache.hadoop.ozone.OzoneConsts.DELETING_KEY_PREFIX;
|
import static org.apache.hadoop.ozone.OzoneConsts.DELETING_KEY_PREFIX;
|
||||||
import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_ADDRESS_KEY;
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
|
||||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
||||||
.OZONE_SCM_CLIENT_ADDRESS_KEY;
|
.OZONE_SCM_CLIENT_ADDRESS_KEY;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test Key Space Manager operation in distributed handler scenario.
|
* Test Ozone Manager operation in distributed handler scenario.
|
||||||
*/
|
*/
|
||||||
public class TestKeySpaceManager {
|
public class TestOzoneManager {
|
||||||
private static MiniOzoneCluster cluster = null;
|
private static MiniOzoneCluster cluster = null;
|
||||||
private static StorageHandler storageHandler;
|
private static StorageHandler storageHandler;
|
||||||
private static UserArgs userArgs;
|
private static UserArgs userArgs;
|
||||||
private static KSMMetrics ksmMetrics;
|
private static OMMetrics omMetrics;
|
||||||
private static OzoneConfiguration conf;
|
private static OzoneConfiguration conf;
|
||||||
private static String clusterId;
|
private static String clusterId;
|
||||||
private static String scmId;
|
private static String scmId;
|
||||||
private static String ksmId;
|
private static String omId;
|
||||||
|
|
||||||
@Rule
|
@Rule
|
||||||
public ExpectedException exception = ExpectedException.none();
|
public ExpectedException exception = ExpectedException.none();
|
||||||
|
@ -119,20 +118,20 @@ public class TestKeySpaceManager {
|
||||||
conf = new OzoneConfiguration();
|
conf = new OzoneConfiguration();
|
||||||
clusterId = UUID.randomUUID().toString();
|
clusterId = UUID.randomUUID().toString();
|
||||||
scmId = UUID.randomUUID().toString();
|
scmId = UUID.randomUUID().toString();
|
||||||
ksmId = UUID.randomUUID().toString();
|
omId = UUID.randomUUID().toString();
|
||||||
conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
|
conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
|
||||||
OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
|
OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
|
||||||
conf.setInt(OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS, 2);
|
conf.setInt(OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS, 2);
|
||||||
cluster = MiniOzoneCluster.newBuilder(conf)
|
cluster = MiniOzoneCluster.newBuilder(conf)
|
||||||
.setClusterId(clusterId)
|
.setClusterId(clusterId)
|
||||||
.setScmId(scmId)
|
.setScmId(scmId)
|
||||||
.setKsmId(ksmId)
|
.setOmId(omId)
|
||||||
.build();
|
.build();
|
||||||
cluster.waitForClusterToBeReady();
|
cluster.waitForClusterToBeReady();
|
||||||
storageHandler = new ObjectStoreHandler(conf).getStorageHandler();
|
storageHandler = new ObjectStoreHandler(conf).getStorageHandler();
|
||||||
userArgs = new UserArgs(null, OzoneUtils.getRequestID(),
|
userArgs = new UserArgs(null, OzoneUtils.getRequestID(),
|
||||||
null, null, null, null);
|
null, null, null, null);
|
||||||
ksmMetrics = cluster.getKeySpaceManager().getMetrics();
|
omMetrics = cluster.getOzoneManager().getMetrics();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -148,7 +147,7 @@ public class TestKeySpaceManager {
|
||||||
// Create a volume and test its attribute after creating them
|
// Create a volume and test its attribute after creating them
|
||||||
@Test(timeout = 60000)
|
@Test(timeout = 60000)
|
||||||
public void testCreateVolume() throws IOException, OzoneException {
|
public void testCreateVolume() throws IOException, OzoneException {
|
||||||
long volumeCreateFailCount = ksmMetrics.getNumVolumeCreateFails();
|
long volumeCreateFailCount = omMetrics.getNumVolumeCreateFails();
|
||||||
String userName = "user" + RandomStringUtils.randomNumeric(5);
|
String userName = "user" + RandomStringUtils.randomNumeric(5);
|
||||||
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
|
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
|
||||||
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
||||||
|
@ -163,14 +162,14 @@ public class TestKeySpaceManager {
|
||||||
Assert.assertTrue(retVolumeinfo.getVolumeName().equals(volumeName));
|
Assert.assertTrue(retVolumeinfo.getVolumeName().equals(volumeName));
|
||||||
Assert.assertTrue(retVolumeinfo.getOwner().getName().equals(userName));
|
Assert.assertTrue(retVolumeinfo.getOwner().getName().equals(userName));
|
||||||
Assert.assertEquals(volumeCreateFailCount,
|
Assert.assertEquals(volumeCreateFailCount,
|
||||||
ksmMetrics.getNumVolumeCreateFails());
|
omMetrics.getNumVolumeCreateFails());
|
||||||
}
|
}
|
||||||
|
|
||||||
// Create a volume and modify the volume owner and then test its attributes
|
// Create a volume and modify the volume owner and then test its attributes
|
||||||
@Test(timeout = 60000)
|
@Test(timeout = 60000)
|
||||||
public void testChangeVolumeOwner() throws IOException, OzoneException {
|
public void testChangeVolumeOwner() throws IOException, OzoneException {
|
||||||
long volumeCreateFailCount = ksmMetrics.getNumVolumeCreateFails();
|
long volumeCreateFailCount = omMetrics.getNumVolumeCreateFails();
|
||||||
long volumeInfoFailCount = ksmMetrics.getNumVolumeInfoFails();
|
long volumeInfoFailCount = omMetrics.getNumVolumeInfoFails();
|
||||||
String userName = "user" + RandomStringUtils.randomNumeric(5);
|
String userName = "user" + RandomStringUtils.randomNumeric(5);
|
||||||
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
|
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
|
||||||
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
||||||
|
@ -191,16 +190,16 @@ public class TestKeySpaceManager {
|
||||||
Assert.assertFalse(retVolumeInfo.getOwner().getName().equals(userName));
|
Assert.assertFalse(retVolumeInfo.getOwner().getName().equals(userName));
|
||||||
Assert.assertTrue(retVolumeInfo.getOwner().getName().equals(newUserName));
|
Assert.assertTrue(retVolumeInfo.getOwner().getName().equals(newUserName));
|
||||||
Assert.assertEquals(volumeCreateFailCount,
|
Assert.assertEquals(volumeCreateFailCount,
|
||||||
ksmMetrics.getNumVolumeCreateFails());
|
omMetrics.getNumVolumeCreateFails());
|
||||||
Assert.assertEquals(volumeInfoFailCount,
|
Assert.assertEquals(volumeInfoFailCount,
|
||||||
ksmMetrics.getNumVolumeInfoFails());
|
omMetrics.getNumVolumeInfoFails());
|
||||||
}
|
}
|
||||||
|
|
||||||
// Create a volume and modify the volume owner and then test its attributes
|
// Create a volume and modify the volume owner and then test its attributes
|
||||||
@Test(timeout = 60000)
|
@Test(timeout = 60000)
|
||||||
public void testChangeVolumeQuota() throws IOException, OzoneException {
|
public void testChangeVolumeQuota() throws IOException, OzoneException {
|
||||||
long numVolumeCreateFail = ksmMetrics.getNumVolumeCreateFails();
|
long numVolumeCreateFail = omMetrics.getNumVolumeCreateFails();
|
||||||
long numVolumeInfoFail = ksmMetrics.getNumVolumeInfoFails();
|
long numVolumeInfoFail = omMetrics.getNumVolumeInfoFails();
|
||||||
String userName = "user" + RandomStringUtils.randomNumeric(5);
|
String userName = "user" + RandomStringUtils.randomNumeric(5);
|
||||||
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
|
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
|
||||||
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
||||||
|
@ -237,15 +236,15 @@ public class TestKeySpaceManager {
|
||||||
Assert.assertEquals(OzoneConsts.MAX_QUOTA_IN_BYTES,
|
Assert.assertEquals(OzoneConsts.MAX_QUOTA_IN_BYTES,
|
||||||
retVolumeInfo.getQuota().sizeInBytes());
|
retVolumeInfo.getQuota().sizeInBytes());
|
||||||
Assert.assertEquals(numVolumeCreateFail,
|
Assert.assertEquals(numVolumeCreateFail,
|
||||||
ksmMetrics.getNumVolumeCreateFails());
|
omMetrics.getNumVolumeCreateFails());
|
||||||
Assert.assertEquals(numVolumeInfoFail,
|
Assert.assertEquals(numVolumeInfoFail,
|
||||||
ksmMetrics.getNumVolumeInfoFails());
|
omMetrics.getNumVolumeInfoFails());
|
||||||
}
|
}
|
||||||
|
|
||||||
// Create a volume and then delete it and then check for deletion
|
// Create a volume and then delete it and then check for deletion
|
||||||
@Test(timeout = 60000)
|
@Test(timeout = 60000)
|
||||||
public void testDeleteVolume() throws IOException, OzoneException {
|
public void testDeleteVolume() throws IOException, OzoneException {
|
||||||
long volumeCreateFailCount = ksmMetrics.getNumVolumeCreateFails();
|
long volumeCreateFailCount = omMetrics.getNumVolumeCreateFails();
|
||||||
String userName = "user" + RandomStringUtils.randomNumeric(5);
|
String userName = "user" + RandomStringUtils.randomNumeric(5);
|
||||||
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
|
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
|
||||||
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
||||||
|
@ -270,7 +269,7 @@ public class TestKeySpaceManager {
|
||||||
Assert.assertTrue(volumeInfo.getVolumeName().equals(volumeName1));
|
Assert.assertTrue(volumeInfo.getVolumeName().equals(volumeName1));
|
||||||
Assert.assertTrue(volumeInfo.getOwner().getName().equals(userName));
|
Assert.assertTrue(volumeInfo.getOwner().getName().equals(userName));
|
||||||
Assert.assertEquals(volumeCreateFailCount,
|
Assert.assertEquals(volumeCreateFailCount,
|
||||||
ksmMetrics.getNumVolumeCreateFails());
|
omMetrics.getNumVolumeCreateFails());
|
||||||
|
|
||||||
// Volume with _A should be able to delete as it is empty.
|
// Volume with _A should be able to delete as it is empty.
|
||||||
storageHandler.deleteVolume(volumeArgs);
|
storageHandler.deleteVolume(volumeArgs);
|
||||||
|
@ -291,7 +290,7 @@ public class TestKeySpaceManager {
|
||||||
// then delete it and then check for deletion failure
|
// then delete it and then check for deletion failure
|
||||||
@Test(timeout = 60000)
|
@Test(timeout = 60000)
|
||||||
public void testFailedDeleteVolume() throws IOException, OzoneException {
|
public void testFailedDeleteVolume() throws IOException, OzoneException {
|
||||||
long numVolumeCreateFails = ksmMetrics.getNumVolumeCreateFails();
|
long numVolumeCreateFails = omMetrics.getNumVolumeCreateFails();
|
||||||
String userName = "user" + RandomStringUtils.randomNumeric(5);
|
String userName = "user" + RandomStringUtils.randomNumeric(5);
|
||||||
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
|
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
|
||||||
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
||||||
|
@ -307,7 +306,7 @@ public class TestKeySpaceManager {
|
||||||
Assert.assertTrue(retVolumeInfo.getVolumeName().equals(volumeName));
|
Assert.assertTrue(retVolumeInfo.getVolumeName().equals(volumeName));
|
||||||
Assert.assertTrue(retVolumeInfo.getOwner().getName().equals(userName));
|
Assert.assertTrue(retVolumeInfo.getOwner().getName().equals(userName));
|
||||||
Assert.assertEquals(numVolumeCreateFails,
|
Assert.assertEquals(numVolumeCreateFails,
|
||||||
ksmMetrics.getNumVolumeCreateFails());
|
omMetrics.getNumVolumeCreateFails());
|
||||||
|
|
||||||
BucketArgs bucketArgs = new BucketArgs(volumeName, bucketName, userArgs);
|
BucketArgs bucketArgs = new BucketArgs(volumeName, bucketName, userArgs);
|
||||||
storageHandler.createBucket(bucketArgs);
|
storageHandler.createBucket(bucketArgs);
|
||||||
|
@ -366,15 +365,15 @@ public class TestKeySpaceManager {
|
||||||
OzoneAcl.OzoneACLRights.READ);
|
OzoneAcl.OzoneACLRights.READ);
|
||||||
Assert.assertFalse(storageHandler.checkVolumeAccess(volumeName, worldAcl));
|
Assert.assertFalse(storageHandler.checkVolumeAccess(volumeName, worldAcl));
|
||||||
|
|
||||||
Assert.assertEquals(0, ksmMetrics.getNumVolumeCheckAccessFails());
|
Assert.assertEquals(0, omMetrics.getNumVolumeCheckAccessFails());
|
||||||
Assert.assertEquals(0, ksmMetrics.getNumVolumeCreateFails());
|
Assert.assertEquals(0, omMetrics.getNumVolumeCreateFails());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 60000)
|
@Test(timeout = 60000)
|
||||||
public void testCreateBucket() throws IOException, OzoneException {
|
public void testCreateBucket() throws IOException, OzoneException {
|
||||||
long numVolumeCreateFail = ksmMetrics.getNumVolumeCreateFails();
|
long numVolumeCreateFail = omMetrics.getNumVolumeCreateFails();
|
||||||
long numBucketCreateFail = ksmMetrics.getNumBucketCreateFails();
|
long numBucketCreateFail = omMetrics.getNumBucketCreateFails();
|
||||||
long numBucketInfoFail = ksmMetrics.getNumBucketInfoFails();
|
long numBucketInfoFail = omMetrics.getNumBucketInfoFails();
|
||||||
String userName = "user" + RandomStringUtils.randomNumeric(5);
|
String userName = "user" + RandomStringUtils.randomNumeric(5);
|
||||||
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
|
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
|
||||||
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
||||||
|
@ -394,11 +393,11 @@ public class TestKeySpaceManager {
|
||||||
Assert.assertTrue(bucketInfo.getVolumeName().equals(volumeName));
|
Assert.assertTrue(bucketInfo.getVolumeName().equals(volumeName));
|
||||||
Assert.assertTrue(bucketInfo.getBucketName().equals(bucketName));
|
Assert.assertTrue(bucketInfo.getBucketName().equals(bucketName));
|
||||||
Assert.assertEquals(numVolumeCreateFail,
|
Assert.assertEquals(numVolumeCreateFail,
|
||||||
ksmMetrics.getNumVolumeCreateFails());
|
omMetrics.getNumVolumeCreateFails());
|
||||||
Assert.assertEquals(numBucketCreateFail,
|
Assert.assertEquals(numBucketCreateFail,
|
||||||
ksmMetrics.getNumBucketCreateFails());
|
omMetrics.getNumBucketCreateFails());
|
||||||
Assert.assertEquals(numBucketInfoFail,
|
Assert.assertEquals(numBucketInfoFail,
|
||||||
ksmMetrics.getNumBucketInfoFails());
|
omMetrics.getNumBucketInfoFails());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 60000)
|
@Test(timeout = 60000)
|
||||||
|
@ -479,7 +478,7 @@ public class TestKeySpaceManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Basic test of both putKey and getKey from KSM, as one can not be tested
|
* Basic test of both putKey and getKey from OM, as one can not be tested
|
||||||
* without the other.
|
* without the other.
|
||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
|
@ -492,8 +491,8 @@ public class TestKeySpaceManager {
|
||||||
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
||||||
String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
|
String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
|
||||||
String keyName = "key" + RandomStringUtils.randomNumeric(5);
|
String keyName = "key" + RandomStringUtils.randomNumeric(5);
|
||||||
long numKeyAllocates = ksmMetrics.getNumKeyAllocates();
|
long numKeyAllocates = omMetrics.getNumKeyAllocates();
|
||||||
long numKeyLookups = ksmMetrics.getNumKeyLookups();
|
long numKeyLookups = omMetrics.getNumKeyLookups();
|
||||||
|
|
||||||
VolumeArgs createVolumeArgs = new VolumeArgs(volumeName, userArgs);
|
VolumeArgs createVolumeArgs = new VolumeArgs(volumeName, userArgs);
|
||||||
createVolumeArgs.setUserName(userName);
|
createVolumeArgs.setUserName(userName);
|
||||||
|
@ -512,14 +511,14 @@ public class TestKeySpaceManager {
|
||||||
try (OutputStream stream = storageHandler.newKeyWriter(keyArgs)) {
|
try (OutputStream stream = storageHandler.newKeyWriter(keyArgs)) {
|
||||||
stream.write(dataString.getBytes());
|
stream.write(dataString.getBytes());
|
||||||
}
|
}
|
||||||
Assert.assertEquals(1 + numKeyAllocates, ksmMetrics.getNumKeyAllocates());
|
Assert.assertEquals(1 + numKeyAllocates, omMetrics.getNumKeyAllocates());
|
||||||
|
|
||||||
byte[] data = new byte[dataString.length()];
|
byte[] data = new byte[dataString.length()];
|
||||||
try (InputStream in = storageHandler.newKeyReader(keyArgs)) {
|
try (InputStream in = storageHandler.newKeyReader(keyArgs)) {
|
||||||
in.read(data);
|
in.read(data);
|
||||||
}
|
}
|
||||||
Assert.assertEquals(dataString, DFSUtil.bytes2String(data));
|
Assert.assertEquals(dataString, DFSUtil.bytes2String(data));
|
||||||
Assert.assertEquals(1 + numKeyLookups, ksmMetrics.getNumKeyLookups());
|
Assert.assertEquals(1 + numKeyLookups, omMetrics.getNumKeyLookups());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -536,7 +535,7 @@ public class TestKeySpaceManager {
|
||||||
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
||||||
String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
|
String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
|
||||||
String keyName = "key" + RandomStringUtils.randomNumeric(5);
|
String keyName = "key" + RandomStringUtils.randomNumeric(5);
|
||||||
long numKeyAllocateFails = ksmMetrics.getNumKeyAllocateFails();
|
long numKeyAllocateFails = omMetrics.getNumKeyAllocateFails();
|
||||||
|
|
||||||
VolumeArgs createVolumeArgs = new VolumeArgs(volumeName, userArgs);
|
VolumeArgs createVolumeArgs = new VolumeArgs(volumeName, userArgs);
|
||||||
createVolumeArgs.setUserName(userName);
|
createVolumeArgs.setUserName(userName);
|
||||||
|
@ -558,12 +557,12 @@ public class TestKeySpaceManager {
|
||||||
|
|
||||||
// We allow the key overwrite to be successful. Please note : Till
|
// 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.
|
// HDFS-11922 is fixed this causes a data block leak on the data node side.
|
||||||
// That is this overwrite only overwrites the keys on KSM. We need to
|
// That is this overwrite only overwrites the keys on OM. We need to
|
||||||
// garbage collect those blocks from datanode.
|
// garbage collect those blocks from datanode.
|
||||||
KeyArgs keyArgs2 = new KeyArgs(volumeName, bucketName, keyName, userArgs);
|
KeyArgs keyArgs2 = new KeyArgs(volumeName, bucketName, keyName, userArgs);
|
||||||
storageHandler.newKeyWriter(keyArgs2);
|
storageHandler.newKeyWriter(keyArgs2);
|
||||||
Assert
|
Assert
|
||||||
.assertEquals(numKeyAllocateFails, ksmMetrics.getNumKeyAllocateFails());
|
.assertEquals(numKeyAllocateFails, omMetrics.getNumKeyAllocateFails());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -579,7 +578,7 @@ public class TestKeySpaceManager {
|
||||||
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
||||||
String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
|
String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
|
||||||
String keyName = "key" + RandomStringUtils.randomNumeric(5);
|
String keyName = "key" + RandomStringUtils.randomNumeric(5);
|
||||||
long numKeyLookupFails = ksmMetrics.getNumKeyLookupFails();
|
long numKeyLookupFails = omMetrics.getNumKeyLookupFails();
|
||||||
|
|
||||||
VolumeArgs createVolumeArgs = new VolumeArgs(volumeName, userArgs);
|
VolumeArgs createVolumeArgs = new VolumeArgs(volumeName, userArgs);
|
||||||
createVolumeArgs.setUserName(userName);
|
createVolumeArgs.setUserName(userName);
|
||||||
|
@ -598,11 +597,11 @@ public class TestKeySpaceManager {
|
||||||
exception.expectMessage("KEY_NOT_FOUND");
|
exception.expectMessage("KEY_NOT_FOUND");
|
||||||
storageHandler.newKeyReader(keyArgs);
|
storageHandler.newKeyReader(keyArgs);
|
||||||
Assert.assertEquals(1 + numKeyLookupFails,
|
Assert.assertEquals(1 + numKeyLookupFails,
|
||||||
ksmMetrics.getNumKeyLookupFails());
|
omMetrics.getNumKeyLookupFails());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test delete keys for ksm.
|
* Test delete keys for om.
|
||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
* @throws OzoneException
|
* @throws OzoneException
|
||||||
|
@ -614,8 +613,8 @@ public class TestKeySpaceManager {
|
||||||
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
||||||
String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
|
String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
|
||||||
String keyName = "key" + RandomStringUtils.randomNumeric(5);
|
String keyName = "key" + RandomStringUtils.randomNumeric(5);
|
||||||
long numKeyDeletes = ksmMetrics.getNumKeyDeletes();
|
long numKeyDeletes = omMetrics.getNumKeyDeletes();
|
||||||
long numKeyDeleteFails = ksmMetrics.getNumKeyDeletesFails();
|
long numKeyDeleteFails = omMetrics.getNumKeyDeletesFails();
|
||||||
|
|
||||||
VolumeArgs createVolumeArgs = new VolumeArgs(volumeName, userArgs);
|
VolumeArgs createVolumeArgs = new VolumeArgs(volumeName, userArgs);
|
||||||
createVolumeArgs.setUserName(userName);
|
createVolumeArgs.setUserName(userName);
|
||||||
|
@ -633,10 +632,10 @@ public class TestKeySpaceManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
storageHandler.deleteKey(keyArgs);
|
storageHandler.deleteKey(keyArgs);
|
||||||
Assert.assertEquals(1 + numKeyDeletes, ksmMetrics.getNumKeyDeletes());
|
Assert.assertEquals(1 + numKeyDeletes, omMetrics.getNumKeyDeletes());
|
||||||
|
|
||||||
// Make sure the deleted key has been renamed.
|
// Make sure the deleted key has been renamed.
|
||||||
MetadataStore store = cluster.getKeySpaceManager().
|
MetadataStore store = cluster.getOzoneManager().
|
||||||
getMetadataManager().getStore();
|
getMetadataManager().getStore();
|
||||||
List<Map.Entry<byte[], byte[]>> list = store.getRangeKVs(null, 10,
|
List<Map.Entry<byte[], byte[]>> list = store.getRangeKVs(null, 10,
|
||||||
new MetadataKeyFilters.KeyPrefixFilter()
|
new MetadataKeyFilters.KeyPrefixFilter()
|
||||||
|
@ -651,11 +650,11 @@ public class TestKeySpaceManager {
|
||||||
Assert.assertTrue(ioe.getMessage().contains("KEY_NOT_FOUND"));
|
Assert.assertTrue(ioe.getMessage().contains("KEY_NOT_FOUND"));
|
||||||
}
|
}
|
||||||
Assert.assertEquals(1 + numKeyDeleteFails,
|
Assert.assertEquals(1 + numKeyDeleteFails,
|
||||||
ksmMetrics.getNumKeyDeletesFails());
|
omMetrics.getNumKeyDeletesFails());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test rename key for ksm.
|
* Test rename key for om.
|
||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
* @throws OzoneException
|
* @throws OzoneException
|
||||||
|
@ -667,8 +666,8 @@ public class TestKeySpaceManager {
|
||||||
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
|
||||||
String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
|
String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
|
||||||
String keyName = "key" + RandomStringUtils.randomNumeric(5);
|
String keyName = "key" + RandomStringUtils.randomNumeric(5);
|
||||||
long numKeyRenames = ksmMetrics.getNumKeyRenames();
|
long numKeyRenames = omMetrics.getNumKeyRenames();
|
||||||
long numKeyRenameFails = ksmMetrics.getNumKeyRenameFails();
|
long numKeyRenameFails = omMetrics.getNumKeyRenameFails();
|
||||||
int testRenameFails = 0;
|
int testRenameFails = 0;
|
||||||
int testRenames = 0;
|
int testRenames = 0;
|
||||||
IOException ioe = null;
|
IOException ioe = null;
|
||||||
|
@ -706,9 +705,9 @@ public class TestKeySpaceManager {
|
||||||
testRenames++;
|
testRenames++;
|
||||||
storageHandler.renameKey(keyArgs, toKeyName);
|
storageHandler.renameKey(keyArgs, toKeyName);
|
||||||
Assert.assertEquals(numKeyRenames + testRenames,
|
Assert.assertEquals(numKeyRenames + testRenames,
|
||||||
ksmMetrics.getNumKeyRenames());
|
omMetrics.getNumKeyRenames());
|
||||||
Assert.assertEquals(numKeyRenameFails + testRenameFails,
|
Assert.assertEquals(numKeyRenameFails + testRenameFails,
|
||||||
ksmMetrics.getNumKeyRenameFails());
|
omMetrics.getNumKeyRenameFails());
|
||||||
|
|
||||||
// Try to get the key, should fail as it has been renamed
|
// Try to get the key, should fail as it has been renamed
|
||||||
try {
|
try {
|
||||||
|
@ -764,9 +763,9 @@ public class TestKeySpaceManager {
|
||||||
Assert.assertTrue(ioe.getMessage().contains("Rename key failed, error"));
|
Assert.assertTrue(ioe.getMessage().contains("Rename key failed, error"));
|
||||||
|
|
||||||
Assert.assertEquals(numKeyRenames + testRenames,
|
Assert.assertEquals(numKeyRenames + testRenames,
|
||||||
ksmMetrics.getNumKeyRenames());
|
omMetrics.getNumKeyRenames());
|
||||||
Assert.assertEquals(numKeyRenameFails + testRenameFails,
|
Assert.assertEquals(numKeyRenameFails + testRenameFails,
|
||||||
ksmMetrics.getNumKeyRenameFails());
|
omMetrics.getNumKeyRenameFails());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 60000)
|
@Test(timeout = 60000)
|
||||||
|
@ -1183,7 +1182,7 @@ public class TestKeySpaceManager {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testGetScmInfo() throws IOException {
|
public void testGetScmInfo() throws IOException {
|
||||||
ScmInfo info = cluster.getKeySpaceManager().getScmInfo();
|
ScmInfo info = cluster.getOzoneManager().getScmInfo();
|
||||||
Assert.assertEquals(clusterId, info.getClusterId());
|
Assert.assertEquals(clusterId, info.getClusterId());
|
||||||
Assert.assertEquals(scmId, info.getScmId());
|
Assert.assertEquals(scmId, info.getScmId());
|
||||||
}
|
}
|
||||||
|
@ -1192,7 +1191,7 @@ public class TestKeySpaceManager {
|
||||||
@Test
|
@Test
|
||||||
public void testExpiredOpenKey() throws Exception {
|
public void testExpiredOpenKey() throws Exception {
|
||||||
BackgroundService openKeyCleanUpService = ((KeyManagerImpl)cluster
|
BackgroundService openKeyCleanUpService = ((KeyManagerImpl)cluster
|
||||||
.getKeySpaceManager().getKeyManager()).getOpenKeyCleanupService();
|
.getOzoneManager().getKeyManager()).getOpenKeyCleanupService();
|
||||||
|
|
||||||
String userName = "user" + RandomStringUtils.randomNumeric(5);
|
String userName = "user" + RandomStringUtils.randomNumeric(5);
|
||||||
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
|
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
|
||||||
|
@ -1228,7 +1227,7 @@ public class TestKeySpaceManager {
|
||||||
|
|
||||||
// Now all k1-k4 should be in open state, so ExpiredOpenKeys should not
|
// Now all k1-k4 should be in open state, so ExpiredOpenKeys should not
|
||||||
// contain these values.
|
// contain these values.
|
||||||
openKeys = cluster.getKeySpaceManager()
|
openKeys = cluster.getOzoneManager()
|
||||||
.getMetadataManager().getExpiredOpenKeys();
|
.getMetadataManager().getExpiredOpenKeys();
|
||||||
|
|
||||||
for (BlockGroup bg : openKeys) {
|
for (BlockGroup bg : openKeys) {
|
||||||
|
@ -1239,7 +1238,7 @@ public class TestKeySpaceManager {
|
||||||
|
|
||||||
Thread.sleep(2000);
|
Thread.sleep(2000);
|
||||||
// Now all k1-k4 should be in ExpiredOpenKeys
|
// Now all k1-k4 should be in ExpiredOpenKeys
|
||||||
openKeys = cluster.getKeySpaceManager()
|
openKeys = cluster.getOzoneManager()
|
||||||
.getMetadataManager().getExpiredOpenKeys();
|
.getMetadataManager().getExpiredOpenKeys();
|
||||||
for (BlockGroup bg : openKeys) {
|
for (BlockGroup bg : openKeys) {
|
||||||
String[] subs = bg.getGroupID().split("/");
|
String[] subs = bg.getGroupID().split("/");
|
||||||
|
@ -1258,7 +1257,7 @@ public class TestKeySpaceManager {
|
||||||
// now all k1-k4 should have been removed by the clean-up task, only k5
|
// now all k1-k4 should have been removed by the clean-up task, only k5
|
||||||
// should be present in ExpiredOpenKeys.
|
// should be present in ExpiredOpenKeys.
|
||||||
openKeys =
|
openKeys =
|
||||||
cluster.getKeySpaceManager().getMetadataManager().getExpiredOpenKeys();
|
cluster.getOzoneManager().getMetadataManager().getExpiredOpenKeys();
|
||||||
System.out.println(openKeys);
|
System.out.println(openKeys);
|
||||||
boolean key5found = false;
|
boolean key5found = false;
|
||||||
Set<String> removed = Stream.of(
|
Set<String> removed = Stream.of(
|
||||||
|
@ -1276,68 +1275,68 @@ public class TestKeySpaceManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Tests the KSM Initialization.
|
* Tests the OM Initialization.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testKSMInitialization() throws IOException {
|
public void testOmInitialization() throws IOException {
|
||||||
// Read the version file info from KSM version file
|
// Read the version file info from OM version file
|
||||||
KSMStorage ksmStorage = cluster.getKeySpaceManager().getKsmStorage();
|
OMStorage omStorage = cluster.getOzoneManager().getOmStorage();
|
||||||
SCMStorage scmStorage = new SCMStorage(conf);
|
SCMStorage scmStorage = new SCMStorage(conf);
|
||||||
// asserts whether cluster Id and SCM ID are properly set in SCM Version
|
// asserts whether cluster Id and SCM ID are properly set in SCM Version
|
||||||
// file.
|
// file.
|
||||||
Assert.assertEquals(clusterId, scmStorage.getClusterID());
|
Assert.assertEquals(clusterId, scmStorage.getClusterID());
|
||||||
Assert.assertEquals(scmId, scmStorage.getScmId());
|
Assert.assertEquals(scmId, scmStorage.getScmId());
|
||||||
// asserts whether KSM Id is properly set in KSM Version file.
|
// asserts whether OM Id is properly set in OM Version file.
|
||||||
Assert.assertEquals(ksmId, ksmStorage.getKsmId());
|
Assert.assertEquals(omId, omStorage.getOmId());
|
||||||
// asserts whether the SCM info is correct in KSM Version file.
|
// asserts whether the SCM info is correct in OM Version file.
|
||||||
Assert.assertEquals(clusterId, ksmStorage.getClusterID());
|
Assert.assertEquals(clusterId, omStorage.getClusterID());
|
||||||
Assert.assertEquals(scmId, ksmStorage.getScmId());
|
Assert.assertEquals(scmId, omStorage.getScmId());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Tests the KSM Initialization Failure.
|
* Tests the OM Initialization Failure.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testKSMInitializationFailure() throws Exception {
|
public void testOmInitializationFailure() throws Exception {
|
||||||
OzoneConfiguration config = new OzoneConfiguration();
|
OzoneConfiguration config = new OzoneConfiguration();
|
||||||
final String path =
|
final String path =
|
||||||
GenericTestUtils.getTempPath(UUID.randomUUID().toString());
|
GenericTestUtils.getTempPath(UUID.randomUUID().toString());
|
||||||
Path metaDirPath = Paths.get(path, "ksm-meta");
|
Path metaDirPath = Paths.get(path, "om-meta");
|
||||||
config.set(OzoneConfigKeys.OZONE_METADATA_DIRS, metaDirPath.toString());
|
config.set(OzoneConfigKeys.OZONE_METADATA_DIRS, metaDirPath.toString());
|
||||||
config.setBoolean(OzoneConfigKeys.OZONE_ENABLED, true);
|
config.setBoolean(OzoneConfigKeys.OZONE_ENABLED, true);
|
||||||
config.set(ScmConfigKeys.OZONE_SCM_CLIENT_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,
|
config.set(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY,
|
||||||
conf.get(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY));
|
conf.get(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY));
|
||||||
exception.expect(KSMException.class);
|
exception.expect(OMException.class);
|
||||||
exception.expectMessage("KSM not initialized.");
|
exception.expectMessage("OM not initialized.");
|
||||||
KeySpaceManager.createKSM(null, config);
|
OzoneManager.createOm(null, config);
|
||||||
KSMStorage ksmStore = new KSMStorage(config);
|
OMStorage omStore = new OMStorage(config);
|
||||||
ksmStore.setClusterId("testClusterId");
|
omStore.setClusterId("testClusterId");
|
||||||
ksmStore.setScmId("testScmId");
|
omStore.setScmId("testScmId");
|
||||||
// writes the version file properties
|
// writes the version file properties
|
||||||
ksmStore.initialize();
|
omStore.initialize();
|
||||||
exception.expect(KSMException.class);
|
exception.expect(OMException.class);
|
||||||
exception.expectMessage("SCM version info mismatch.");
|
exception.expectMessage("SCM version info mismatch.");
|
||||||
KeySpaceManager.createKSM(null, conf);
|
OzoneManager.createOm(null, conf);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testGetServiceList() throws IOException {
|
public void testGetServiceList() throws IOException {
|
||||||
long numGetServiceListCalls = ksmMetrics.getNumGetServiceLists();
|
long numGetServiceListCalls = omMetrics.getNumGetServiceLists();
|
||||||
List<ServiceInfo> services = cluster.getKeySpaceManager().getServiceList();
|
List<ServiceInfo> services = cluster.getOzoneManager().getServiceList();
|
||||||
|
|
||||||
Assert.assertEquals(numGetServiceListCalls + 1,
|
Assert.assertEquals(numGetServiceListCalls + 1,
|
||||||
ksmMetrics.getNumGetServiceLists());
|
omMetrics.getNumGetServiceLists());
|
||||||
|
|
||||||
ServiceInfo ksmInfo = services.stream().filter(
|
ServiceInfo omInfo = services.stream().filter(
|
||||||
a -> a.getNodeType().equals(HddsProtos.NodeType.KSM))
|
a -> a.getNodeType().equals(HddsProtos.NodeType.OM))
|
||||||
.collect(Collectors.toList()).get(0);
|
.collect(Collectors.toList()).get(0);
|
||||||
InetSocketAddress ksmAddress = new InetSocketAddress(ksmInfo.getHostname(),
|
InetSocketAddress omAddress = new InetSocketAddress(omInfo.getHostname(),
|
||||||
ksmInfo.getPort(ServicePort.Type.RPC));
|
omInfo.getPort(ServicePort.Type.RPC));
|
||||||
Assert.assertEquals(NetUtils.createSocketAddr(
|
Assert.assertEquals(NetUtils.createSocketAddr(
|
||||||
conf.get(OZONE_KSM_ADDRESS_KEY)), ksmAddress);
|
conf.get(OZONE_OM_ADDRESS_KEY)), omAddress);
|
||||||
|
|
||||||
ServiceInfo scmInfo = services.stream().filter(
|
ServiceInfo scmInfo = services.stream().filter(
|
||||||
a -> a.getNodeType().equals(HddsProtos.NodeType.SCM))
|
a -> a.getNodeType().equals(HddsProtos.NodeType.SCM))
|
|
@ -16,7 +16,7 @@
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package org.apache.hadoop.ozone.ksm;
|
package org.apache.hadoop.ozone.om;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.fasterxml.jackson.core.type.TypeReference;
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
|
@ -24,9 +24,9 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.net.NetUtils;
|
import org.apache.hadoop.net.NetUtils;
|
||||||
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
|
import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.ServicePort;
|
.OzoneManagerProtocolProtos.ServicePort;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.http.HttpResponse;
|
import org.apache.http.HttpResponse;
|
||||||
import org.apache.http.client.HttpClient;
|
import org.apache.http.client.HttpClient;
|
||||||
|
@ -44,12 +44,12 @@ import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
|
import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
|
||||||
import static org.apache.hadoop.ozone.KsmUtils.getKsmAddressForClients;
|
import static org.apache.hadoop.ozone.OmUtils.getOmAddressForClients;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class is to test the REST interface exposed by KeySpaceManager.
|
* This class is to test the REST interface exposed by OzoneManager.
|
||||||
*/
|
*/
|
||||||
public class TestKeySpaceManagerRestInterface {
|
public class TestOzoneManagerRestInterface {
|
||||||
|
|
||||||
private static MiniOzoneCluster cluster;
|
private static MiniOzoneCluster cluster;
|
||||||
private static OzoneConfiguration conf;
|
private static OzoneConfiguration conf;
|
||||||
|
@ -70,8 +70,8 @@ public class TestKeySpaceManagerRestInterface {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testGetServiceList() throws Exception {
|
public void testGetServiceList() throws Exception {
|
||||||
KeySpaceManagerHttpServer server =
|
OzoneManagerHttpServer server =
|
||||||
cluster.getKeySpaceManager().getHttpServer();
|
cluster.getOzoneManager().getHttpServer();
|
||||||
HttpClient client = HttpClients.createDefault();
|
HttpClient client = HttpClients.createDefault();
|
||||||
String connectionUri = "http://" +
|
String connectionUri = "http://" +
|
||||||
NetUtils.getHostPortString(server.getHttpAddress());
|
NetUtils.getHostPortString(server.getHttpAddress());
|
||||||
|
@ -89,15 +89,15 @@ public class TestKeySpaceManagerRestInterface {
|
||||||
serviceMap.put(serviceInfo.getNodeType(), serviceInfo);
|
serviceMap.put(serviceInfo.getNodeType(), serviceInfo);
|
||||||
}
|
}
|
||||||
|
|
||||||
InetSocketAddress ksmAddress =
|
InetSocketAddress omAddress =
|
||||||
getKsmAddressForClients(conf);
|
getOmAddressForClients(conf);
|
||||||
ServiceInfo ksmInfo = serviceMap.get(HddsProtos.NodeType.KSM);
|
ServiceInfo omInfo = serviceMap.get(HddsProtos.NodeType.OM);
|
||||||
|
|
||||||
Assert.assertEquals(ksmAddress.getHostName(), ksmInfo.getHostname());
|
Assert.assertEquals(omAddress.getHostName(), omInfo.getHostname());
|
||||||
Assert.assertEquals(ksmAddress.getPort(),
|
Assert.assertEquals(omAddress.getPort(),
|
||||||
ksmInfo.getPort(ServicePort.Type.RPC));
|
omInfo.getPort(ServicePort.Type.RPC));
|
||||||
Assert.assertEquals(server.getHttpAddress().getPort(),
|
Assert.assertEquals(server.getHttpAddress().getPort(),
|
||||||
ksmInfo.getPort(ServicePort.Type.HTTP));
|
omInfo.getPort(ServicePort.Type.HTTP));
|
||||||
|
|
||||||
InetSocketAddress scmAddress =
|
InetSocketAddress scmAddress =
|
||||||
getScmAddressForClients(conf);
|
getScmAddressForClients(conf);
|
||||||
|
@ -123,7 +123,7 @@ public class TestKeySpaceManagerRestInterface {
|
||||||
ports.get(type));
|
ports.get(type));
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
// KSM only sends Datanode's info port details
|
// OM only sends Datanode's info port details
|
||||||
// i.e. HTTP or HTTPS
|
// i.e. HTTP or HTTPS
|
||||||
// Other ports are not expected as of now.
|
// Other ports are not expected as of now.
|
||||||
Assert.fail();
|
Assert.fail();
|
|
@ -59,7 +59,7 @@ import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
|
||||||
import org.apache.hadoop.ozone.client.rest.OzoneException;
|
import org.apache.hadoop.ozone.client.rest.OzoneException;
|
||||||
import org.apache.hadoop.ozone.client.rest.RestClient;
|
import org.apache.hadoop.ozone.client.rest.RestClient;
|
||||||
import org.apache.hadoop.ozone.client.rpc.RpcClient;
|
import org.apache.hadoop.ozone.client.rpc.RpcClient;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
|
import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
|
||||||
import org.apache.hadoop.ozone.web.ozShell.Shell;
|
import org.apache.hadoop.ozone.web.ozShell.Shell;
|
||||||
import org.apache.hadoop.ozone.web.request.OzoneQuota;
|
import org.apache.hadoop.ozone.web.request.OzoneQuota;
|
||||||
import org.apache.hadoop.ozone.web.response.BucketInfo;
|
import org.apache.hadoop.ozone.web.response.BucketInfo;
|
||||||
|
@ -167,23 +167,23 @@ public class TestOzoneShell {
|
||||||
System.setOut(new PrintStream(out));
|
System.setOut(new PrintStream(out));
|
||||||
System.setErr(new PrintStream(err));
|
System.setErr(new PrintStream(err));
|
||||||
if(clientProtocol.equals(RestClient.class)) {
|
if(clientProtocol.equals(RestClient.class)) {
|
||||||
String hostName = cluster.getKeySpaceManager().getHttpServer()
|
String hostName = cluster.getOzoneManager().getHttpServer()
|
||||||
.getHttpAddress().getHostName();
|
.getHttpAddress().getHostName();
|
||||||
int port = cluster
|
int port = cluster
|
||||||
.getKeySpaceManager().getHttpServer().getHttpAddress().getPort();
|
.getOzoneManager().getHttpServer().getHttpAddress().getPort();
|
||||||
url = String.format("http://" + hostName + ":" + port);
|
url = String.format("http://" + hostName + ":" + port);
|
||||||
} else {
|
} else {
|
||||||
List<ServiceInfo> services = null;
|
List<ServiceInfo> services = null;
|
||||||
try {
|
try {
|
||||||
services = cluster.getKeySpaceManager().getServiceList();
|
services = cluster.getOzoneManager().getServiceList();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
LOG.error("Could not get service list from KSM");
|
LOG.error("Could not get service list from OM");
|
||||||
}
|
}
|
||||||
String hostName = services.stream().filter(
|
String hostName = services.stream().filter(
|
||||||
a -> a.getNodeType().equals(HddsProtos.NodeType.KSM))
|
a -> a.getNodeType().equals(HddsProtos.NodeType.OM))
|
||||||
.collect(Collectors.toList()).get(0).getHostname();
|
.collect(Collectors.toList()).get(0).getHostname();
|
||||||
|
|
||||||
String port = cluster.getKeySpaceManager().getRpcPort();
|
String port = cluster.getOzoneManager().getRpcPort();
|
||||||
url = String.format("o3://" + hostName + ":" + port);
|
url = String.format("o3://" + hostName + ":" + port);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacem
|
||||||
import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementCapacity;
|
import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementCapacity;
|
||||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
|
||||||
import org.apache.hadoop.ozone.scm.cli.SQLCLI;
|
import org.apache.hadoop.ozone.scm.cli.SQLCLI;
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
|
@ -113,7 +112,7 @@ public class TestContainerSQLCli {
|
||||||
cluster.waitForClusterToBeReady();
|
cluster.waitForClusterToBeReady();
|
||||||
datanodeIpAddress = cluster.getHddsDatanodes().get(0)
|
datanodeIpAddress = cluster.getHddsDatanodes().get(0)
|
||||||
.getDatanodeDetails().getIpAddress();
|
.getDatanodeDetails().getIpAddress();
|
||||||
cluster.getKeySpaceManager().stop();
|
cluster.getOzoneManager().stop();
|
||||||
cluster.getStorageContainerManager().stop();
|
cluster.getStorageContainerManager().stop();
|
||||||
|
|
||||||
nodeManager = cluster.getStorageContainerManager().getScmNodeManager();
|
nodeManager = cluster.getStorageContainerManager().getScmNodeManager();
|
||||||
|
|
|
@ -90,7 +90,7 @@ public class TestDistributedOzoneVolumes extends TestOzoneHelper {
|
||||||
@Test
|
@Test
|
||||||
public void testCreateVolumes() throws IOException {
|
public void testCreateVolumes() throws IOException {
|
||||||
super.testCreateVolumes(port);
|
super.testCreateVolumes(port);
|
||||||
Assert.assertEquals(0, cluster.getKeySpaceManager()
|
Assert.assertEquals(0, cluster.getOzoneManager()
|
||||||
.getMetrics().getNumVolumeCreateFails());
|
.getMetrics().getNumVolumeCreateFails());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -102,7 +102,7 @@ public class TestDistributedOzoneVolumes extends TestOzoneHelper {
|
||||||
@Test
|
@Test
|
||||||
public void testCreateVolumesWithQuota() throws IOException {
|
public void testCreateVolumesWithQuota() throws IOException {
|
||||||
super.testCreateVolumesWithQuota(port);
|
super.testCreateVolumesWithQuota(port);
|
||||||
Assert.assertEquals(0, cluster.getKeySpaceManager()
|
Assert.assertEquals(0, cluster.getOzoneManager()
|
||||||
.getMetrics().getNumVolumeCreateFails());
|
.getMetrics().getNumVolumeCreateFails());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -114,7 +114,7 @@ public class TestDistributedOzoneVolumes extends TestOzoneHelper {
|
||||||
@Test
|
@Test
|
||||||
public void testCreateVolumesWithInvalidQuota() throws IOException {
|
public void testCreateVolumesWithInvalidQuota() throws IOException {
|
||||||
super.testCreateVolumesWithInvalidQuota(port);
|
super.testCreateVolumesWithInvalidQuota(port);
|
||||||
Assert.assertEquals(0, cluster.getKeySpaceManager()
|
Assert.assertEquals(0, cluster.getOzoneManager()
|
||||||
.getMetrics().getNumVolumeCreateFails());
|
.getMetrics().getNumVolumeCreateFails());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -128,7 +128,7 @@ public class TestDistributedOzoneVolumes extends TestOzoneHelper {
|
||||||
@Test
|
@Test
|
||||||
public void testCreateVolumesWithInvalidUser() throws IOException {
|
public void testCreateVolumesWithInvalidUser() throws IOException {
|
||||||
super.testCreateVolumesWithInvalidUser(port);
|
super.testCreateVolumesWithInvalidUser(port);
|
||||||
Assert.assertEquals(0, cluster.getKeySpaceManager()
|
Assert.assertEquals(0, cluster.getOzoneManager()
|
||||||
.getMetrics().getNumVolumeCreateFails());
|
.getMetrics().getNumVolumeCreateFails());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -143,7 +143,7 @@ public class TestDistributedOzoneVolumes extends TestOzoneHelper {
|
||||||
@Test
|
@Test
|
||||||
public void testCreateVolumesWithOutAdminRights() throws IOException {
|
public void testCreateVolumesWithOutAdminRights() throws IOException {
|
||||||
super.testCreateVolumesWithOutAdminRights(port);
|
super.testCreateVolumesWithOutAdminRights(port);
|
||||||
Assert.assertEquals(0, cluster.getKeySpaceManager()
|
Assert.assertEquals(0, cluster.getOzoneManager()
|
||||||
.getMetrics().getNumVolumeCreateFails());
|
.getMetrics().getNumVolumeCreateFails());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -155,7 +155,7 @@ public class TestDistributedOzoneVolumes extends TestOzoneHelper {
|
||||||
@Test
|
@Test
|
||||||
public void testCreateVolumesInLoop() throws IOException {
|
public void testCreateVolumesInLoop() throws IOException {
|
||||||
super.testCreateVolumesInLoop(port);
|
super.testCreateVolumesInLoop(port);
|
||||||
Assert.assertEquals(0, cluster.getKeySpaceManager()
|
Assert.assertEquals(0, cluster.getOzoneManager()
|
||||||
.getMetrics().getNumVolumeCreateFails());
|
.getMetrics().getNumVolumeCreateFails());
|
||||||
}
|
}
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -48,13 +48,13 @@ import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
|
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
|
||||||
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
|
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
|
||||||
import org.apache.hadoop.ozone.ksm.KeySpaceManager;
|
import org.apache.hadoop.ozone.om.OzoneManager;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
||||||
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
||||||
.Status;
|
.Status;
|
||||||
import org.apache.hadoop.ozone.client.rest.OzoneException;
|
import org.apache.hadoop.ozone.client.rest.OzoneException;
|
||||||
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
|
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
|
||||||
|
@ -644,15 +644,15 @@ public class TestKeys {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private int countKsmKeys(KeySpaceManager ksm) throws IOException {
|
private int countOmKeys(OzoneManager om) throws IOException {
|
||||||
int totalCount = 0;
|
int totalCount = 0;
|
||||||
List<KsmVolumeArgs> volumes =
|
List<OmVolumeArgs> volumes =
|
||||||
ksm.listAllVolumes(null, null, Integer.MAX_VALUE);
|
om.listAllVolumes(null, null, Integer.MAX_VALUE);
|
||||||
for (KsmVolumeArgs volume : volumes) {
|
for (OmVolumeArgs volume : volumes) {
|
||||||
List<KsmBucketInfo> buckets =
|
List<OmBucketInfo> buckets =
|
||||||
ksm.listBuckets(volume.getVolume(), null, null, Integer.MAX_VALUE);
|
om.listBuckets(volume.getVolume(), null, null, Integer.MAX_VALUE);
|
||||||
for (KsmBucketInfo bucket : buckets) {
|
for (OmBucketInfo bucket : buckets) {
|
||||||
List<KsmKeyInfo> keys = ksm.listKeys(bucket.getVolumeName(),
|
List<OmKeyInfo> keys = om.listKeys(bucket.getVolumeName(),
|
||||||
bucket.getBucketName(), null, null, Integer.MAX_VALUE);
|
bucket.getBucketName(), null, null, Integer.MAX_VALUE);
|
||||||
totalCount += keys.size();
|
totalCount += keys.size();
|
||||||
}
|
}
|
||||||
|
@ -662,10 +662,10 @@ public class TestKeys {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testDeleteKey() throws Exception {
|
public void testDeleteKey() throws Exception {
|
||||||
KeySpaceManager ksm = ozoneCluster.getKeySpaceManager();
|
OzoneManager ozoneManager = ozoneCluster.getOzoneManager();
|
||||||
// To avoid interference from other test cases,
|
// To avoid interference from other test cases,
|
||||||
// we collect number of existing keys at the beginning
|
// we collect number of existing keys at the beginning
|
||||||
int numOfExistedKeys = countKsmKeys(ksm);
|
int numOfExistedKeys = countOmKeys(ozoneManager);
|
||||||
|
|
||||||
// Keep tracking bucket keys info while creating them
|
// Keep tracking bucket keys info while creating them
|
||||||
PutHelper helper = new PutHelper(client, path);
|
PutHelper helper = new PutHelper(client, path);
|
||||||
|
@ -689,15 +689,15 @@ public class TestKeys {
|
||||||
// count the total number of created keys.
|
// count the total number of created keys.
|
||||||
Set<Pair<String, String>> buckets = bucketKeys.getAllBuckets();
|
Set<Pair<String, String>> buckets = bucketKeys.getAllBuckets();
|
||||||
for (Pair<String, String> buk : buckets) {
|
for (Pair<String, String> buk : buckets) {
|
||||||
List<KsmKeyInfo> createdKeys =
|
List<OmKeyInfo> createdKeys =
|
||||||
ksm.listKeys(buk.getKey(), buk.getValue(), null, null, 20);
|
ozoneManager.listKeys(buk.getKey(), buk.getValue(), null, null, 20);
|
||||||
|
|
||||||
// Memorize chunks that has been created,
|
// Memorize chunks that has been created,
|
||||||
// so we can verify actual deletions at DN side later.
|
// so we can verify actual deletions at DN side later.
|
||||||
for (KsmKeyInfo keyInfo : createdKeys) {
|
for (OmKeyInfo keyInfo : createdKeys) {
|
||||||
List<KsmKeyLocationInfo> locations =
|
List<OmKeyLocationInfo> locations =
|
||||||
keyInfo.getLatestVersionLocations().getLocationList();
|
keyInfo.getLatestVersionLocations().getLocationList();
|
||||||
for (KsmKeyLocationInfo location : locations) {
|
for (OmKeyLocationInfo location : locations) {
|
||||||
KeyData keyData = new KeyData(location.getBlockID());
|
KeyData keyData = new KeyData(location.getBlockID());
|
||||||
KeyData blockInfo = cm.getContainerManager()
|
KeyData blockInfo = cm.getContainerManager()
|
||||||
.getKeyManager().getKey(keyData);
|
.getKeyManager().getKey(keyData);
|
||||||
|
@ -721,9 +721,9 @@ public class TestKeys {
|
||||||
// Ensure all keys are created.
|
// Ensure all keys are created.
|
||||||
Assert.assertEquals(20, numOfCreatedKeys);
|
Assert.assertEquals(20, numOfCreatedKeys);
|
||||||
|
|
||||||
// Ensure all keys are visible from KSM.
|
// Ensure all keys are visible from OM.
|
||||||
// Total number should be numOfCreated + numOfExisted
|
// Total number should be numOfCreated + numOfExisted
|
||||||
Assert.assertEquals(20 + numOfExistedKeys, countKsmKeys(ksm));
|
Assert.assertEquals(20 + numOfExistedKeys, countOmKeys(ozoneManager));
|
||||||
|
|
||||||
// Delete 10 keys
|
// Delete 10 keys
|
||||||
int delCount = 20;
|
int delCount = 20;
|
||||||
|
@ -732,21 +732,21 @@ public class TestKeys {
|
||||||
List<String> bks = bucketKeys.getBucketKeys(bucketInfo.getValue());
|
List<String> bks = bucketKeys.getBucketKeys(bucketInfo.getValue());
|
||||||
for (String keyName : bks) {
|
for (String keyName : bks) {
|
||||||
if (delCount > 0) {
|
if (delCount > 0) {
|
||||||
KsmKeyArgs arg =
|
OmKeyArgs arg =
|
||||||
new KsmKeyArgs.Builder().setVolumeName(bucketInfo.getKey())
|
new OmKeyArgs.Builder().setVolumeName(bucketInfo.getKey())
|
||||||
.setBucketName(bucketInfo.getValue()).setKeyName(keyName)
|
.setBucketName(bucketInfo.getValue()).setKeyName(keyName)
|
||||||
.build();
|
.build();
|
||||||
ksm.deleteKey(arg);
|
ozoneManager.deleteKey(arg);
|
||||||
delCount--;
|
delCount--;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// It should be pretty quick that keys are removed from KSM namespace,
|
// It should be pretty quick that keys are removed from OM namespace,
|
||||||
// because actual deletion happens in async mode.
|
// because actual deletion happens in async mode.
|
||||||
GenericTestUtils.waitFor(() -> {
|
GenericTestUtils.waitFor(() -> {
|
||||||
try {
|
try {
|
||||||
int num = countKsmKeys(ksm);
|
int num = countOmKeys(ozoneManager);
|
||||||
return num == (numOfExistedKeys);
|
return num == (numOfExistedKeys);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
return false;
|
return false;
|
||||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs.server.datanode;
|
||||||
|
|
||||||
import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForBlockClients;
|
import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForBlockClients;
|
||||||
import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
|
import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
|
||||||
import static org.apache.hadoop.ozone.KsmUtils.getKsmAddress;
|
import static org.apache.hadoop.ozone.OmUtils.getOmAddress;
|
||||||
import static org.apache.hadoop.ozone.OzoneConfigKeys.*;
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.*;
|
||||||
import static com.sun.jersey.api.core.ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS;
|
import static com.sun.jersey.api.core.ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS;
|
||||||
import static com.sun.jersey.api.core.ResourceConfig.FEATURE_TRACE;
|
import static com.sun.jersey.api.core.ResourceConfig.FEATURE_TRACE;
|
||||||
|
@ -34,9 +34,8 @@ import com.sun.jersey.api.container.ContainerFactory;
|
||||||
import com.sun.jersey.api.core.ApplicationAdapter;
|
import com.sun.jersey.api.core.ApplicationAdapter;
|
||||||
|
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
import org.apache.hadoop.ozone.ksm.protocolPB
|
import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolClientSideTranslatorPB;
|
||||||
.KeySpaceManagerProtocolClientSideTranslatorPB;
|
import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB;
|
||||||
import org.apache.hadoop.ozone.ksm.protocolPB.KeySpaceManagerProtocolPB;
|
|
||||||
import org.apache.hadoop.ozone.OzoneConsts;
|
import org.apache.hadoop.ozone.OzoneConsts;
|
||||||
import org.apache.hadoop.ozone.web.ObjectStoreApplication;
|
import org.apache.hadoop.ozone.web.ObjectStoreApplication;
|
||||||
import org.apache.hadoop.ozone.web.handlers.ServiceFilter;
|
import org.apache.hadoop.ozone.web.handlers.ServiceFilter;
|
||||||
|
@ -72,8 +71,8 @@ public final class ObjectStoreHandler implements Closeable {
|
||||||
LoggerFactory.getLogger(ObjectStoreHandler.class);
|
LoggerFactory.getLogger(ObjectStoreHandler.class);
|
||||||
|
|
||||||
private final ObjectStoreJerseyContainer objectStoreJerseyContainer;
|
private final ObjectStoreJerseyContainer objectStoreJerseyContainer;
|
||||||
private final KeySpaceManagerProtocolClientSideTranslatorPB
|
private final OzoneManagerProtocolClientSideTranslatorPB
|
||||||
keySpaceManagerClient;
|
ozoneManagerClient;
|
||||||
private final StorageContainerLocationProtocolClientSideTranslatorPB
|
private final StorageContainerLocationProtocolClientSideTranslatorPB
|
||||||
storageContainerLocationClient;
|
storageContainerLocationClient;
|
||||||
private final ScmBlockLocationProtocolClientSideTranslatorPB
|
private final ScmBlockLocationProtocolClientSideTranslatorPB
|
||||||
|
@ -119,28 +118,28 @@ public final class ObjectStoreHandler implements Closeable {
|
||||||
NetUtils.getDefaultSocketFactory(conf),
|
NetUtils.getDefaultSocketFactory(conf),
|
||||||
Client.getRpcTimeout(conf)));
|
Client.getRpcTimeout(conf)));
|
||||||
|
|
||||||
RPC.setProtocolEngine(conf, KeySpaceManagerProtocolPB.class,
|
RPC.setProtocolEngine(conf, OzoneManagerProtocolPB.class,
|
||||||
ProtobufRpcEngine.class);
|
ProtobufRpcEngine.class);
|
||||||
long ksmVersion =
|
long omVersion =
|
||||||
RPC.getProtocolVersion(KeySpaceManagerProtocolPB.class);
|
RPC.getProtocolVersion(OzoneManagerProtocolPB.class);
|
||||||
InetSocketAddress ksmAddress = getKsmAddress(conf);
|
InetSocketAddress omAddress = getOmAddress(conf);
|
||||||
this.keySpaceManagerClient =
|
this.ozoneManagerClient =
|
||||||
new KeySpaceManagerProtocolClientSideTranslatorPB(
|
new OzoneManagerProtocolClientSideTranslatorPB(
|
||||||
RPC.getProxy(KeySpaceManagerProtocolPB.class, ksmVersion,
|
RPC.getProxy(OzoneManagerProtocolPB.class, omVersion,
|
||||||
ksmAddress, UserGroupInformation.getCurrentUser(), conf,
|
omAddress, UserGroupInformation.getCurrentUser(), conf,
|
||||||
NetUtils.getDefaultSocketFactory(conf),
|
NetUtils.getDefaultSocketFactory(conf),
|
||||||
Client.getRpcTimeout(conf)));
|
Client.getRpcTimeout(conf)));
|
||||||
|
|
||||||
storageHandler = new DistributedStorageHandler(
|
storageHandler = new DistributedStorageHandler(
|
||||||
new OzoneConfiguration(conf),
|
new OzoneConfiguration(conf),
|
||||||
this.storageContainerLocationClient,
|
this.storageContainerLocationClient,
|
||||||
this.keySpaceManagerClient);
|
this.ozoneManagerClient);
|
||||||
} else {
|
} else {
|
||||||
if (OzoneConsts.OZONE_HANDLER_LOCAL.equalsIgnoreCase(shType)) {
|
if (OzoneConsts.OZONE_HANDLER_LOCAL.equalsIgnoreCase(shType)) {
|
||||||
storageHandler = new LocalStorageHandler(conf);
|
storageHandler = new LocalStorageHandler(conf);
|
||||||
this.storageContainerLocationClient = null;
|
this.storageContainerLocationClient = null;
|
||||||
this.scmBlockLocationClient = null;
|
this.scmBlockLocationClient = null;
|
||||||
this.keySpaceManagerClient = null;
|
this.ozoneManagerClient = null;
|
||||||
} else {
|
} else {
|
||||||
throw new IllegalArgumentException(
|
throw new IllegalArgumentException(
|
||||||
String.format("Unrecognized value for %s: %s,"
|
String.format("Unrecognized value for %s: %s,"
|
||||||
|
@ -186,6 +185,6 @@ public final class ObjectStoreHandler implements Closeable {
|
||||||
storageHandler.close();
|
storageHandler.close();
|
||||||
IOUtils.cleanupWithLogger(LOG, storageContainerLocationClient);
|
IOUtils.cleanupWithLogger(LOG, storageContainerLocationClient);
|
||||||
IOUtils.cleanupWithLogger(LOG, scmBlockLocationClient);
|
IOUtils.cleanupWithLogger(LOG, scmBlockLocationClient);
|
||||||
IOUtils.cleanupWithLogger(LOG, keySpaceManagerClient);
|
IOUtils.cleanupWithLogger(LOG, ozoneManagerClient);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,7 +21,7 @@ package org.apache.hadoop.ozone.web.handlers;
|
||||||
import org.apache.commons.codec.binary.Base64;
|
import org.apache.commons.codec.binary.Base64;
|
||||||
|
|
||||||
import org.apache.hadoop.ozone.OzoneRestUtils;
|
import org.apache.hadoop.ozone.OzoneRestUtils;
|
||||||
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos;
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
||||||
import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
|
import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
|
||||||
import org.apache.hadoop.ozone.client.rest.OzoneException;
|
import org.apache.hadoop.ozone.client.rest.OzoneException;
|
||||||
import org.apache.hadoop.ozone.client.rest.headers.Header;
|
import org.apache.hadoop.ozone.client.rest.headers.Header;
|
||||||
|
@ -102,7 +102,7 @@ public abstract class KeyProcessTemplate {
|
||||||
LOG.error("IOException:", fsExp);
|
LOG.error("IOException:", fsExp);
|
||||||
// Map KEY_NOT_FOUND to INVALID_KEY
|
// Map KEY_NOT_FOUND to INVALID_KEY
|
||||||
if (fsExp.getMessage().endsWith(
|
if (fsExp.getMessage().endsWith(
|
||||||
KeySpaceManagerProtocolProtos.Status.KEY_NOT_FOUND.name())) {
|
OzoneManagerProtocolProtos.Status.KEY_NOT_FOUND.name())) {
|
||||||
throw ErrorTable.newError(ErrorTable.INVALID_KEY, userArgs, fsExp);
|
throw ErrorTable.newError(ErrorTable.INVALID_KEY, userArgs, fsExp);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -30,7 +30,7 @@ import java.nio.file.NoSuchFileException;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.ozone.OzoneRestUtils;
|
import org.apache.hadoop.ozone.OzoneRestUtils;
|
||||||
import org.apache.hadoop.ozone.client.rest.OzoneException;
|
import org.apache.hadoop.ozone.client.rest.OzoneException;
|
||||||
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos;
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
||||||
import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
|
import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
|
||||||
import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
|
import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
|
||||||
import org.apache.hadoop.ozone.web.interfaces.UserAuth;
|
import org.apache.hadoop.ozone.web.interfaces.UserAuth;
|
||||||
|
@ -135,7 +135,7 @@ public abstract class VolumeProcessTemplate {
|
||||||
OzoneException exp = null;
|
OzoneException exp = null;
|
||||||
|
|
||||||
if ((fsExp != null && fsExp.getMessage().endsWith(
|
if ((fsExp != null && fsExp.getMessage().endsWith(
|
||||||
KeySpaceManagerProtocolProtos.Status.VOLUME_ALREADY_EXISTS.name()))
|
OzoneManagerProtocolProtos.Status.VOLUME_ALREADY_EXISTS.name()))
|
||||||
|| fsExp instanceof FileAlreadyExistsException) {
|
|| fsExp instanceof FileAlreadyExistsException) {
|
||||||
exp = ErrorTable
|
exp = ErrorTable
|
||||||
.newError(ErrorTable.VOLUME_ALREADY_EXISTS, reqID, volume, hostName);
|
.newError(ErrorTable.VOLUME_ALREADY_EXISTS, reqID, volume, hostName);
|
||||||
|
|
|
@ -22,14 +22,13 @@ import com.google.common.base.Strings;
|
||||||
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
|
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
import org.apache.hadoop.ozone.client.io.LengthInputStream;
|
import org.apache.hadoop.ozone.client.io.LengthInputStream;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession;
|
import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
|
||||||
import org.apache.hadoop.ozone.ksm.protocolPB
|
import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolClientSideTranslatorPB;
|
||||||
.KeySpaceManagerProtocolClientSideTranslatorPB;
|
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.ozone.OzoneConsts;
|
import org.apache.hadoop.ozone.OzoneConsts;
|
||||||
|
@ -37,9 +36,9 @@ import org.apache.hadoop.ozone.OzoneConsts.Versioning;
|
||||||
import org.apache.hadoop.ozone.client.io.ChunkGroupInputStream;
|
import org.apache.hadoop.ozone.client.io.ChunkGroupInputStream;
|
||||||
import org.apache.hadoop.ozone.client.io.ChunkGroupOutputStream;
|
import org.apache.hadoop.ozone.client.io.ChunkGroupOutputStream;
|
||||||
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
||||||
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos;
|
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
||||||
import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
|
import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
|
||||||
import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
|
import org.apache.hadoop.ozone.om.OMConfigKeys;
|
||||||
import org.apache.hadoop.ozone.OzoneAcl;
|
import org.apache.hadoop.ozone.OzoneAcl;
|
||||||
import org.apache.hadoop.ozone.web.request.OzoneQuota;
|
import org.apache.hadoop.ozone.web.request.OzoneQuota;
|
||||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||||
|
@ -77,8 +76,8 @@ public final class DistributedStorageHandler implements StorageHandler {
|
||||||
|
|
||||||
private final StorageContainerLocationProtocolClientSideTranslatorPB
|
private final StorageContainerLocationProtocolClientSideTranslatorPB
|
||||||
storageContainerLocationClient;
|
storageContainerLocationClient;
|
||||||
private final KeySpaceManagerProtocolClientSideTranslatorPB
|
private final OzoneManagerProtocolClientSideTranslatorPB
|
||||||
keySpaceManagerClient;
|
ozoneManagerClient;
|
||||||
private final XceiverClientManager xceiverClientManager;
|
private final XceiverClientManager xceiverClientManager;
|
||||||
private final OzoneAcl.OzoneACLRights userRights;
|
private final OzoneAcl.OzoneACLRights userRights;
|
||||||
private final OzoneAcl.OzoneACLRights groupRights;
|
private final OzoneAcl.OzoneACLRights groupRights;
|
||||||
|
@ -92,14 +91,14 @@ public final class DistributedStorageHandler implements StorageHandler {
|
||||||
*
|
*
|
||||||
* @param conf configuration
|
* @param conf configuration
|
||||||
* @param storageContainerLocation StorageContainerLocationProtocol proxy
|
* @param storageContainerLocation StorageContainerLocationProtocol proxy
|
||||||
* @param keySpaceManagerClient KeySpaceManager proxy
|
* @param ozoneManagerClient OzoneManager proxy
|
||||||
*/
|
*/
|
||||||
public DistributedStorageHandler(OzoneConfiguration conf,
|
public DistributedStorageHandler(OzoneConfiguration conf,
|
||||||
StorageContainerLocationProtocolClientSideTranslatorPB
|
StorageContainerLocationProtocolClientSideTranslatorPB
|
||||||
storageContainerLocation,
|
storageContainerLocation,
|
||||||
KeySpaceManagerProtocolClientSideTranslatorPB
|
OzoneManagerProtocolClientSideTranslatorPB
|
||||||
keySpaceManagerClient) {
|
ozoneManagerClient) {
|
||||||
this.keySpaceManagerClient = keySpaceManagerClient;
|
this.ozoneManagerClient = ozoneManagerClient;
|
||||||
this.storageContainerLocationClient = storageContainerLocation;
|
this.storageContainerLocationClient = storageContainerLocation;
|
||||||
this.xceiverClientManager = new XceiverClientManager(conf);
|
this.xceiverClientManager = new XceiverClientManager(conf);
|
||||||
this.useRatis = conf.getBoolean(
|
this.useRatis = conf.getBoolean(
|
||||||
|
@ -116,10 +115,10 @@ public final class DistributedStorageHandler implements StorageHandler {
|
||||||
|
|
||||||
chunkSize = conf.getInt(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY,
|
chunkSize = conf.getInt(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY,
|
||||||
ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT);
|
ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT);
|
||||||
userRights = conf.getEnum(KSMConfigKeys.OZONE_KSM_USER_RIGHTS,
|
userRights = conf.getEnum(OMConfigKeys.OZONE_OM_USER_RIGHTS,
|
||||||
KSMConfigKeys.OZONE_KSM_USER_RIGHTS_DEFAULT);
|
OMConfigKeys.OZONE_OM_USER_RIGHTS_DEFAULT);
|
||||||
groupRights = conf.getEnum(KSMConfigKeys.OZONE_KSM_GROUP_RIGHTS,
|
groupRights = conf.getEnum(OMConfigKeys.OZONE_OM_GROUP_RIGHTS,
|
||||||
KSMConfigKeys.OZONE_KSM_GROUP_RIGHTS_DEFAULT);
|
OMConfigKeys.OZONE_OM_GROUP_RIGHTS_DEFAULT);
|
||||||
if(chunkSize > ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE) {
|
if(chunkSize > ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE) {
|
||||||
LOG.warn("The chunk size ({}) is not allowed to be more than"
|
LOG.warn("The chunk size ({}) is not allowed to be more than"
|
||||||
+ " the maximum size ({}),"
|
+ " the maximum size ({}),"
|
||||||
|
@ -136,26 +135,26 @@ public final class DistributedStorageHandler implements StorageHandler {
|
||||||
OzoneAcl userAcl =
|
OzoneAcl userAcl =
|
||||||
new OzoneAcl(OzoneAcl.OzoneACLType.USER,
|
new OzoneAcl(OzoneAcl.OzoneACLType.USER,
|
||||||
args.getUserName(), userRights);
|
args.getUserName(), userRights);
|
||||||
KsmVolumeArgs.Builder builder = KsmVolumeArgs.newBuilder();
|
OmVolumeArgs.Builder builder = OmVolumeArgs.newBuilder();
|
||||||
builder.setAdminName(args.getAdminName())
|
builder.setAdminName(args.getAdminName())
|
||||||
.setOwnerName(args.getUserName())
|
.setOwnerName(args.getUserName())
|
||||||
.setVolume(args.getVolumeName())
|
.setVolume(args.getVolumeName())
|
||||||
.setQuotaInBytes(quota)
|
.setQuotaInBytes(quota)
|
||||||
.addOzoneAcls(KSMPBHelper.convertOzoneAcl(userAcl));
|
.addOzoneAcls(OMPBHelper.convertOzoneAcl(userAcl));
|
||||||
if (args.getGroups() != null) {
|
if (args.getGroups() != null) {
|
||||||
for (String group : args.getGroups()) {
|
for (String group : args.getGroups()) {
|
||||||
OzoneAcl groupAcl =
|
OzoneAcl groupAcl =
|
||||||
new OzoneAcl(OzoneAcl.OzoneACLType.GROUP, group, groupRights);
|
new OzoneAcl(OzoneAcl.OzoneACLType.GROUP, group, groupRights);
|
||||||
builder.addOzoneAcls(KSMPBHelper.convertOzoneAcl(groupAcl));
|
builder.addOzoneAcls(OMPBHelper.convertOzoneAcl(groupAcl));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
keySpaceManagerClient.createVolume(builder.build());
|
ozoneManagerClient.createVolume(builder.build());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setVolumeOwner(VolumeArgs args) throws
|
public void setVolumeOwner(VolumeArgs args) throws
|
||||||
IOException, OzoneException {
|
IOException, OzoneException {
|
||||||
keySpaceManagerClient.setOwner(args.getVolumeName(), args.getUserName());
|
ozoneManagerClient.setOwner(args.getVolumeName(), args.getUserName());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -163,14 +162,14 @@ public final class DistributedStorageHandler implements StorageHandler {
|
||||||
throws IOException, OzoneException {
|
throws IOException, OzoneException {
|
||||||
long quota = remove ? OzoneConsts.MAX_QUOTA_IN_BYTES :
|
long quota = remove ? OzoneConsts.MAX_QUOTA_IN_BYTES :
|
||||||
args.getQuota().sizeInBytes();
|
args.getQuota().sizeInBytes();
|
||||||
keySpaceManagerClient.setQuota(args.getVolumeName(), quota);
|
ozoneManagerClient.setQuota(args.getVolumeName(), quota);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean checkVolumeAccess(String volume, OzoneAcl acl)
|
public boolean checkVolumeAccess(String volume, OzoneAcl acl)
|
||||||
throws IOException, OzoneException {
|
throws IOException, OzoneException {
|
||||||
return keySpaceManagerClient
|
return ozoneManagerClient
|
||||||
.checkVolumeAccess(volume, KSMPBHelper.convertOzoneAcl(acl));
|
.checkVolumeAccess(volume, OMPBHelper.convertOzoneAcl(acl));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -185,9 +184,9 @@ public final class DistributedStorageHandler implements StorageHandler {
|
||||||
OzoneConsts.MAX_LISTVOLUMES_SIZE, maxNumOfKeys));
|
OzoneConsts.MAX_LISTVOLUMES_SIZE, maxNumOfKeys));
|
||||||
}
|
}
|
||||||
|
|
||||||
List<KsmVolumeArgs> listResult;
|
List<OmVolumeArgs> listResult;
|
||||||
if (args.isRootScan()) {
|
if (args.isRootScan()) {
|
||||||
listResult = keySpaceManagerClient.listAllVolumes(args.getPrefix(),
|
listResult = ozoneManagerClient.listAllVolumes(args.getPrefix(),
|
||||||
args.getPrevKey(), args.getMaxKeys());
|
args.getPrevKey(), args.getMaxKeys());
|
||||||
} else {
|
} else {
|
||||||
UserArgs userArgs = args.getArgs();
|
UserArgs userArgs = args.getArgs();
|
||||||
|
@ -195,16 +194,16 @@ public final class DistributedStorageHandler implements StorageHandler {
|
||||||
throw new IllegalArgumentException("Illegal argument,"
|
throw new IllegalArgumentException("Illegal argument,"
|
||||||
+ " missing user argument.");
|
+ " missing user argument.");
|
||||||
}
|
}
|
||||||
listResult = keySpaceManagerClient.listVolumeByUser(
|
listResult = ozoneManagerClient.listVolumeByUser(
|
||||||
args.getArgs().getUserName(), args.getPrefix(), args.getPrevKey(),
|
args.getArgs().getUserName(), args.getPrefix(), args.getPrevKey(),
|
||||||
args.getMaxKeys());
|
args.getMaxKeys());
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO Add missing fields createdBy, bucketCount and bytesUsed
|
// TODO Add missing fields createdBy, bucketCount and bytesUsed
|
||||||
ListVolumes result = new ListVolumes();
|
ListVolumes result = new ListVolumes();
|
||||||
for (KsmVolumeArgs volumeArgs : listResult) {
|
for (OmVolumeArgs volumeArgs : listResult) {
|
||||||
VolumeInfo info = new VolumeInfo();
|
VolumeInfo info = new VolumeInfo();
|
||||||
KeySpaceManagerProtocolProtos.VolumeInfo
|
OzoneManagerProtocolProtos.VolumeInfo
|
||||||
infoProto = volumeArgs.getProtobuf();
|
infoProto = volumeArgs.getProtobuf();
|
||||||
info.setOwner(new VolumeOwner(infoProto.getOwnerName()));
|
info.setOwner(new VolumeOwner(infoProto.getOwnerName()));
|
||||||
info.setQuota(OzoneQuota.getOzoneQuota(infoProto.getQuotaInBytes()));
|
info.setQuota(OzoneQuota.getOzoneQuota(infoProto.getQuotaInBytes()));
|
||||||
|
@ -220,14 +219,14 @@ public final class DistributedStorageHandler implements StorageHandler {
|
||||||
@Override
|
@Override
|
||||||
public void deleteVolume(VolumeArgs args)
|
public void deleteVolume(VolumeArgs args)
|
||||||
throws IOException, OzoneException {
|
throws IOException, OzoneException {
|
||||||
keySpaceManagerClient.deleteVolume(args.getVolumeName());
|
ozoneManagerClient.deleteVolume(args.getVolumeName());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public VolumeInfo getVolumeInfo(VolumeArgs args)
|
public VolumeInfo getVolumeInfo(VolumeArgs args)
|
||||||
throws IOException, OzoneException {
|
throws IOException, OzoneException {
|
||||||
KsmVolumeArgs volumeArgs =
|
OmVolumeArgs volumeArgs =
|
||||||
keySpaceManagerClient.getVolumeInfo(args.getVolumeName());
|
ozoneManagerClient.getVolumeInfo(args.getVolumeName());
|
||||||
//TODO: add support for createdOn and other fields in getVolumeInfo
|
//TODO: add support for createdOn and other fields in getVolumeInfo
|
||||||
VolumeInfo volInfo =
|
VolumeInfo volInfo =
|
||||||
new VolumeInfo(volumeArgs.getVolume(), null,
|
new VolumeInfo(volumeArgs.getVolume(), null,
|
||||||
|
@ -242,7 +241,7 @@ public final class DistributedStorageHandler implements StorageHandler {
|
||||||
@Override
|
@Override
|
||||||
public void createBucket(final BucketArgs args)
|
public void createBucket(final BucketArgs args)
|
||||||
throws IOException, OzoneException {
|
throws IOException, OzoneException {
|
||||||
KsmBucketInfo.Builder builder = KsmBucketInfo.newBuilder();
|
OmBucketInfo.Builder builder = OmBucketInfo.newBuilder();
|
||||||
builder.setVolumeName(args.getVolumeName())
|
builder.setVolumeName(args.getVolumeName())
|
||||||
.setBucketName(args.getBucketName());
|
.setBucketName(args.getBucketName());
|
||||||
if(args.getAddAcls() != null) {
|
if(args.getAddAcls() != null) {
|
||||||
|
@ -255,7 +254,7 @@ public final class DistributedStorageHandler implements StorageHandler {
|
||||||
builder.setIsVersionEnabled(getBucketVersioningProtobuf(
|
builder.setIsVersionEnabled(getBucketVersioningProtobuf(
|
||||||
args.getVersioning()));
|
args.getVersioning()));
|
||||||
}
|
}
|
||||||
keySpaceManagerClient.createBucket(builder.build());
|
ozoneManagerClient.createBucket(builder.build());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -285,7 +284,7 @@ public final class DistributedStorageHandler implements StorageHandler {
|
||||||
List<OzoneAcl> removeAcls = args.getRemoveAcls();
|
List<OzoneAcl> removeAcls = args.getRemoveAcls();
|
||||||
List<OzoneAcl> addAcls = args.getAddAcls();
|
List<OzoneAcl> addAcls = args.getAddAcls();
|
||||||
if(removeAcls != null || addAcls != null) {
|
if(removeAcls != null || addAcls != null) {
|
||||||
KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
|
OmBucketArgs.Builder builder = OmBucketArgs.newBuilder();
|
||||||
builder.setVolumeName(args.getVolumeName())
|
builder.setVolumeName(args.getVolumeName())
|
||||||
.setBucketName(args.getBucketName());
|
.setBucketName(args.getBucketName());
|
||||||
if(removeAcls != null && !removeAcls.isEmpty()) {
|
if(removeAcls != null && !removeAcls.isEmpty()) {
|
||||||
|
@ -294,35 +293,35 @@ public final class DistributedStorageHandler implements StorageHandler {
|
||||||
if(addAcls != null && !addAcls.isEmpty()) {
|
if(addAcls != null && !addAcls.isEmpty()) {
|
||||||
builder.setAddAcls(args.getAddAcls());
|
builder.setAddAcls(args.getAddAcls());
|
||||||
}
|
}
|
||||||
keySpaceManagerClient.setBucketProperty(builder.build());
|
ozoneManagerClient.setBucketProperty(builder.build());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setBucketVersioning(BucketArgs args)
|
public void setBucketVersioning(BucketArgs args)
|
||||||
throws IOException, OzoneException {
|
throws IOException, OzoneException {
|
||||||
KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
|
OmBucketArgs.Builder builder = OmBucketArgs.newBuilder();
|
||||||
builder.setVolumeName(args.getVolumeName())
|
builder.setVolumeName(args.getVolumeName())
|
||||||
.setBucketName(args.getBucketName())
|
.setBucketName(args.getBucketName())
|
||||||
.setIsVersionEnabled(getBucketVersioningProtobuf(
|
.setIsVersionEnabled(getBucketVersioningProtobuf(
|
||||||
args.getVersioning()));
|
args.getVersioning()));
|
||||||
keySpaceManagerClient.setBucketProperty(builder.build());
|
ozoneManagerClient.setBucketProperty(builder.build());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setBucketStorageClass(BucketArgs args)
|
public void setBucketStorageClass(BucketArgs args)
|
||||||
throws IOException, OzoneException {
|
throws IOException, OzoneException {
|
||||||
KsmBucketArgs.Builder builder = KsmBucketArgs.newBuilder();
|
OmBucketArgs.Builder builder = OmBucketArgs.newBuilder();
|
||||||
builder.setVolumeName(args.getVolumeName())
|
builder.setVolumeName(args.getVolumeName())
|
||||||
.setBucketName(args.getBucketName())
|
.setBucketName(args.getBucketName())
|
||||||
.setStorageType(args.getStorageType());
|
.setStorageType(args.getStorageType());
|
||||||
keySpaceManagerClient.setBucketProperty(builder.build());
|
ozoneManagerClient.setBucketProperty(builder.build());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void deleteBucket(BucketArgs args)
|
public void deleteBucket(BucketArgs args)
|
||||||
throws IOException, OzoneException {
|
throws IOException, OzoneException {
|
||||||
keySpaceManagerClient.deleteBucket(args.getVolumeName(),
|
ozoneManagerClient.deleteBucket(args.getVolumeName(),
|
||||||
args.getBucketName());
|
args.getBucketName());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -354,12 +353,12 @@ public final class DistributedStorageHandler implements StorageHandler {
|
||||||
OzoneConsts.MAX_LISTBUCKETS_SIZE, maxNumOfKeys));
|
OzoneConsts.MAX_LISTBUCKETS_SIZE, maxNumOfKeys));
|
||||||
}
|
}
|
||||||
|
|
||||||
List<KsmBucketInfo> buckets =
|
List<OmBucketInfo> buckets =
|
||||||
keySpaceManagerClient.listBuckets(va.getVolumeName(),
|
ozoneManagerClient.listBuckets(va.getVolumeName(),
|
||||||
args.getPrevKey(), args.getPrefix(), args.getMaxKeys());
|
args.getPrevKey(), args.getPrefix(), args.getMaxKeys());
|
||||||
|
|
||||||
// Convert the result for the web layer.
|
// Convert the result for the web layer.
|
||||||
for (KsmBucketInfo bucketInfo : buckets) {
|
for (OmBucketInfo bucketInfo : buckets) {
|
||||||
BucketInfo bk = new BucketInfo();
|
BucketInfo bk = new BucketInfo();
|
||||||
bk.setVolumeName(bucketInfo.getVolumeName());
|
bk.setVolumeName(bucketInfo.getVolumeName());
|
||||||
bk.setBucketName(bucketInfo.getBucketName());
|
bk.setBucketName(bucketInfo.getBucketName());
|
||||||
|
@ -382,26 +381,26 @@ public final class DistributedStorageHandler implements StorageHandler {
|
||||||
throws IOException {
|
throws IOException {
|
||||||
String volumeName = args.getVolumeName();
|
String volumeName = args.getVolumeName();
|
||||||
String bucketName = args.getBucketName();
|
String bucketName = args.getBucketName();
|
||||||
KsmBucketInfo ksmBucketInfo = keySpaceManagerClient.getBucketInfo(
|
OmBucketInfo omBucketInfo = ozoneManagerClient.getBucketInfo(
|
||||||
volumeName, bucketName);
|
volumeName, bucketName);
|
||||||
BucketInfo bucketInfo = new BucketInfo(ksmBucketInfo.getVolumeName(),
|
BucketInfo bucketInfo = new BucketInfo(omBucketInfo.getVolumeName(),
|
||||||
ksmBucketInfo.getBucketName());
|
omBucketInfo.getBucketName());
|
||||||
if(ksmBucketInfo.getIsVersionEnabled()) {
|
if(omBucketInfo.getIsVersionEnabled()) {
|
||||||
bucketInfo.setVersioning(Versioning.ENABLED);
|
bucketInfo.setVersioning(Versioning.ENABLED);
|
||||||
} else {
|
} else {
|
||||||
bucketInfo.setVersioning(Versioning.DISABLED);
|
bucketInfo.setVersioning(Versioning.DISABLED);
|
||||||
}
|
}
|
||||||
bucketInfo.setStorageType(ksmBucketInfo.getStorageType());
|
bucketInfo.setStorageType(omBucketInfo.getStorageType());
|
||||||
bucketInfo.setAcls(ksmBucketInfo.getAcls());
|
bucketInfo.setAcls(omBucketInfo.getAcls());
|
||||||
bucketInfo.setCreatedOn(
|
bucketInfo.setCreatedOn(
|
||||||
HddsClientUtils.formatDateTime(ksmBucketInfo.getCreationTime()));
|
HddsClientUtils.formatDateTime(omBucketInfo.getCreationTime()));
|
||||||
return bucketInfo;
|
return bucketInfo;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public OutputStream newKeyWriter(KeyArgs args) throws IOException,
|
public OutputStream newKeyWriter(KeyArgs args) throws IOException,
|
||||||
OzoneException {
|
OzoneException {
|
||||||
KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
|
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
|
||||||
.setVolumeName(args.getVolumeName())
|
.setVolumeName(args.getVolumeName())
|
||||||
.setBucketName(args.getBucketName())
|
.setBucketName(args.getBucketName())
|
||||||
.setKeyName(args.getKeyName())
|
.setKeyName(args.getKeyName())
|
||||||
|
@ -409,14 +408,14 @@ public final class DistributedStorageHandler implements StorageHandler {
|
||||||
.setType(xceiverClientManager.getType())
|
.setType(xceiverClientManager.getType())
|
||||||
.setFactor(xceiverClientManager.getFactor())
|
.setFactor(xceiverClientManager.getFactor())
|
||||||
.build();
|
.build();
|
||||||
// contact KSM to allocate a block for key.
|
// contact OM to allocate a block for key.
|
||||||
OpenKeySession openKey = keySpaceManagerClient.openKey(keyArgs);
|
OpenKeySession openKey = ozoneManagerClient.openKey(keyArgs);
|
||||||
ChunkGroupOutputStream groupOutputStream =
|
ChunkGroupOutputStream groupOutputStream =
|
||||||
new ChunkGroupOutputStream.Builder()
|
new ChunkGroupOutputStream.Builder()
|
||||||
.setHandler(openKey)
|
.setHandler(openKey)
|
||||||
.setXceiverClientManager(xceiverClientManager)
|
.setXceiverClientManager(xceiverClientManager)
|
||||||
.setScmClient(storageContainerLocationClient)
|
.setScmClient(storageContainerLocationClient)
|
||||||
.setKsmClient(keySpaceManagerClient)
|
.setOmClient(ozoneManagerClient)
|
||||||
.setChunkSize(chunkSize)
|
.setChunkSize(chunkSize)
|
||||||
.setRequestID(args.getRequestID())
|
.setRequestID(args.getRequestID())
|
||||||
.setType(xceiverClientManager.getType())
|
.setType(xceiverClientManager.getType())
|
||||||
|
@ -437,56 +436,56 @@ public final class DistributedStorageHandler implements StorageHandler {
|
||||||
@Override
|
@Override
|
||||||
public LengthInputStream newKeyReader(KeyArgs args) throws IOException,
|
public LengthInputStream newKeyReader(KeyArgs args) throws IOException,
|
||||||
OzoneException {
|
OzoneException {
|
||||||
KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
|
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
|
||||||
.setVolumeName(args.getVolumeName())
|
.setVolumeName(args.getVolumeName())
|
||||||
.setBucketName(args.getBucketName())
|
.setBucketName(args.getBucketName())
|
||||||
.setKeyName(args.getKeyName())
|
.setKeyName(args.getKeyName())
|
||||||
.setDataSize(args.getSize())
|
.setDataSize(args.getSize())
|
||||||
.build();
|
.build();
|
||||||
KsmKeyInfo keyInfo = keySpaceManagerClient.lookupKey(keyArgs);
|
OmKeyInfo keyInfo = ozoneManagerClient.lookupKey(keyArgs);
|
||||||
return ChunkGroupInputStream.getFromKsmKeyInfo(
|
return ChunkGroupInputStream.getFromOmKeyInfo(
|
||||||
keyInfo, xceiverClientManager, storageContainerLocationClient,
|
keyInfo, xceiverClientManager, storageContainerLocationClient,
|
||||||
args.getRequestID());
|
args.getRequestID());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void deleteKey(KeyArgs args) throws IOException, OzoneException {
|
public void deleteKey(KeyArgs args) throws IOException, OzoneException {
|
||||||
KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
|
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
|
||||||
.setVolumeName(args.getVolumeName())
|
.setVolumeName(args.getVolumeName())
|
||||||
.setBucketName(args.getBucketName())
|
.setBucketName(args.getBucketName())
|
||||||
.setKeyName(args.getKeyName())
|
.setKeyName(args.getKeyName())
|
||||||
.build();
|
.build();
|
||||||
keySpaceManagerClient.deleteKey(keyArgs);
|
ozoneManagerClient.deleteKey(keyArgs);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void renameKey(KeyArgs args, String toKeyName)
|
public void renameKey(KeyArgs args, String toKeyName)
|
||||||
throws IOException, OzoneException {
|
throws IOException, OzoneException {
|
||||||
KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
|
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
|
||||||
.setVolumeName(args.getVolumeName())
|
.setVolumeName(args.getVolumeName())
|
||||||
.setBucketName(args.getBucketName())
|
.setBucketName(args.getBucketName())
|
||||||
.setKeyName(args.getKeyName())
|
.setKeyName(args.getKeyName())
|
||||||
.build();
|
.build();
|
||||||
keySpaceManagerClient.renameKey(keyArgs, toKeyName);
|
ozoneManagerClient.renameKey(keyArgs, toKeyName);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public KeyInfo getKeyInfo(KeyArgs args) throws IOException, OzoneException {
|
public KeyInfo getKeyInfo(KeyArgs args) throws IOException, OzoneException {
|
||||||
KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
|
OmKeyArgs keyArgs = new OmKeyArgs.Builder()
|
||||||
.setVolumeName(args.getVolumeName())
|
.setVolumeName(args.getVolumeName())
|
||||||
.setBucketName(args.getBucketName())
|
.setBucketName(args.getBucketName())
|
||||||
.setKeyName(args.getKeyName())
|
.setKeyName(args.getKeyName())
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
KsmKeyInfo ksmKeyInfo = keySpaceManagerClient.lookupKey(keyArgs);
|
OmKeyInfo omKeyInfo = ozoneManagerClient.lookupKey(keyArgs);
|
||||||
KeyInfo keyInfo = new KeyInfo();
|
KeyInfo keyInfo = new KeyInfo();
|
||||||
keyInfo.setVersion(0);
|
keyInfo.setVersion(0);
|
||||||
keyInfo.setKeyName(ksmKeyInfo.getKeyName());
|
keyInfo.setKeyName(omKeyInfo.getKeyName());
|
||||||
keyInfo.setSize(ksmKeyInfo.getDataSize());
|
keyInfo.setSize(omKeyInfo.getDataSize());
|
||||||
keyInfo.setCreatedOn(
|
keyInfo.setCreatedOn(
|
||||||
HddsClientUtils.formatDateTime(ksmKeyInfo.getCreationTime()));
|
HddsClientUtils.formatDateTime(omKeyInfo.getCreationTime()));
|
||||||
keyInfo.setModifiedOn(
|
keyInfo.setModifiedOn(
|
||||||
HddsClientUtils.formatDateTime(ksmKeyInfo.getModificationTime()));
|
HddsClientUtils.formatDateTime(omKeyInfo.getModificationTime()));
|
||||||
return keyInfo;
|
return keyInfo;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -515,13 +514,13 @@ public final class DistributedStorageHandler implements StorageHandler {
|
||||||
OzoneConsts.MAX_LISTKEYS_SIZE, maxNumOfKeys));
|
OzoneConsts.MAX_LISTKEYS_SIZE, maxNumOfKeys));
|
||||||
}
|
}
|
||||||
|
|
||||||
List<KsmKeyInfo> keys=
|
List<OmKeyInfo> keys=
|
||||||
keySpaceManagerClient.listKeys(bucketArgs.getVolumeName(),
|
ozoneManagerClient.listKeys(bucketArgs.getVolumeName(),
|
||||||
bucketArgs.getBucketName(),
|
bucketArgs.getBucketName(),
|
||||||
args.getPrevKey(), args.getPrefix(), args.getMaxKeys());
|
args.getPrevKey(), args.getPrefix(), args.getMaxKeys());
|
||||||
|
|
||||||
// Convert the result for the web layer.
|
// Convert the result for the web layer.
|
||||||
for (KsmKeyInfo info : keys) {
|
for (OmKeyInfo info : keys) {
|
||||||
KeyInfo tempInfo = new KeyInfo();
|
KeyInfo tempInfo = new KeyInfo();
|
||||||
tempInfo.setVersion(0);
|
tempInfo.setVersion(0);
|
||||||
tempInfo.setKeyName(info.getKeyName());
|
tempInfo.setKeyName(info.getKeyName());
|
||||||
|
@ -547,7 +546,7 @@ public final class DistributedStorageHandler implements StorageHandler {
|
||||||
@Override
|
@Override
|
||||||
public void close() {
|
public void close() {
|
||||||
IOUtils.cleanupWithLogger(LOG, xceiverClientManager);
|
IOUtils.cleanupWithLogger(LOG, xceiverClientManager);
|
||||||
IOUtils.cleanupWithLogger(LOG, keySpaceManagerClient);
|
IOUtils.cleanupWithLogger(LOG, ozoneManagerClient);
|
||||||
IOUtils.cleanupWithLogger(LOG, storageContainerLocationClient);
|
IOUtils.cleanupWithLogger(LOG, storageContainerLocationClient);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -14,10 +14,10 @@
|
||||||
* License for the specific language governing permissions and limitations under
|
* License for the specific language governing permissions and limitations under
|
||||||
* the License.
|
* the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm;
|
package org.apache.hadoop.ozone.om;
|
||||||
|
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -28,15 +28,15 @@ import java.util.List;
|
||||||
public interface BucketManager {
|
public interface BucketManager {
|
||||||
/**
|
/**
|
||||||
* Creates a bucket.
|
* Creates a bucket.
|
||||||
* @param bucketInfo - KsmBucketInfo for creating bucket.
|
* @param bucketInfo - OmBucketInfo for creating bucket.
|
||||||
*/
|
*/
|
||||||
void createBucket(KsmBucketInfo bucketInfo) throws IOException;
|
void createBucket(OmBucketInfo bucketInfo) throws IOException;
|
||||||
/**
|
/**
|
||||||
* Returns Bucket Information.
|
* Returns Bucket Information.
|
||||||
* @param volumeName - Name of the Volume.
|
* @param volumeName - Name of the Volume.
|
||||||
* @param bucketName - Name of the Bucket.
|
* @param bucketName - Name of the Bucket.
|
||||||
*/
|
*/
|
||||||
KsmBucketInfo getBucketInfo(String volumeName, String bucketName)
|
OmBucketInfo getBucketInfo(String volumeName, String bucketName)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -44,7 +44,7 @@ public interface BucketManager {
|
||||||
* @param args - BucketArgs.
|
* @param args - BucketArgs.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
void setBucketProperty(KsmBucketArgs args) throws IOException;
|
void setBucketProperty(OmBucketArgs args) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Deletes an existing empty bucket from volume.
|
* Deletes an existing empty bucket from volume.
|
||||||
|
@ -55,7 +55,7 @@ public interface BucketManager {
|
||||||
void deleteBucket(String volumeName, String bucketName) throws IOException;
|
void deleteBucket(String volumeName, String bucketName) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a list of buckets represented by {@link KsmBucketInfo}
|
* Returns a list of buckets represented by {@link OmBucketInfo}
|
||||||
* in the given volume.
|
* in the given volume.
|
||||||
*
|
*
|
||||||
* @param volumeName
|
* @param volumeName
|
||||||
|
@ -73,7 +73,7 @@ public interface BucketManager {
|
||||||
* @return a list of buckets.
|
* @return a list of buckets.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
List<KsmBucketInfo> listBuckets(String volumeName,
|
List<OmBucketInfo> listBuckets(String volumeName,
|
||||||
String startBucket, String bucketPrefix, int maxNumOfBuckets)
|
String startBucket, String bucketPrefix, int maxNumOfBuckets)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
}
|
}
|
|
@ -14,15 +14,15 @@
|
||||||
* License for the specific language governing permissions and limitations under
|
* License for the specific language governing permissions and limitations under
|
||||||
* the License.
|
* the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.ozone.ksm;
|
package org.apache.hadoop.ozone.om;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.hadoop.fs.StorageType;
|
import org.apache.hadoop.fs.StorageType;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs;
|
import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
|
||||||
import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
|
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
||||||
import org.apache.hadoop.ozone.ksm.exceptions.KSMException;
|
import org.apache.hadoop.ozone.om.exceptions.OMException;
|
||||||
import org.apache.hadoop.ozone.protocol.proto
|
import org.apache.hadoop.ozone.protocol.proto
|
||||||
.KeySpaceManagerProtocolProtos.BucketInfo;
|
.OzoneManagerProtocolProtos.BucketInfo;
|
||||||
import org.apache.hadoop.ozone.OzoneAcl;
|
import org.apache.hadoop.ozone.OzoneAcl;
|
||||||
import org.apache.hadoop.util.Time;
|
import org.apache.hadoop.util.Time;
|
||||||
import org.iq80.leveldb.DBException;
|
import org.iq80.leveldb.DBException;
|
||||||
|
@ -33,22 +33,22 @@ import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* KSM bucket manager.
|
* OM bucket manager.
|
||||||
*/
|
*/
|
||||||
public class BucketManagerImpl implements BucketManager {
|
public class BucketManagerImpl implements BucketManager {
|
||||||
private static final Logger LOG =
|
private static final Logger LOG =
|
||||||
LoggerFactory.getLogger(BucketManagerImpl.class);
|
LoggerFactory.getLogger(BucketManagerImpl.class);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* KSMMetadataManager is used for accessing KSM MetadataDB and ReadWriteLock.
|
* OMMetadataManager is used for accessing OM MetadataDB and ReadWriteLock.
|
||||||
*/
|
*/
|
||||||
private final KSMMetadataManager metadataManager;
|
private final OMMetadataManager metadataManager;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructs BucketManager.
|
* Constructs BucketManager.
|
||||||
* @param metadataManager
|
* @param metadataManager
|
||||||
*/
|
*/
|
||||||
public BucketManagerImpl(KSMMetadataManager metadataManager){
|
public BucketManagerImpl(OMMetadataManager metadataManager){
|
||||||
this.metadataManager = metadataManager;
|
this.metadataManager = metadataManager;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -73,10 +73,10 @@ public class BucketManagerImpl implements BucketManager {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a bucket.
|
* Creates a bucket.
|
||||||
* @param bucketInfo - KsmBucketInfo.
|
* @param bucketInfo - OmBucketInfo.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void createBucket(KsmBucketInfo bucketInfo) throws IOException {
|
public void createBucket(OmBucketInfo bucketInfo) throws IOException {
|
||||||
Preconditions.checkNotNull(bucketInfo);
|
Preconditions.checkNotNull(bucketInfo);
|
||||||
metadataManager.writeLock().lock();
|
metadataManager.writeLock().lock();
|
||||||
String volumeName = bucketInfo.getVolumeName();
|
String volumeName = bucketInfo.getVolumeName();
|
||||||
|
@ -88,17 +88,17 @@ public class BucketManagerImpl implements BucketManager {
|
||||||
//Check if the volume exists
|
//Check if the volume exists
|
||||||
if (metadataManager.get(volumeKey) == null) {
|
if (metadataManager.get(volumeKey) == null) {
|
||||||
LOG.debug("volume: {} not found ", volumeName);
|
LOG.debug("volume: {} not found ", volumeName);
|
||||||
throw new KSMException("Volume doesn't exist",
|
throw new OMException("Volume doesn't exist",
|
||||||
KSMException.ResultCodes.FAILED_VOLUME_NOT_FOUND);
|
OMException.ResultCodes.FAILED_VOLUME_NOT_FOUND);
|
||||||
}
|
}
|
||||||
//Check if bucket already exists
|
//Check if bucket already exists
|
||||||
if (metadataManager.get(bucketKey) != null) {
|
if (metadataManager.get(bucketKey) != null) {
|
||||||
LOG.debug("bucket: {} already exists ", bucketName);
|
LOG.debug("bucket: {} already exists ", bucketName);
|
||||||
throw new KSMException("Bucket already exist",
|
throw new OMException("Bucket already exist",
|
||||||
KSMException.ResultCodes.FAILED_BUCKET_ALREADY_EXISTS);
|
OMException.ResultCodes.FAILED_BUCKET_ALREADY_EXISTS);
|
||||||
}
|
}
|
||||||
|
|
||||||
KsmBucketInfo ksmBucketInfo = KsmBucketInfo.newBuilder()
|
OmBucketInfo omBucketInfo = OmBucketInfo.newBuilder()
|
||||||
.setVolumeName(bucketInfo.getVolumeName())
|
.setVolumeName(bucketInfo.getVolumeName())
|
||||||
.setBucketName(bucketInfo.getBucketName())
|
.setBucketName(bucketInfo.getBucketName())
|
||||||
.setAcls(bucketInfo.getAcls())
|
.setAcls(bucketInfo.getAcls())
|
||||||
|
@ -106,11 +106,11 @@ public class BucketManagerImpl implements BucketManager {
|
||||||
.setIsVersionEnabled(bucketInfo.getIsVersionEnabled())
|
.setIsVersionEnabled(bucketInfo.getIsVersionEnabled())
|
||||||
.setCreationTime(Time.now())
|
.setCreationTime(Time.now())
|
||||||
.build();
|
.build();
|
||||||
metadataManager.put(bucketKey, ksmBucketInfo.getProtobuf().toByteArray());
|
metadataManager.put(bucketKey, omBucketInfo.getProtobuf().toByteArray());
|
||||||
|
|
||||||
LOG.debug("created bucket: {} in volume: {}", bucketName, volumeName);
|
LOG.debug("created bucket: {} in volume: {}", bucketName, volumeName);
|
||||||
} catch (IOException | DBException ex) {
|
} catch (IOException | DBException ex) {
|
||||||
if (!(ex instanceof KSMException)) {
|
if (!(ex instanceof OMException)) {
|
||||||
LOG.error("Bucket creation failed for bucket:{} in volume:{}",
|
LOG.error("Bucket creation failed for bucket:{} in volume:{}",
|
||||||
bucketName, volumeName, ex);
|
bucketName, volumeName, ex);
|
||||||
}
|
}
|
||||||
|
@ -127,7 +127,7 @@ public class BucketManagerImpl implements BucketManager {
|
||||||
* @param bucketName - Name of the Bucket.
|
* @param bucketName - Name of the Bucket.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public KsmBucketInfo getBucketInfo(String volumeName, String bucketName)
|
public OmBucketInfo getBucketInfo(String volumeName, String bucketName)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Preconditions.checkNotNull(volumeName);
|
Preconditions.checkNotNull(volumeName);
|
||||||
Preconditions.checkNotNull(bucketName);
|
Preconditions.checkNotNull(bucketName);
|
||||||
|
@ -138,12 +138,12 @@ public class BucketManagerImpl implements BucketManager {
|
||||||
if (value == null) {
|
if (value == null) {
|
||||||
LOG.debug("bucket: {} not found in volume: {}.", bucketName,
|
LOG.debug("bucket: {} not found in volume: {}.", bucketName,
|
||||||
volumeName);
|
volumeName);
|
||||||
throw new KSMException("Bucket not found",
|
throw new OMException("Bucket not found",
|
||||||
KSMException.ResultCodes.FAILED_BUCKET_NOT_FOUND);
|
OMException.ResultCodes.FAILED_BUCKET_NOT_FOUND);
|
||||||
}
|
}
|
||||||
return KsmBucketInfo.getFromProtobuf(BucketInfo.parseFrom(value));
|
return OmBucketInfo.getFromProtobuf(BucketInfo.parseFrom(value));
|
||||||
} catch (IOException | DBException ex) {
|
} catch (IOException | DBException ex) {
|
||||||
if (!(ex instanceof KSMException)) {
|
if (!(ex instanceof OMException)) {
|
||||||
LOG.error("Exception while getting bucket info for bucket: {}",
|
LOG.error("Exception while getting bucket info for bucket: {}",
|
||||||
bucketName, ex);
|
bucketName, ex);
|
||||||
}
|
}
|
||||||
|
@ -159,7 +159,7 @@ public class BucketManagerImpl implements BucketManager {
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void setBucketProperty(KsmBucketArgs args) throws IOException {
|
public void setBucketProperty(OmBucketArgs args) throws IOException {
|
||||||
Preconditions.checkNotNull(args);
|
Preconditions.checkNotNull(args);
|
||||||
metadataManager.writeLock().lock();
|
metadataManager.writeLock().lock();
|
||||||
String volumeName = args.getVolumeName();
|
String volumeName = args.getVolumeName();
|
||||||
|
@ -170,19 +170,19 @@ public class BucketManagerImpl implements BucketManager {
|
||||||
if(metadataManager.get(metadataManager.getVolumeKey(volumeName)) ==
|
if(metadataManager.get(metadataManager.getVolumeKey(volumeName)) ==
|
||||||
null) {
|
null) {
|
||||||
LOG.debug("volume: {} not found ", volumeName);
|
LOG.debug("volume: {} not found ", volumeName);
|
||||||
throw new KSMException("Volume doesn't exist",
|
throw new OMException("Volume doesn't exist",
|
||||||
KSMException.ResultCodes.FAILED_VOLUME_NOT_FOUND);
|
OMException.ResultCodes.FAILED_VOLUME_NOT_FOUND);
|
||||||
}
|
}
|
||||||
byte[] value = metadataManager.get(bucketKey);
|
byte[] value = metadataManager.get(bucketKey);
|
||||||
//Check if bucket exist
|
//Check if bucket exist
|
||||||
if(value == null) {
|
if(value == null) {
|
||||||
LOG.debug("bucket: {} not found ", bucketName);
|
LOG.debug("bucket: {} not found ", bucketName);
|
||||||
throw new KSMException("Bucket doesn't exist",
|
throw new OMException("Bucket doesn't exist",
|
||||||
KSMException.ResultCodes.FAILED_BUCKET_NOT_FOUND);
|
OMException.ResultCodes.FAILED_BUCKET_NOT_FOUND);
|
||||||
}
|
}
|
||||||
KsmBucketInfo oldBucketInfo = KsmBucketInfo.getFromProtobuf(
|
OmBucketInfo oldBucketInfo = OmBucketInfo.getFromProtobuf(
|
||||||
BucketInfo.parseFrom(value));
|
BucketInfo.parseFrom(value));
|
||||||
KsmBucketInfo.Builder bucketInfoBuilder = KsmBucketInfo.newBuilder();
|
OmBucketInfo.Builder bucketInfoBuilder = OmBucketInfo.newBuilder();
|
||||||
bucketInfoBuilder.setVolumeName(oldBucketInfo.getVolumeName())
|
bucketInfoBuilder.setVolumeName(oldBucketInfo.getVolumeName())
|
||||||
.setBucketName(oldBucketInfo.getBucketName());
|
.setBucketName(oldBucketInfo.getBucketName());
|
||||||
|
|
||||||
|
@ -221,7 +221,7 @@ public class BucketManagerImpl implements BucketManager {
|
||||||
metadataManager.put(bucketKey,
|
metadataManager.put(bucketKey,
|
||||||
bucketInfoBuilder.build().getProtobuf().toByteArray());
|
bucketInfoBuilder.build().getProtobuf().toByteArray());
|
||||||
} catch (IOException | DBException ex) {
|
} catch (IOException | DBException ex) {
|
||||||
if (!(ex instanceof KSMException)) {
|
if (!(ex instanceof OMException)) {
|
||||||
LOG.error("Setting bucket property failed for bucket:{} in volume:{}",
|
LOG.error("Setting bucket property failed for bucket:{} in volume:{}",
|
||||||
bucketName, volumeName, ex);
|
bucketName, volumeName, ex);
|
||||||
}
|
}
|
||||||
|
@ -269,24 +269,24 @@ public class BucketManagerImpl implements BucketManager {
|
||||||
if (metadataManager.get(metadataManager.getVolumeKey(volumeName))
|
if (metadataManager.get(metadataManager.getVolumeKey(volumeName))
|
||||||
== null) {
|
== null) {
|
||||||
LOG.debug("volume: {} not found ", volumeName);
|
LOG.debug("volume: {} not found ", volumeName);
|
||||||
throw new KSMException("Volume doesn't exist",
|
throw new OMException("Volume doesn't exist",
|
||||||
KSMException.ResultCodes.FAILED_VOLUME_NOT_FOUND);
|
OMException.ResultCodes.FAILED_VOLUME_NOT_FOUND);
|
||||||
}
|
}
|
||||||
//Check if bucket exist
|
//Check if bucket exist
|
||||||
if (metadataManager.get(bucketKey) == null) {
|
if (metadataManager.get(bucketKey) == null) {
|
||||||
LOG.debug("bucket: {} not found ", bucketName);
|
LOG.debug("bucket: {} not found ", bucketName);
|
||||||
throw new KSMException("Bucket doesn't exist",
|
throw new OMException("Bucket doesn't exist",
|
||||||
KSMException.ResultCodes.FAILED_BUCKET_NOT_FOUND);
|
OMException.ResultCodes.FAILED_BUCKET_NOT_FOUND);
|
||||||
}
|
}
|
||||||
//Check if bucket is empty
|
//Check if bucket is empty
|
||||||
if (!metadataManager.isBucketEmpty(volumeName, bucketName)) {
|
if (!metadataManager.isBucketEmpty(volumeName, bucketName)) {
|
||||||
LOG.debug("bucket: {} is not empty ", bucketName);
|
LOG.debug("bucket: {} is not empty ", bucketName);
|
||||||
throw new KSMException("Bucket is not empty",
|
throw new OMException("Bucket is not empty",
|
||||||
KSMException.ResultCodes.FAILED_BUCKET_NOT_EMPTY);
|
OMException.ResultCodes.FAILED_BUCKET_NOT_EMPTY);
|
||||||
}
|
}
|
||||||
metadataManager.delete(bucketKey);
|
metadataManager.delete(bucketKey);
|
||||||
} catch (IOException ex) {
|
} catch (IOException ex) {
|
||||||
if (!(ex instanceof KSMException)) {
|
if (!(ex instanceof OMException)) {
|
||||||
LOG.error("Delete bucket failed for bucket:{} in volume:{}", bucketName,
|
LOG.error("Delete bucket failed for bucket:{} in volume:{}", bucketName,
|
||||||
volumeName, ex);
|
volumeName, ex);
|
||||||
}
|
}
|
||||||
|
@ -300,7 +300,7 @@ public class BucketManagerImpl implements BucketManager {
|
||||||
* {@inheritDoc}
|
* {@inheritDoc}
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public List<KsmBucketInfo> listBuckets(String volumeName,
|
public List<OmBucketInfo> listBuckets(String volumeName,
|
||||||
String startBucket, String bucketPrefix, int maxNumOfBuckets)
|
String startBucket, String bucketPrefix, int maxNumOfBuckets)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Preconditions.checkNotNull(volumeName);
|
Preconditions.checkNotNull(volumeName);
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue