+ * 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.hdds.scm;
+
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+
+import java.util.List;
+
+/**
+ * A {@link PipelineChoosePolicy} support choosing pipeline from exist list.
+ */
+public interface PipelineChoosePolicy {
+
+ /**
+ * Given an initial list of pipelines, return one of the pipelines.
+ *
+ * @param pipelineList list of pipelines.
+ * @return one of the pipelines.
+ */
+ Pipeline choosePipeline(List pipelineList,
+ PipelineRequestInformation pri);
+}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/PipelineRequestInformation.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/PipelineRequestInformation.java
new file mode 100644
index 000000000000..ac0cfbe57beb
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/PipelineRequestInformation.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.scm;
+
+/**
+ * The information of the request of pipeline.
+ */
+public final class PipelineRequestInformation {
+ private long size;
+
+ /**
+ * Builder for PipelineRequestInformation.
+ */
+ public static class Builder {
+ private long size;
+
+ public static Builder getBuilder() {
+ return new Builder();
+ }
+
+ /**
+ * sets the size.
+ * @param sz request size
+ * @return Builder for PipelineRequestInformation
+ */
+ public Builder setSize(long sz) {
+ this.size = sz;
+ return this;
+ }
+
+ public PipelineRequestInformation build() {
+ return new PipelineRequestInformation(size);
+ }
+ }
+
+ private PipelineRequestInformation(long size) {
+ this.size = size;
+ }
+
+ public long getSize() {
+ return size;
+ }
+}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java
index 73701ea55c9d..3084bb4cc396 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java
@@ -59,6 +59,21 @@ public class ScmConfig {
)
private String action;
+ @Config(key = "pipeline.choose.policy.impl",
+ type = ConfigType.STRING,
+ defaultValue = "org.apache.hadoop.hdds.scm.pipeline.choose.algorithms" +
+ ".RandomPipelineChoosePolicy",
+ tags = { ConfigTag.SCM, ConfigTag.PIPELINE },
+ description =
+ "The full name of class which implements "
+ + "org.apache.hadoop.hdds.scm.PipelineChoosePolicy. "
+ + "The class decides which pipeline will be used to find or "
+ + "allocate container. If not set, "
+ + "org.apache.hadoop.hdds.scm.pipeline.choose.algorithms. "
+ + "RandomPipelineChoosePolicy will be used as default value."
+ )
+ private String pipelineChoosePolicyName;
+
public void setKerberosPrincipal(String kerberosPrincipal) {
this.principal = kerberosPrincipal;
}
@@ -72,6 +87,10 @@ public void setUnknownContainerAction(String unknownContainerAction) {
this.action = unknownContainerAction;
}
+ public void setPipelineChoosePolicyName(String pipelineChoosePolicyName) {
+ this.pipelineChoosePolicyName = pipelineChoosePolicyName;
+ }
+
public String getKerberosPrincipal() {
return this.principal;
}
@@ -84,6 +103,10 @@ public String getUnknownContainerAction() {
return this.action;
}
+ public String getPipelineChoosePolicyName() {
+ return pipelineChoosePolicyName;
+ }
+
/**
* Configuration strings class.
* required for SCMSecurityProtocol where the KerberosInfo references
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
index 2efd82b62977..4e624c695889 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
@@ -44,9 +44,11 @@ public final class ScmConfigKeys {
= "dfs.container.ratis.rpc.type";
public static final String DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT
= "GRPC";
- public static final String DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_KEY
- = "dfs.container.ratis.num.write.chunk.threads";
- public static final int DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_DEFAULT
+ public static final String
+ DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_PER_VOLUME
+ = "dfs.container.ratis.num.write.chunk.threads.per.volume";
+ public static final int
+ DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_PER_VOLUME_DEFAULT
= 10;
public static final String DFS_CONTAINER_RATIS_REPLICATION_LEVEL_KEY
= "dfs.container.ratis.replication.level";
@@ -287,6 +289,7 @@ public final class ScmConfigKeys {
public static final String OZONE_SCM_PIPELINE_OWNER_CONTAINER_COUNT =
"ozone.scm.pipeline.owner.container.count";
public static final int OZONE_SCM_PIPELINE_OWNER_CONTAINER_COUNT_DEFAULT = 3;
+
// Pipeline placement policy:
// Upper limit for how many pipelines a datanode can engage in.
public static final String OZONE_DATANODE_PIPELINE_LIMIT =
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java
index 328777799bd1..1c7d1f6408d8 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java
@@ -21,10 +21,12 @@
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
+import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -186,4 +188,13 @@ public abstract XceiverClientReply watchForCommit(long index)
* @return min commit index replicated to all servers.
*/
public abstract long getReplicatedMinCommitIndex();
+
+ /**
+ * Sends command to all nodes in the pipeline.
+ * @return a map containing datanode as the key and
+ * the command response from that datanode
+ */
+ public abstract Map
+ sendCommandOnAllNodes(ContainerCommandRequestProto request)
+ throws IOException, InterruptedException;
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/exceptions/SCMException.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/exceptions/SCMException.java
index db1f82ae411d..0146eaed6e33 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/exceptions/SCMException.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/exceptions/SCMException.java
@@ -122,6 +122,7 @@ public enum ResultCodes {
FAILED_TO_FIND_ACTIVE_PIPELINE,
FAILED_TO_INIT_CONTAINER_PLACEMENT_POLICY,
FAILED_TO_ALLOCATE_ENOUGH_BLOCKS,
- INTERNAL_ERROR
+ INTERNAL_ERROR,
+ FAILED_TO_INIT_PIPELINE_CHOOSE_POLICY
}
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
index 6b0d8f8cda2a..11acf82ff32f 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.hdds.scm.storage;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.XceiverClientReply;
import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
import org.apache.hadoop.hdds.scm.container.common.helpers
@@ -73,7 +74,9 @@
import java.io.IOException;
import java.util.ArrayList;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import java.util.concurrent.ExecutionException;
/**
@@ -578,4 +581,35 @@ public static List getValidatorList() {
validators.add(validator);
return validators;
}
+
+ public static HashMap
+ getBlockFromAllNodes(
+ XceiverClientSpi xceiverClient,
+ DatanodeBlockID datanodeBlockID) throws IOException,
+ InterruptedException {
+ GetBlockRequestProto.Builder readBlockRequest = GetBlockRequestProto
+ .newBuilder()
+ .setBlockID(datanodeBlockID);
+ HashMap datanodeToResponseMap
+ = new HashMap<>();
+ String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+ ContainerCommandRequestProto.Builder builder = ContainerCommandRequestProto
+ .newBuilder()
+ .setCmdType(Type.GetBlock)
+ .setContainerID(datanodeBlockID.getContainerID())
+ .setDatanodeUuid(id)
+ .setGetBlock(readBlockRequest);
+ String encodedToken = getEncodedBlockToken(getService(datanodeBlockID));
+ if (encodedToken != null) {
+ builder.setEncodedToken(encodedToken);
+ }
+ ContainerCommandRequestProto request = builder.build();
+ Map responses =
+ xceiverClient.sendCommandOnAllNodes(request);
+ for(Map.Entry entry:
+ responses.entrySet()){
+ datanodeToResponseMap.put(entry.getKey(), entry.getValue().getGetBlock());
+ }
+ return datanodeToResponseMap;
+ }
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
index 7d46b01a6dbf..482ac88f366c 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
@@ -79,15 +79,6 @@ public final class OzoneConfigKeys {
"ozone.trace.enabled";
public static final boolean OZONE_TRACE_ENABLED_DEFAULT = false;
- public static final String OZONE_METADATA_STORE_IMPL =
- "ozone.metastore.impl";
- public static final String OZONE_METADATA_STORE_IMPL_LEVELDB =
- "LevelDB";
- public static final String OZONE_METADATA_STORE_IMPL_ROCKSDB =
- "RocksDB";
- public static final String OZONE_METADATA_STORE_IMPL_DEFAULT =
- OZONE_METADATA_STORE_IMPL_ROCKSDB;
-
public static final String OZONE_METADATA_STORE_ROCKSDB_STATISTICS =
"ozone.metastore.rocksdb.statistics";
@@ -104,6 +95,9 @@ public final class OzoneConfigKeys {
public static final String OZONE_CONTAINER_CACHE_SIZE =
"ozone.container.cache.size";
public static final int OZONE_CONTAINER_CACHE_DEFAULT = 1024;
+ public static final String OZONE_CONTAINER_CACHE_LOCK_STRIPES =
+ "ozone.container.cache.lock.stripes";
+ public static final int OZONE_CONTAINER_CACHE_LOCK_STRIPES_DEFAULT = 1024;
public static final String OZONE_SCM_BLOCK_SIZE =
"ozone.scm.block.size";
@@ -248,10 +242,13 @@ public final class OzoneConfigKeys {
= ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY;
public static final String DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT
= ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT;
- public static final String DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_KEY
- = ScmConfigKeys.DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_KEY;
- public static final int DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_DEFAULT
- = ScmConfigKeys.DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_DEFAULT;
+ public static final String
+ DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_PER_VOLUME_KEY
+ = ScmConfigKeys.DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_PER_VOLUME;
+ public static final int
+ DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_PER_VOLUME_DEFAULT
+ = ScmConfigKeys.
+ DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_PER_VOLUME_DEFAULT;
public static final String DFS_CONTAINER_RATIS_REPLICATION_LEVEL_KEY
= ScmConfigKeys.DFS_CONTAINER_RATIS_REPLICATION_LEVEL_KEY;
public static final ReplicationLevel
@@ -471,6 +468,7 @@ public final class OzoneConfigKeys {
public static final String OZONE_CLIENT_HTTPS_NEED_AUTH_KEY =
"ozone.https.client.need-auth";
public static final boolean OZONE_CLIENT_HTTPS_NEED_AUTH_DEFAULT = false;
+
/**
* There is no need to instantiate this class.
*/
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index a6833a5ee990..9854d40494be 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -293,9 +293,15 @@ private OzoneConsts() {
public static final String MAX_PARTS = "maxParts";
public static final String S3_BUCKET = "s3Bucket";
public static final String S3_GETSECRET_USER = "S3GetSecretUser";
+ public static final String RENAMED_KEYS_MAP = "renamedKeysMap";
+ public static final String UNRENAMED_KEYS_MAP = "unRenamedKeysMap";
public static final String MULTIPART_UPLOAD_PART_NUMBER = "partNumber";
public static final String MULTIPART_UPLOAD_PART_NAME = "partName";
public static final String BUCKET_ENCRYPTION_KEY = "bucketEncryptionKey";
+ public static final String DELETED_KEYS_LIST = "deletedKeysList";
+ public static final String UNDELETED_KEYS_LIST = "unDeletedKeysList";
+ public static final String SOURCE_VOLUME = "sourceVolume";
+ public static final String SOURCE_BUCKET = "sourceBucket";
@@ -360,4 +366,9 @@ private OzoneConsts() {
public static final String TRANSACTION_INFO_KEY = "#TRANSACTIONINFO";
public static final String TRANSACTION_INFO_SPLIT_KEY = "#";
+ public static final String CONTAINER_DB_TYPE_ROCKSDB = "RocksDB";
+ public static final String CONTAINER_DB_TYPE_LEVELDB = "LevelDB";
+
+ // An on-disk transient marker file used when replacing DB with checkpoint
+ public static final String DB_TRANSIENT_MARKER = "dbInconsistentMarker";
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java
index 25f8e8630035..d5e1348027a0 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Storage.java
@@ -168,7 +168,7 @@ public File getCurrentDir() {
*
* @return the version file path
*/
- private File getVersionFile() {
+ public File getVersionFile() {
return new File(getCurrentDir(), STORAGE_FILE_VERSION);
}
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index 8b724d9818e7..577044815579 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -37,6 +37,16 @@
size of that cache.
+
+ ozone.container.cache.lock.stripes
+ 1024
+ PERFORMANCE, CONTAINER, STORAGE
+ Container DB open is an exclusive operation. We use a stripe
+ lock to guarantee that different threads can open different container DBs
+ concurrently, while for one container DB, only one thread can open it at
+ the same time. This setting controls the lock stripes.
+
+ dfs.container.ipc9859
@@ -196,7 +206,7 @@
- dfs.container.ratis.num.write.chunk.threads
+ dfs.container.ratis.num.write.chunk.threads.per.volume10OZONE, RATIS, PERFORMANCEMaximum number of threads in the thread pool that Datanode
@@ -634,17 +644,6 @@
dfs.container.ratis.datanode.storage.dir be configured separately.
-
- ozone.metastore.impl
- RocksDB
- OZONE, OM, SCM, CONTAINER, STORAGE
-
- Ozone metadata store implementation. Ozone metadata are well
- distributed to multiple services such as ozoneManager, scm. They are stored in
- some local key-value databases. This property determines which database
- library to use. Supported value is either LevelDB or RocksDB.
-
- ozone.metastore.rocksdb.statistics
@@ -819,7 +818,7 @@
ozone.scm.pipeline.owner.container.count3OZONE, SCM, PIPELINE
- Number of containers per owner in a pipeline.
+ Number of containers per owner per disk in a pipeline.
@@ -2291,15 +2290,6 @@
Whether to enable topology aware read to improve the read performance.
-
- ozone.recon.container.db.impl
- RocksDB
- OZONE, RECON, STORAGE
-
- Ozone Recon container DB store implementation.Supported value is either
- LevelDB or RocksDB.
-
- ozone.recon.om.db.dir
@@ -2441,6 +2431,14 @@
information will be extracted
+
+ ozone.s3g.client.buffer.size
+ OZONE, S3GATEWAY
+ 4KB
+
+ The size of the buffer which is for read block. (4KB by default).
+
+ ssl.server.keystore.keypasswordOZONE, SECURITY, MANAGEMENT
@@ -2498,4 +2496,19 @@
rules in Amazon S3's object key naming guide.
+
+
+ ozone.om.enable.filesystem.paths
+ OZONE, OM
+ false
+ If true, key names will be interpreted as file system paths.
+ "/" will be treated as a special character and paths will be normalized
+ and must follow Unix filesystem path naming conventions. This flag will
+ be helpful when objects created by S3G need to be accessed using OFS/O3Fs.
+ If false, it will fallback to default behavior of Key/MPU create
+ requests where key paths are not normalized and any intermediate
+ directories will not be created or any file checks happens to check
+ filesystem semantics.
+
+
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/ratis/conf/TestRaftClientConfig.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/ratis/conf/TestRaftClientConfig.java
new file mode 100644
index 000000000000..46edc2005cf5
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/ratis/conf/TestRaftClientConfig.java
@@ -0,0 +1,62 @@
+/*
+ * 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
+ *
+ * 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.hdds.ratis.conf;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.junit.jupiter.api.Test;
+
+import java.time.Duration;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * Tests for {@link RatisClientConfig.RaftConfig}.
+ */
+class TestRaftClientConfig {
+
+ @Test
+ void defaults() {
+ RatisClientConfig.RaftConfig subject = new RatisClientConfig.RaftConfig();
+ RatisClientConfig.RaftConfig fromConfig =
+ new OzoneConfiguration().getObject(RatisClientConfig.RaftConfig.class);
+
+ assertEquals(fromConfig.getMaxOutstandingRequests(),
+ subject.getMaxOutstandingRequests());
+ assertEquals(fromConfig.getRpcRequestTimeout(),
+ subject.getRpcRequestTimeout());
+ assertEquals(fromConfig.getRpcWatchRequestTimeout(),
+ subject.getRpcWatchRequestTimeout());
+ }
+
+ @Test
+ void setAndGet() {
+ RatisClientConfig.RaftConfig subject = new RatisClientConfig.RaftConfig();
+ final int maxOutstandingRequests = 42;
+ final Duration rpcRequestTimeout = Duration.ofMillis(12313);
+ final Duration rpcWatchRequestTimeout = Duration.ofSeconds(99);
+
+ subject.setMaxOutstandingRequests(maxOutstandingRequests);
+ subject.setRpcRequestTimeout(rpcRequestTimeout);
+ subject.setRpcWatchRequestTimeout(rpcWatchRequestTimeout);
+
+ assertEquals(maxOutstandingRequests, subject.getMaxOutstandingRequests());
+ assertEquals(rpcRequestTimeout, subject.getRpcRequestTimeout());
+ assertEquals(rpcWatchRequestTimeout, subject.getRpcWatchRequestTimeout());
+ }
+
+}
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/ratis/conf/TestRatisClientConfig.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/ratis/conf/TestRatisClientConfig.java
new file mode 100644
index 000000000000..2d16af2a0cbd
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/ratis/conf/TestRatisClientConfig.java
@@ -0,0 +1,68 @@
+/*
+ * 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
+ *
+ * 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.hdds.conf;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * In memory, mutable configuration source for testing.
+ */
+public class InMemoryConfiguration implements MutableConfigurationSource {
+
+ private Map configs = new HashMap<>();
+
+ public InMemoryConfiguration() {
+ }
+
+ public InMemoryConfiguration(String key, String value) {
+ set(key, value);
+ }
+
+ @Override
+ public String get(String key) {
+ return configs.get(key);
+ }
+
+ @Override
+ public Collection getConfigKeys() {
+ return configs.keySet();
+ }
+
+ @Override
+ public char[] getPassword(String key) throws IOException {
+ return configs.get(key).toCharArray();
+ }
+
+ @Override
+ public void set(String key, String value) {
+ configs.put(key, value);
+ }
+}
diff --git a/hadoop-hdds/container-service/pom.xml b/hadoop-hdds/container-service/pom.xml
index d10d2a358ce1..b71f8e3471e7 100644
--- a/hadoop-hdds/container-service/pom.xml
+++ b/hadoop-hdds/container-service/pom.xml
@@ -20,10 +20,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
org.apache.hadoophadoop-hdds
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOThadoop-hdds-container-service
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOTApache Hadoop Distributed Data Store Container ServiceApache Hadoop HDDS Container Servicejar
@@ -70,6 +70,13 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
org.yamlsnakeyaml
+
+
+ org.apache.hadoop
+ hadoop-hdds-docs
+ provided
+
+
com.github.spotbugsspotbugs
@@ -107,6 +114,37 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
${basedir}/dev-support/findbugsExcludeFile.xml
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
+
+
+ copy-common-html
+ prepare-package
+
+ unpack
+
+
+
+
+ org.apache.hadoop
+ hadoop-hdds-server-framework
+ ${project.build.outputDirectory}
+
+ webapps/static/**/*.*
+
+
+ org.apache.hadoop
+ hadoop-hdds-docs
+ ${project.build.outputDirectory}/webapps/hddsDatanode
+ docs/**/*.*
+
+
+ true
+
+
+
+
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/DNMXBean.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/DNMXBean.java
new file mode 100644
index 000000000000..d36fcdb6fc70
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/DNMXBean.java
@@ -0,0 +1,29 @@
+/**
+ * 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
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.server.ServiceRuntimeInfo;
+
+/**
+ * This is the JMX management interface for DN information.
+ */
+@InterfaceAudience.Private
+public interface DNMXBean extends ServiceRuntimeInfo {
+}
diff --git a/hadoop-ozone/upgrade/src/main/java/org/apache/hadoop/ozone/upgrade/Execute.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/DNMXBeanImpl.java
similarity index 64%
rename from hadoop-ozone/upgrade/src/main/java/org/apache/hadoop/ozone/upgrade/Execute.java
rename to hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/DNMXBeanImpl.java
index 0837200c1fac..18ad66ce5a69 100644
--- a/hadoop-ozone/upgrade/src/main/java/org/apache/hadoop/ozone/upgrade/Execute.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/DNMXBeanImpl.java
@@ -15,23 +15,18 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.ozone.upgrade;
-import java.util.concurrent.Callable;
+package org.apache.hadoop.ozone;
-import picocli.CommandLine.Command;
+import org.apache.hadoop.hdds.server.ServiceRuntimeInfoImpl;
+import org.apache.hadoop.hdds.utils.VersionInfo;
/**
- * Execute Ozone specific HDFS ballanced..
+ * This is the JMX management class for DN information.
*/
-@Command(name = "execute",
- description = "Start/restart upgrade from HDFS to Ozone cluster.")
-public class Execute implements Callable {
-
- @Override
- public Void call() throws Exception {
- System.err.println("In-Place upgrade : execute] is not yet supported.");
- return null;
+public class DNMXBeanImpl extends ServiceRuntimeInfoImpl implements DNMXBean {
+ public DNMXBeanImpl(
+ VersionInfo versionInfo) {
+ super(versionInfo);
}
-
}
\ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
index 7e896e715598..cfb22e30dcd2 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
@@ -23,12 +23,14 @@
import java.security.KeyPair;
import java.security.cert.CertificateException;
import java.util.Arrays;
+import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.ConcurrentHashMap;
+import com.sun.jmx.mbeanserver.Introspector;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.hdds.DFSConfigKeysLegacy;
import org.apache.hadoop.hdds.HddsUtils;
@@ -49,6 +51,7 @@
import org.apache.hadoop.hdds.server.http.RatisDropwizardExports;
import org.apache.hadoop.hdds.tracing.TracingUtil;
import org.apache.hadoop.hdds.utils.HddsVersionInfo;
+import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
import org.apache.hadoop.ozone.container.common.utils.HddsVolumeUtil;
@@ -65,11 +68,15 @@
import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_PLUGINS_KEY;
import static org.apache.hadoop.util.ExitUtil.terminate;
+
+import org.apache.hadoop.util.Time;
import org.bouncycastle.pkcs.PKCS10CertificationRequest;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import picocli.CommandLine.Command;
+import javax.management.ObjectName;
+
/**
* Datanode service plugin to start the HDDS container services.
*/
@@ -92,9 +99,13 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
private HddsDatanodeHttpServer httpServer;
private boolean printBanner;
private String[] args;
- private volatile AtomicBoolean isStopped = new AtomicBoolean(false);
+ private final AtomicBoolean isStopped = new AtomicBoolean(false);
private final Map ratisMetricsMap =
new ConcurrentHashMap<>();
+ private DNMXBeanImpl serviceRuntimeInfo =
+ new DNMXBeanImpl(HddsVersionInfo.HDDS_VERSION_INFO) {};
+ private ObjectName dnInfoBeanName;
+
//Constructor for DataNode PluginService
public HddsDatanodeService(){}
@@ -132,6 +143,7 @@ private static HddsDatanodeService createHddsDatanodeService(
public static void main(String[] args) {
try {
+ Introspector.checkCompliance(DNMXBeanImpl.class);
HddsDatanodeService hddsDatanodeService =
createHddsDatanodeService(args, true);
hddsDatanodeService.run(args);
@@ -180,6 +192,8 @@ public void start(OzoneConfiguration configuration) {
}
public void start() {
+ serviceRuntimeInfo.setStartTime();
+
RatisDropwizardExports.
registerRatisMetricReporters(ratisMetricsMap);
@@ -191,6 +205,12 @@ public void start() {
datanodeDetails = initializeDatanodeDetails();
datanodeDetails.setHostName(hostname);
datanodeDetails.setIpAddress(ip);
+ datanodeDetails.setVersion(
+ HddsVersionInfo.HDDS_VERSION_INFO.getVersion());
+ datanodeDetails.setSetupTime(Time.now());
+ datanodeDetails.setRevision(
+ HddsVersionInfo.HDDS_VERSION_INFO.getRevision());
+ datanodeDetails.setBuildDate(HddsVersionInfo.HDDS_VERSION_INFO.getDate());
TracingUtil.initTracing(
"HddsDatanodeService." + datanodeDetails.getUuidString()
.substring(0, 8), conf);
@@ -245,7 +265,7 @@ public void start() {
.equalsIgnoreCase(System.getenv("OZONE_DATANODE_STANDALONE_TEST"))) {
startRatisForTest();
}
-
+ registerMXBean();
} catch (IOException e) {
throw new RuntimeException("Can't start the HDDS datanode plugin", e);
} catch (AuthenticationException ex) {
@@ -332,9 +352,15 @@ private void getSCMSignedCert(OzoneConfiguration config) {
dnCertClient.storeCertificate(pemEncodedCert, true);
dnCertClient.storeCertificate(response.getX509CACertificate(), true,
true);
- datanodeDetails.setCertSerialId(getX509Certificate(pemEncodedCert).
- getSerialNumber().toString());
+ String dnCertSerialId = getX509Certificate(pemEncodedCert).
+ getSerialNumber().toString();
+ datanodeDetails.setCertSerialId(dnCertSerialId);
persistDatanodeDetails(datanodeDetails);
+ // Rebuild dnCertClient with the new CSR result so that the default
+ // certSerialId and the x509Certificate can be updated.
+ dnCertClient = new DNCertificateClient(
+ new SecurityConfig(config), dnCertSerialId);
+
} else {
throw new RuntimeException("Unable to retrieve datanode certificate " +
"chain");
@@ -345,6 +371,21 @@ private void getSCMSignedCert(OzoneConfiguration config) {
}
}
+ private void registerMXBean() {
+ Map jmxProperties = new HashMap<>();
+ jmxProperties.put("component", "ServerRuntime");
+ this.dnInfoBeanName = HddsUtils.registerWithJmxProperties(
+ "HddsDatanodeService",
+ "HddsDatanodeServiceInfo", jmxProperties, this.serviceRuntimeInfo);
+ }
+
+ private void unregisterMXBean() {
+ if (this.dnInfoBeanName != null) {
+ MBeans.unregister(this.dnInfoBeanName);
+ this.dnInfoBeanName = null;
+ }
+ }
+
/**
* Creates CSR for DN.
* @param config
@@ -490,8 +531,7 @@ public void terminateDatanode() {
@Override
public void stop() {
- if (!isStopped.get()) {
- isStopped.set(true);
+ if (!isStopped.getAndSet(true)) {
if (plugins != null) {
for (ServicePlugin plugin : plugins) {
try {
@@ -512,6 +552,7 @@ public void stop() {
LOG.error("Stopping HttpServer is failed.", e);
}
}
+ unregisterMXBean();
}
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/StorageLocationReport.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/StorageLocationReport.java
index 061d09bd4a5e..2ad7f0d49817 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/StorageLocationReport.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/StorageLocationReport.java
@@ -88,11 +88,14 @@ public StorageType getStorageType() {
return storageType;
}
+ private StorageTypeProto getStorageTypeProto() throws IOException {
+ return getStorageTypeProto(getStorageType());
+ }
- private StorageTypeProto getStorageTypeProto() throws
- IOException {
+ public static StorageTypeProto getStorageTypeProto(StorageType type)
+ throws IOException {
StorageTypeProto storageTypeProto;
- switch (getStorageType()) {
+ switch (type) {
case SSD:
storageTypeProto = StorageTypeProto.SSD;
break;
@@ -145,7 +148,7 @@ private static StorageType getStorageType(StorageTypeProto proto) throws
* @return SCMStorageReport
* @throws IOException In case, the storage type specified is invalid.
*/
- public StorageReportProto getProtoBufMessage() throws IOException{
+ public StorageReportProto getProtoBufMessage() throws IOException {
StorageReportProto.Builder srb = StorageReportProto.newBuilder();
return srb.setStorageUuid(getId())
.setCapacity(getCapacity())
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java
index 41fc26716c19..2cee75c00fe8 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/TopNOrderedContainerDeletionChoosingPolicy.java
@@ -45,7 +45,7 @@ public class TopNOrderedContainerDeletionChoosingPolicy
private static final Comparator
KEY_VALUE_CONTAINER_DATA_COMPARATOR = (KeyValueContainerData c1,
KeyValueContainerData c2) ->
- Integer.compare(c2.getNumPendingDeletionBlocks(),
+ Long.compare(c2.getNumPendingDeletionBlocks(),
c1.getNumPendingDeletionBlocks());
@Override
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
index 779b60a1d816..425074d6888a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
@@ -19,11 +19,13 @@
import java.io.Closeable;
import java.io.IOException;
import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.hadoop.hdds.HddsUtils;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.CommandStatusReportsProto;
@@ -50,7 +52,6 @@
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import org.apache.hadoop.util.JvmPauseMonitor;
import org.apache.hadoop.util.Time;
-import org.apache.hadoop.util.concurrent.HadoopExecutors;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -103,9 +104,10 @@ public DatanodeStateMachine(DatanodeDetails datanodeDetails,
this.hddsDatanodeStopService = hddsDatanodeStopService;
this.conf = conf;
this.datanodeDetails = datanodeDetails;
- executorService = HadoopExecutors.newCachedThreadPool(
- new ThreadFactoryBuilder().setDaemon(true)
- .setNameFormat("Datanode State Machine Thread - %d").build());
+ executorService = Executors.newFixedThreadPool(
+ getEndPointTaskThreadPoolSize(),
+ new ThreadFactoryBuilder()
+ .setNameFormat("Datanode State Machine Task Thread - %d").build());
connectionManager = new SCMConnectionManager(conf);
context = new StateContext(this.conf, DatanodeStates.getInitState(), this);
// OzoneContainer instance is used in a non-thread safe way by the context
@@ -124,7 +126,9 @@ public DatanodeStateMachine(DatanodeDetails datanodeDetails,
ContainerReplicator replicator =
new DownloadAndImportReplicator(container.getContainerSet(),
container.getController(),
- new SimpleContainerDownloader(conf), new TarContainerPacker());
+ new SimpleContainerDownloader(conf,
+ dnCertClient != null ? dnCertClient.getCACertificate() : null),
+ new TarContainerPacker());
supervisor =
new ReplicationSupervisor(container.getContainerSet(), replicator,
@@ -155,6 +159,21 @@ public DatanodeStateMachine(DatanodeDetails datanodeDetails,
.build();
}
+ private int getEndPointTaskThreadPoolSize() {
+ // TODO(runzhiwang): current only support one recon, if support multiple
+ // recon in future reconServerCount should be the real number of recon
+ int reconServerCount = 1;
+ int totalServerCount = reconServerCount;
+
+ try {
+ totalServerCount += HddsUtils.getSCMAddresses(conf).size();
+ } catch (Exception e) {
+ LOG.error("Fail to get scm addresses", e);
+ }
+
+ return totalServerCount;
+ }
+
/**
*
* Return DatanodeDetails if set, return null otherwise.
@@ -207,19 +226,26 @@ private void start() throws IOException {
nextHB.set(Time.monotonicNow() + heartbeatFrequency);
context.execute(executorService, heartbeatFrequency,
TimeUnit.MILLISECONDS);
- now = Time.monotonicNow();
- if (now < nextHB.get()) {
- if(!Thread.interrupted()) {
- Thread.sleep(nextHB.get() - now);
- }
- }
} catch (InterruptedException e) {
// Some one has sent interrupt signal, this could be because
// 1. Trigger heartbeat immediately
// 2. Shutdown has be initiated.
+ LOG.warn("Interrupt the execution.", e);
+ Thread.currentThread().interrupt();
} catch (Exception e) {
LOG.error("Unable to finish the execution.", e);
}
+
+ now = Time.monotonicNow();
+ if (now < nextHB.get()) {
+ if(!Thread.interrupted()) {
+ try {
+ Thread.sleep(nextHB.get() - now);
+ } catch (InterruptedException e) {
+ LOG.warn("Interrupt the execution.", e);
+ }
+ }
+ }
}
// If we have got some exception in stateMachine we set the state to
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
index f3a599d1f21e..4cd769f4d245 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java
@@ -28,6 +28,7 @@
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicLong;
@@ -35,6 +36,7 @@
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Consumer;
+import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.CommandStatus.Status;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerAction;
@@ -51,6 +53,8 @@
import com.google.protobuf.GeneratedMessage;
import static java.lang.Math.min;
import org.apache.commons.collections.CollectionUtils;
+
+import static org.apache.hadoop.hdds.utils.HddsServerUtil.getLogWarnInterval;
import static org.apache.hadoop.hdds.utils.HddsServerUtil.getScmHeartbeatInterval;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -74,6 +78,7 @@ public class StateContext {
private DatanodeStateMachine.DatanodeStates state;
private boolean shutdownOnError = false;
private boolean shutdownGracefully = false;
+ private final AtomicLong threadPoolNotAvailableCount;
/**
* Starting with a 2 sec heartbeat frequency which will be updated to the
@@ -103,6 +108,7 @@ public StateContext(ConfigurationSource conf,
pipelineActions = new HashMap<>();
lock = new ReentrantLock();
stateExecutionCount = new AtomicLong(0);
+ threadPoolNotAvailableCount = new AtomicLong(0);
}
/**
@@ -393,6 +399,20 @@ public DatanodeState getTask() {
}
}
+ @VisibleForTesting
+ public boolean isThreadPoolAvailable(ExecutorService executor) {
+ if (!(executor instanceof ThreadPoolExecutor)) {
+ return true;
+ }
+
+ ThreadPoolExecutor ex = (ThreadPoolExecutor) executor;
+ if (ex.getQueue().size() == 0) {
+ return true;
+ }
+
+ return false;
+ }
+
/**
* Executes the required state function.
*
@@ -415,6 +435,17 @@ public void execute(ExecutorService service, long time, TimeUnit unit)
if (this.isEntering()) {
task.onEnter();
}
+
+ if (!isThreadPoolAvailable(service)) {
+ long count = threadPoolNotAvailableCount.getAndIncrement();
+ if (count % getLogWarnInterval(conf) == 0) {
+ LOG.warn("No available thread in pool for past {} seconds.",
+ unit.toSeconds(time) * (count + 1));
+ }
+ return;
+ }
+
+ threadPoolNotAvailableCount.set(0);
task.execute(service);
DatanodeStateMachine.DatanodeStates newState = task.await(time, unit);
if (this.state != newState) {
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CreatePipelineCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CreatePipelineCommandHandler.java
index c60c1129f563..78059fee78f3 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CreatePipelineCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CreatePipelineCommandHandler.java
@@ -98,7 +98,8 @@ public void handle(SCMCommand command, OzoneContainer ozoneContainer,
d -> !d.getUuid().equals(dn.getUuid()))
.forEach(d -> {
final RaftPeer peer = RatisHelper.toRaftPeer(d);
- try (RaftClient client = RatisHelper.newRaftClient(peer, conf)) {
+ try (RaftClient client = RatisHelper.newRaftClient(peer, conf,
+ ozoneContainer.getTlsClientConfig())) {
client.groupAdd(group, peer.getId());
} catch (AlreadyExistsException ae) {
// do not log
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
index 64cc804f0004..bd5e7b7c467c 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/DeleteBlocksCommandHandler.java
@@ -95,7 +95,6 @@ public void handle(SCMCommand command, OzoneContainer container,
return;
}
LOG.debug("Processing block deletion command.");
- invocationCount++;
// move blocks to deleting state.
// this is a metadata update, the actual deletion happens in another
@@ -177,6 +176,7 @@ public void handle(SCMCommand command, OzoneContainer container,
updateCommandStatus(context, command, statusUpdater, LOG);
long endTime = Time.monotonicNow();
totalTime += endTime - startTime;
+ invocationCount++;
}
}
@@ -253,28 +253,30 @@ private void deleteKeyValueContainerBlocks(
}
}
- // Finally commit the DB counters.
- BatchOperation batchOperation = new BatchOperation();
+ if (newDeletionBlocks > 0) {
+ // Finally commit the DB counters.
+ BatchOperation batchOperation = new BatchOperation();
- // In memory is updated only when existing delete transactionID is
- // greater.
- if (delTX.getTxID() > containerData.getDeleteTransactionId()) {
- // Update in DB pending delete key count and delete transaction ID.
- batchOperation.put(DB_CONTAINER_DELETE_TRANSACTION_KEY,
- Longs.toByteArray(delTX.getTxID()));
- }
+ // In memory is updated only when existing delete transactionID is
+ // greater.
+ if (delTX.getTxID() > containerData.getDeleteTransactionId()) {
+ // Update in DB pending delete key count and delete transaction ID.
+ batchOperation.put(DB_CONTAINER_DELETE_TRANSACTION_KEY,
+ Longs.toByteArray(delTX.getTxID()));
+ }
- batchOperation.put(DB_PENDING_DELETE_BLOCK_COUNT_KEY, Longs.toByteArray(
- containerData.getNumPendingDeletionBlocks() + newDeletionBlocks));
+ batchOperation.put(DB_PENDING_DELETE_BLOCK_COUNT_KEY, Longs.toByteArray(
+ containerData.getNumPendingDeletionBlocks() + newDeletionBlocks));
- containerDB.getStore().writeBatch(batchOperation);
+ containerDB.getStore().writeBatch(batchOperation);
- // update pending deletion blocks count and delete transaction ID in
- // in-memory container status
- containerData.updateDeleteTransactionId(delTX.getTxID());
+ // update pending deletion blocks count and delete transaction ID in
+ // in-memory container status
+ containerData.updateDeleteTransactionId(delTX.getTxID());
- containerData.incrPendingDeletionBlocks(newDeletionBlocks);
+ containerData.incrPendingDeletionBlocks(newDeletionBlocks);
+ }
}
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
index 8a9bcaff3afb..b0cfb4ce001a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/RunningDatanodeState.java
@@ -16,6 +16,7 @@
*/
package org.apache.hadoop.ozone.container.common.states.datanode;
+import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
import org.apache.hadoop.ozone.container.common.statemachine.EndpointStateMachine;
@@ -42,7 +43,6 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
/**
* Class that implements handshake with SCM.
@@ -152,6 +152,11 @@ public void execute(ExecutorService executor) {
}
}
+ @VisibleForTesting
+ public void setExecutorCompletionService(ExecutorCompletionService e) {
+ this.ecs = e;
+ }
+
private Callable getEndPointTask(
EndpointStateMachine endpoint) {
if (endpointTasks.containsKey(endpoint)) {
@@ -200,10 +205,11 @@ private Callable getEndPointTask(
@Override
public DatanodeStateMachine.DatanodeStates
await(long duration, TimeUnit timeUnit)
- throws InterruptedException, ExecutionException, TimeoutException {
+ throws InterruptedException {
int count = connectionManager.getValues().size();
int returned = 0;
- long timeLeft = timeUnit.toMillis(duration);
+ long durationMS = timeUnit.toMillis(duration);
+ long timeLeft = durationMS;
long startTime = Time.monotonicNow();
List> results = new LinkedList<>();
@@ -214,7 +220,7 @@ private Callable getEndPointTask(
results.add(result);
returned++;
}
- timeLeft = timeLeft - (Time.monotonicNow() - startTime);
+ timeLeft = durationMS - (Time.monotonicNow() - startTime);
}
return computeNextContainerState(results);
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java
index 01f463c5cddd..d8dfefdb5f64 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java
@@ -24,7 +24,8 @@
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.PipelineReport;
-
+import org.apache.hadoop.hdds.protocol.proto
+ .StorageContainerDatanodeProtocolProtos.MetadataStorageReportProto;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
@@ -83,4 +84,13 @@ default void removeGroup(HddsProtos.PipelineID pipelineId)
* @return list of report for each pipeline.
*/
List getPipelineReport();
+
+ /**
+ * Get storage report for the XceiverServer instance.
+ * @return list of report for each storage location.
+ */
+ default List getStorageReport() throws
+ IOException {
+ return null;
+ }
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
index b1c8370a48d6..840d87ca6d8d 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
@@ -103,7 +103,7 @@
* processed in 2 phases. The 2 phases are divided in
* {@link #startTransaction(RaftClientRequest)}, in the first phase the user
* data is written directly into the state machine via
- * {@link #writeStateMachineData} and in the second phase the
+ * {@link #write} and in the second phase the
* transaction is committed via {@link #applyTransaction(TransactionContext)}
*
* For the requests with no stateMachine data, the transaction is directly
@@ -115,7 +115,7 @@
* the write chunk operation will fail otherwise as the container still hasn't
* been created. Hence the create container operation has been split in the
* {@link #startTransaction(RaftClientRequest)}, this will help in synchronizing
- * the calls in {@link #writeStateMachineData}
+ * the calls in {@link #write}
*
* 2) Write chunk commit operation is executed after write chunk state machine
* operation. This will ensure that commit operation is sync'd with the state
@@ -517,7 +517,7 @@ private ExecutorService getChunkExecutor(WriteChunkRequestProto req) {
* and also with applyTransaction.
*/
@Override
- public CompletableFuture writeStateMachineData(LogEntryProto entry) {
+ public CompletableFuture write(LogEntryProto entry) {
try {
metrics.incNumWriteStateMachineOps();
long writeStateMachineStartTime = Time.monotonicNowNanos();
@@ -618,7 +618,7 @@ private ByteString readStateMachineData(
* @return Combined future of all writeChunks till the log index given.
*/
@Override
- public CompletableFuture flushStateMachineData(long index) {
+ public CompletableFuture flush(long index) {
List> futureList =
writeChunkFutureMap.entrySet().stream().filter(x -> x.getKey() <= index)
.map(Map.Entry::getValue).collect(Collectors.toList());
@@ -632,7 +632,7 @@ public CompletableFuture flushStateMachineData(long index) {
* evicted.
*/
@Override
- public CompletableFuture readStateMachineData(
+ public CompletableFuture read(
LogEntryProto entry) {
StateMachineLogEntryProto smLogEntryProto = entry.getStateMachineLogEntry();
metrics.incNumReadStateMachineOps();
@@ -833,7 +833,7 @@ private static CompletableFuture completeExceptionally(Exception e) {
}
@Override
- public CompletableFuture truncateStateMachineData(long index) {
+ public CompletableFuture truncate(long index) {
stateMachineDataCache.removeIf(k -> k >= index);
return CompletableFuture.completedFuture(null);
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
index 53fa2d8e5263..c1d8df66e318 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
@@ -22,7 +22,6 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
-import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
@@ -30,6 +29,7 @@
import java.util.Objects;
import java.util.Set;
import java.util.UUID;
+import java.util.EnumMap;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
@@ -46,6 +46,7 @@
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.MetadataStorageReportProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ClosePipelineInfo;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineAction;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReport;
@@ -59,11 +60,14 @@
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.impl.StorageLocationReport;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi;
import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
import org.apache.hadoop.ozone.container.ozoneimpl.ContainerController;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableList;
@@ -136,6 +140,11 @@ private static long nextCallId() {
// Timeout used while calling submitRequest directly.
private long requestTimeout;
+ /**
+ * Maintains a list of active volumes per StorageType.
+ */
+ private EnumMap> ratisVolumeMap;
+
private XceiverServerRatis(DatanodeDetails dd, int port,
ContainerDispatcher dispatcher, ContainerController containerController,
StateContext context, GrpcTlsConfig tlsConfig, ConfigurationSource conf)
@@ -163,6 +172,7 @@ private XceiverServerRatis(DatanodeDetails dd, int port,
HddsConfigKeys.HDDS_DATANODE_RATIS_SERVER_REQUEST_TIMEOUT,
HddsConfigKeys.HDDS_DATANODE_RATIS_SERVER_REQUEST_TIMEOUT_DEFAULT,
TimeUnit.MILLISECONDS);
+ initializeRatisVolumeMap();
}
private ContainerStateMachine getStateMachine(RaftGroupId gid) {
@@ -213,9 +223,12 @@ private RaftProperties newRaftProperties() {
setNodeFailureTimeout(properties);
// Set the ratis storage directory
- String storageDir = HddsServerUtil.getOzoneDatanodeRatisDirectory(conf);
- RaftServerConfigKeys.setStorageDir(properties,
- Collections.singletonList(new File(storageDir)));
+ Collection storageDirPaths =
+ HddsServerUtil.getOzoneDatanodeRatisDirectory(conf);
+ List storageDirs= new ArrayList<>(storageDirPaths.size());
+ storageDirPaths.stream().forEach(d -> storageDirs.add(new File(d)));
+
+ RaftServerConfigKeys.setStorageDir(properties, storageDirs);
// For grpc set the maximum message size
GrpcConfigKeys.setMessageSizeMax(properties,
@@ -409,12 +422,14 @@ public static XceiverServerRatis newXceiverServerRatis(
// In summary:
// authenticate from server to client is via TLS.
// authenticate from client to server is via block token (or container token).
+ // DN Ratis server act as both SSL client and server and we must pass TLS
+ // configuration for both.
static GrpcTlsConfig createTlsServerConfigForDN(SecurityConfig conf,
CertificateClient caClient) {
if (conf.isSecurityEnabled() && conf.isGrpcTlsEnabled()) {
return new GrpcTlsConfig(
caClient.getPrivateKey(), caClient.getCertificate(),
- null, false);
+ caClient.getCACertificate(), false);
}
return null;
}
@@ -526,6 +541,43 @@ public void submitRequest(ContainerCommandRequestProto request,
}
}
+ private void initializeRatisVolumeMap() throws IOException {
+ ratisVolumeMap = new EnumMap<>(StorageType.class);
+ Collection rawLocations = HddsServerUtil.
+ getOzoneDatanodeRatisDirectory(conf);
+
+ for (String locationString : rawLocations) {
+ try {
+ StorageLocation location = StorageLocation.parse(locationString);
+ StorageType type = location.getStorageType();
+ ratisVolumeMap.computeIfAbsent(type, k -> new ArrayList(1));
+ ratisVolumeMap.get(location.getStorageType()).
+ add(location.getUri().getPath());
+
+ } catch (IOException e) {
+ LOG.error("Failed to parse the storage location: " +
+ locationString, e);
+ }
+ }
+ }
+
+ @Override
+ public List getStorageReport()
+ throws IOException {
+ List reportProto = new ArrayList<>();
+ for (StorageType storageType : ratisVolumeMap.keySet()) {
+ for (String path : ratisVolumeMap.get(storageType)) {
+ MetadataStorageReportProto.Builder builder = MetadataStorageReportProto.
+ newBuilder();
+ builder.setStorageLocation(path);
+ builder.setStorageType(StorageLocationReport.
+ getStorageTypeProto(storageType));
+ reportProto.add(builder.build());
+ }
+ }
+ return reportProto;
+ }
+
private RaftClientRequest createRaftClientRequest(
ContainerCommandRequestProto request, HddsProtos.PipelineID pipelineID,
RaftClientRequest.Type type) {
@@ -552,6 +604,14 @@ private void handlePipelineFailure(RaftGroupId groupId,
msg = datanode + " is in candidate state for " +
roleInfoProto.getCandidateInfo().getLastLeaderElapsedTimeMs() + "ms";
break;
+ case FOLLOWER:
+ msg = datanode + " closes pipeline when installSnapshot from leader " +
+ "because leader snapshot doesn't contain any data to replay, " +
+ "all the log entries prior to the snapshot might have been purged." +
+ "So follower should not try to install snapshot from leader but" +
+ "can close the pipeline here. It's in follower state for " +
+ roleInfoProto.getRoleElapsedTimeMs() + "ms";
+ break;
case LEADER:
StringBuilder sb = new StringBuilder();
sb.append(datanode).append(" has not seen follower/s");
@@ -782,8 +842,10 @@ private static List createChunkExecutors(
ConfigurationSource conf) {
// TODO create single pool with N threads if using non-incremental chunks
final int threadCountPerDisk = conf.getInt(
- OzoneConfigKeys.DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_KEY,
- OzoneConfigKeys.DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_DEFAULT);
+ OzoneConfigKeys
+ .DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_PER_VOLUME_KEY,
+ OzoneConfigKeys
+ .DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_PER_VOLUME_DEFAULT);
final int numberOfDisks =
MutableVolumeSet.getDatanodeStorageDirs(conf).size();
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
index d2d29018b32a..f4d8f43f7065 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/utils/ContainerCache.java
@@ -23,6 +23,7 @@
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
+import com.google.common.util.concurrent.Striped;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.utils.MetadataStore;
import org.apache.hadoop.hdds.utils.MetadataStoreBuilder;
@@ -43,12 +44,14 @@ public final class ContainerCache extends LRUMap {
private final Lock lock = new ReentrantLock();
private static ContainerCache cache;
private static final float LOAD_FACTOR = 0.75f;
+ private final Striped rocksDBLock;
/**
* Constructs a cache that holds DBHandle references.
*/
- private ContainerCache(int maxSize, float loadFactor, boolean
+ private ContainerCache(int maxSize, int stripes, float loadFactor, boolean
scanUntilRemovable) {
super(maxSize, loadFactor, scanUntilRemovable);
+ rocksDBLock = Striped.lazyWeakLock(stripes);
}
/**
@@ -63,7 +66,10 @@ public synchronized static ContainerCache getInstance(
if (cache == null) {
int cacheSize = conf.getInt(OzoneConfigKeys.OZONE_CONTAINER_CACHE_SIZE,
OzoneConfigKeys.OZONE_CONTAINER_CACHE_DEFAULT);
- cache = new ContainerCache(cacheSize, LOAD_FACTOR, true);
+ int stripes = conf.getInt(
+ OzoneConfigKeys.OZONE_CONTAINER_CACHE_LOCK_STRIPES,
+ OzoneConfigKeys.OZONE_CONTAINER_CACHE_LOCK_STRIPES_DEFAULT);
+ cache = new ContainerCache(cacheSize, stripes, LOAD_FACTOR, true);
}
return cache;
}
@@ -117,30 +123,57 @@ public ReferenceCountedDB getDB(long containerID, String containerDBType,
throws IOException {
Preconditions.checkState(containerID >= 0,
"Container ID cannot be negative.");
- lock.lock();
+ ReferenceCountedDB db;
+ Lock containerLock = rocksDBLock.get(containerDBPath);
+ containerLock.lock();
try {
- ReferenceCountedDB db = (ReferenceCountedDB) this.get(containerDBPath);
+ lock.lock();
+ try {
+ db = (ReferenceCountedDB) this.get(containerDBPath);
+ if (db != null) {
+ db.incrementReference();
+ return db;
+ }
+ } finally {
+ lock.unlock();
+ }
- if (db == null) {
+ try {
MetadataStore metadataStore =
MetadataStoreBuilder.newBuilder()
- .setDbFile(new File(containerDBPath))
- .setCreateIfMissing(false)
- .setConf(conf)
- .setDBType(containerDBType)
- .build();
+ .setDbFile(new File(containerDBPath))
+ .setCreateIfMissing(false)
+ .setConf(conf)
+ .setDBType(containerDBType)
+ .build();
db = new ReferenceCountedDB(metadataStore, containerDBPath);
- this.put(containerDBPath, db);
+ } catch (Exception e) {
+ LOG.error("Error opening DB. Container:{} ContainerPath:{}",
+ containerID, containerDBPath, e);
+ throw e;
+ }
+
+ lock.lock();
+ try {
+ ReferenceCountedDB currentDB =
+ (ReferenceCountedDB) this.get(containerDBPath);
+ if (currentDB != null) {
+ // increment the reference before returning the object
+ currentDB.incrementReference();
+ // clean the db created in previous step
+ db.cleanup();
+ return currentDB;
+ } else {
+ this.put(containerDBPath, db);
+ // increment the reference before returning the object
+ db.incrementReference();
+ return db;
+ }
+ } finally {
+ lock.unlock();
}
- // increment the reference before returning the object
- db.incrementReference();
- return db;
- } catch (Exception e) {
- LOG.error("Error opening DB. Container:{} ContainerPath:{}",
- containerID, containerDBPath, e);
- throw e;
} finally {
- lock.unlock();
+ containerLock.unlock();
}
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MutableVolumeSet.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MutableVolumeSet.java
index bc61811c868b..b8c606738ef1 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MutableVolumeSet.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/MutableVolumeSet.java
@@ -36,8 +36,6 @@
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.fs.SpaceUsageCheckFactory;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
import org.apache.hadoop.ozone.common.InconsistentStorageStateException;
import org.apache.hadoop.ozone.container.common.impl.StorageLocationReport;
@@ -487,7 +485,7 @@ public Map> getVolumeStateMap() {
return ImmutableMap.copyOf(volumeStateMap);
}
- public StorageContainerDatanodeProtocolProtos.NodeReportProto getNodeReport()
+ public StorageLocationReport[] getStorageReport()
throws IOException {
boolean failed;
this.readLock();
@@ -540,11 +538,7 @@ public StorageContainerDatanodeProtocolProtos.NodeReportProto getNodeReport()
StorageLocationReport r = builder.build();
reports[counter++] = r;
}
- NodeReportProto.Builder nrb = NodeReportProto.newBuilder();
- for (int i = 0; i < reports.length; i++) {
- nrb.addStorageReport(reports[i].getProtoBufMessage());
- }
- return nrb.build();
+ return reports;
} finally {
this.readUnlock();
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index 49b907f10ec5..a80841f60035 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -39,7 +39,6 @@
import org.apache.hadoop.hdfs.util.Canceler;
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
import org.apache.hadoop.io.nativeio.NativeIO;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.common.impl.ContainerDataYaml;
@@ -129,12 +128,8 @@ public void create(VolumeSet volumeSet, VolumeChoosingPolicy
KeyValueContainerUtil.createContainerMetaData(containerMetaDataPath,
chunksPath, dbFile, config);
- String impl = config.getTrimmed(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL,
- OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_DEFAULT);
-
//Set containerData for the KeyValueContainer.
containerData.setChunksPath(chunksPath.getPath());
- containerData.setContainerDBType(impl);
containerData.setDbFile(dbFile);
containerData.setVolume(containerVolume);
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerCheck.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerCheck.java
index 95795e64c953..d6c4ff0c5575 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerCheck.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerCheck.java
@@ -47,8 +47,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_ROCKSDB;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_TYPE_LEVELDB;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_TYPE_ROCKSDB;
/**
* Class to run integrity checks on Datanode Containers.
@@ -186,8 +186,8 @@ private void checkContainerFile() throws IOException {
}
dbType = onDiskContainerData.getContainerDBType();
- if (!dbType.equals(OZONE_METADATA_STORE_IMPL_ROCKSDB) &&
- !dbType.equals(OZONE_METADATA_STORE_IMPL_LEVELDB)) {
+ if (!dbType.equals(CONTAINER_DB_TYPE_ROCKSDB) &&
+ !dbType.equals(CONTAINER_DB_TYPE_LEVELDB)) {
String errStr = "Unknown DBType [" + dbType
+ "] in Container File for [" + containerID + "]";
throw new IOException(errStr);
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
index 373b3223a68d..68f01fbc437b 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
@@ -38,9 +38,10 @@
import java.io.File;
import java.util.List;
import java.util.Map;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
import static java.lang.Math.max;
+import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_TYPE_ROCKSDB;
import static org.apache.hadoop.ozone.OzoneConsts.DB_BLOCK_COUNT_KEY;
import static org.apache.hadoop.ozone.OzoneConsts.CHUNKS_PATH;
import static org.apache.hadoop.ozone.OzoneConsts.DB_CONTAINER_BYTES_USED_KEY;
@@ -65,14 +66,14 @@ public class KeyValueContainerData extends ContainerData {
private String metadataPath;
//Type of DB used to store key to chunks mapping
- private String containerDBType;
+ private String containerDBType = CONTAINER_DB_TYPE_ROCKSDB;
private File dbFile = null;
/**
* Number of pending deletion blocks in KeyValueContainer.
*/
- private final AtomicInteger numPendingDeletionBlocks;
+ private final AtomicLong numPendingDeletionBlocks;
private long deleteTransactionId;
@@ -97,7 +98,7 @@ public KeyValueContainerData(long id, ChunkLayOutVersion layOutVersion,
long size, String originPipelineId, String originNodeId) {
super(ContainerProtos.ContainerType.KeyValueContainer, id, layOutVersion,
size, originPipelineId, originNodeId);
- this.numPendingDeletionBlocks = new AtomicInteger(0);
+ this.numPendingDeletionBlocks = new AtomicLong(0);
this.deleteTransactionId = 0;
}
@@ -105,7 +106,7 @@ public KeyValueContainerData(ContainerData source) {
super(source);
Preconditions.checkArgument(source.getContainerType()
== ContainerProtos.ContainerType.KeyValueContainer);
- this.numPendingDeletionBlocks = new AtomicInteger(0);
+ this.numPendingDeletionBlocks = new AtomicLong(0);
this.deleteTransactionId = 0;
}
@@ -187,7 +188,7 @@ public void setContainerDBType(String containerDBType) {
*
* @param numBlocks increment number
*/
- public void incrPendingDeletionBlocks(int numBlocks) {
+ public void incrPendingDeletionBlocks(long numBlocks) {
this.numPendingDeletionBlocks.addAndGet(numBlocks);
}
@@ -196,14 +197,14 @@ public void incrPendingDeletionBlocks(int numBlocks) {
*
* @param numBlocks decrement number
*/
- public void decrPendingDeletionBlocks(int numBlocks) {
+ public void decrPendingDeletionBlocks(long numBlocks) {
this.numPendingDeletionBlocks.addAndGet(-1 * numBlocks);
}
/**
* Get the number of pending deletion blocks.
*/
- public int getNumPendingDeletionBlocks() {
+ public long getNumPendingDeletionBlocks() {
return this.numPendingDeletionBlocks.get();
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
index 53797b08b241..e0de6ff90f86 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java
@@ -420,14 +420,16 @@ ContainerCommandResponseProto handlePutBlock(
BlockData blockData = BlockData.getFromProtoBuf(data);
Preconditions.checkNotNull(blockData);
+ boolean incrKeyCount = false;
if (!request.getPutBlock().hasEof() || request.getPutBlock().getEof()) {
chunkManager.finishWriteChunks(kvContainer, blockData);
+ incrKeyCount = true;
}
long bcsId =
dispatcherContext == null ? 0 : dispatcherContext.getLogIndex();
blockData.setBlockCommitSequenceId(bcsId);
- blockManager.putBlock(kvContainer, blockData);
+ blockManager.putBlock(kvContainer, blockData, incrKeyCount);
blockDataProto = blockData.getProtoBufMessage();
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
index 2141bed143a1..2958e9e5d4f5 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/helpers/KeyValueContainerUtil.java
@@ -23,7 +23,6 @@
import java.nio.file.Paths;
import java.util.List;
-import com.google.common.primitives.Ints;
import com.google.common.primitives.Longs;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
@@ -171,7 +170,7 @@ public static void parseKVContainerData(KeyValueContainerData kvContainerData,
containerDB.getStore().get(DB_PENDING_DELETE_BLOCK_COUNT_KEY);
if (pendingDeleteBlockCount != null) {
kvContainerData.incrPendingDeletionBlocks(
- Ints.fromByteArray(pendingDeleteBlockCount));
+ Longs.fromByteArray(pendingDeleteBlockCount));
} else {
// Set pending deleted block count.
MetadataKeyFilters.KeyPrefixFilter filter =
@@ -231,10 +230,22 @@ public static void parseKVContainerData(KeyValueContainerData kvContainerData,
private static void initializeUsedBytesAndBlockCount(
KeyValueContainerData kvContainerData) throws IOException {
+ MetadataKeyFilters.KeyPrefixFilter filter =
+ new MetadataKeyFilters.KeyPrefixFilter();
+
+ // Ignore all blocks except those with no prefix, or those with
+ // #deleting# prefix.
+ filter.addFilter(OzoneConsts.DELETED_KEY_PREFIX, true)
+ .addFilter(OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX, true)
+ .addFilter(OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID_PREFIX, true)
+ .addFilter(OzoneConsts.BLOCK_COUNT, true)
+ .addFilter(OzoneConsts.CONTAINER_BYTES_USED, true)
+ .addFilter(OzoneConsts.PENDING_DELETE_BLOCK_COUNT, true);
+
long blockCount = 0;
try (KeyValueBlockIterator blockIter = new KeyValueBlockIterator(
kvContainerData.getContainerID(),
- new File(kvContainerData.getContainerPath()))) {
+ new File(kvContainerData.getContainerPath()), filter)) {
long usedBytes = 0;
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java
index 9544e29c1ecf..51fa1c9614f6 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java
@@ -83,6 +83,20 @@ public BlockManagerImpl(ConfigurationSource conf) {
* @throws IOException
*/
public long putBlock(Container container, BlockData data) throws IOException {
+ return putBlock(container, data, true);
+ }
+ /**
+ * Puts or overwrites a block.
+ *
+ * @param container - Container for which block need to be added.
+ * @param data - BlockData.
+ * @param incrKeyCount - for FilePerBlockStrategy, increase key count only
+ * when the whole block file is written.
+ * @return length of the block.
+ * @throws IOException
+ */
+ public long putBlock(Container container, BlockData data,
+ boolean incrKeyCount) throws IOException {
Preconditions.checkNotNull(data, "BlockData cannot be null for put " +
"operation.");
Preconditions.checkState(data.getContainerID() >= 0, "Container Id " +
@@ -129,14 +143,18 @@ public long putBlock(Container container, BlockData data) throws IOException {
Longs.toByteArray(container.getContainerData().getBytesUsed()));
// Set Block Count for a container.
- batch.put(DB_BLOCK_COUNT_KEY,
- Longs.toByteArray(container.getContainerData().getKeyCount() + 1));
+ if (incrKeyCount) {
+ batch.put(DB_BLOCK_COUNT_KEY,
+ Longs.toByteArray(container.getContainerData().getKeyCount() + 1));
+ }
db.getStore().writeBatch(batch);
container.updateBlockCommitSequenceId(bcsId);
// Increment block count finally here for in-memory.
- container.getContainerData().incrKeyCount();
+ if (incrKeyCount) {
+ container.getContainerData().incrKeyCount();
+ }
if (LOG.isDebugEnabled()) {
LOG.debug(
"Block " + data.getBlockID() + " successfully committed with bcsId "
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java
index 6812b0d8ff8e..72b104025b59 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java
@@ -40,6 +40,18 @@ public interface BlockManager {
*/
long putBlock(Container container, BlockData data) throws IOException;
+ /**
+ * Puts or overwrites a block.
+ *
+ * @param container - Container for which block need to be added.
+ * @param data - Block Data.
+ * @param incrKeyCount - Whether to increase container key count.
+ * @return length of the Block.
+ * @throws IOException
+ */
+ long putBlock(Container container, BlockData data, boolean incrKeyCount)
+ throws IOException;
+
/**
* Gets an existing block.
*
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
index 1b9b3d690724..fa63cf1b862e 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
@@ -120,6 +120,7 @@ public boolean accept(File pathname) {
return;
}
+ LOG.info("Start to verify containers on volume {}", hddsVolumeRootDir);
for (File scmLoc : scmDir) {
File currentDir = new File(scmLoc, Storage.STORAGE_DIR_CURRENT);
File[] containerTopDirs = currentDir.listFiles();
@@ -144,6 +145,7 @@ public boolean accept(File pathname) {
}
}
}
+ LOG.info("Finish verifying containers on volume {}", hddsVolumeRootDir);
}
private void verifyContainerFile(long containerID, File containerFile) {
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
index bbbec25af783..26da4873dcdb 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java
@@ -33,12 +33,14 @@
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.IncrementalContainerReportProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
+import org.apache.hadoop.hdds.ratis.RatisHelper;
import org.apache.hadoop.hdds.security.token.BlockTokenVerifier;
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
+import org.apache.hadoop.ozone.container.common.impl.StorageLocationReport;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
import org.apache.hadoop.ozone.container.common.interfaces.Handler;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
@@ -58,6 +60,7 @@
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT;
+import org.apache.ratis.grpc.GrpcTlsConfig;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -81,6 +84,7 @@ public class OzoneContainer {
private ContainerMetadataScanner metadataScanner;
private List dataScanners;
private final BlockDeletingService blockDeletingService;
+ private final GrpcTlsConfig tlsClientConfig;
/**
* Construct OzoneContainer object.
@@ -148,6 +152,12 @@ public OzoneContainer(DatanodeDetails datanodeDetails, ConfigurationSource
blockDeletingService =
new BlockDeletingService(this, svcInterval, serviceTimeout,
TimeUnit.MILLISECONDS, config);
+ tlsClientConfig = RatisHelper.createTlsClientConfig(
+ secConf, certClient != null ? certClient.getCACertificate() : null);
+ }
+
+ public GrpcTlsConfig getTlsClientConfig() {
+ return tlsClientConfig;
}
private GrpcReplicationService createReplicationService() {
@@ -162,6 +172,7 @@ private void buildContainerSet() {
Iterator volumeSetIterator = volumeSet.getVolumesList()
.iterator();
ArrayList volumeThreads = new ArrayList();
+ long startTime = System.currentTimeMillis();
//TODO: diskchecker should be run before this, to see how disks are.
// And also handle disk failure tolerance need to be added
@@ -182,6 +193,8 @@ private void buildContainerSet() {
Thread.currentThread().interrupt();
}
+ LOG.info("Build ContainerSet costs {}s",
+ (System.currentTimeMillis() - startTime) / 1000);
}
/**
@@ -294,8 +307,21 @@ public ContainerController getController() {
* Returns node report of container storage usage.
*/
public StorageContainerDatanodeProtocolProtos.NodeReportProto getNodeReport()
- throws IOException {
- return volumeSet.getNodeReport();
+ throws IOException {
+ StorageLocationReport[] reports = volumeSet.getStorageReport();
+ StorageContainerDatanodeProtocolProtos.NodeReportProto.Builder nrb
+ = StorageContainerDatanodeProtocolProtos.
+ NodeReportProto.newBuilder();
+ for (int i = 0; i < reports.length; i++) {
+ nrb.addStorageReport(reports[i].getProtoBufMessage());
+ }
+ List metadataReport =
+ writeChannel.getStorageReport();
+ if (metadataReport != null) {
+ nrb.addAllMetadataStorageReport(metadataReport);
+ }
+ return nrb.build();
}
@VisibleForTesting
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationClient.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationClient.java
index 660ba4ee639b..abeaf03c1667 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationClient.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/GrpcReplicationClient.java
@@ -24,6 +24,7 @@
import java.io.UncheckedIOException;
import java.nio.file.Files;
import java.nio.file.Path;
+import java.security.cert.X509Certificate;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
@@ -37,10 +38,13 @@
.IntraDatanodeProtocolServiceGrpc.IntraDatanodeProtocolServiceStub;
import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.ratis.thirdparty.io.grpc.ManagedChannel;
+import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts;
import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder;
import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
+import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContextBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -58,13 +62,26 @@ public class GrpcReplicationClient {
private final Path workingDirectory;
- public GrpcReplicationClient(String host,
- int port, Path workingDir) {
+ public GrpcReplicationClient(String host, int port, Path workingDir,
+ SecurityConfig secConfig, X509Certificate caCert) throws IOException {
+ NettyChannelBuilder channelBuilder =
+ NettyChannelBuilder.forAddress(host, port)
+ .usePlaintext()
+ .maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE);
- channel = NettyChannelBuilder.forAddress(host, port)
- .usePlaintext()
- .maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE)
- .build();
+ if (secConfig.isGrpcTlsEnabled()) {
+ channelBuilder.useTransportSecurity();
+
+ SslContextBuilder sslContextBuilder = GrpcSslContexts.forClient();
+ if (caCert != null) {
+ sslContextBuilder.trustManager(caCert);
+ }
+ if (secConfig.useTestCert()) {
+ channelBuilder.overrideAuthority("localhost");
+ }
+ channelBuilder.sslContext(sslContextBuilder.build());
+ }
+ channel = channelBuilder.build();
client = IntraDatanodeProtocolServiceGrpc.newStub(channel);
workingDirectory = workingDir;
}
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SimpleContainerDownloader.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SimpleContainerDownloader.java
index d7666ea1127b..9d7b5516a5c3 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SimpleContainerDownloader.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/replication/SimpleContainerDownloader.java
@@ -18,8 +18,10 @@
package org.apache.hadoop.ozone.container.replication;
+import java.io.IOException;
import java.nio.file.Path;
import java.nio.file.Paths;
+import java.security.cert.X509Certificate;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.function.Function;
@@ -27,6 +29,7 @@
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.DatanodeDetails.Port.Name;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.slf4j.Logger;
@@ -45,9 +48,11 @@ public class SimpleContainerDownloader implements ContainerDownloader {
LoggerFactory.getLogger(SimpleContainerDownloader.class);
private final Path workingDirectory;
+ private final SecurityConfig securityConfig;
+ private final X509Certificate caCert;
- public SimpleContainerDownloader(ConfigurationSource conf) {
-
+ public SimpleContainerDownloader(ConfigurationSource conf,
+ X509Certificate caCert) {
String workDirString =
conf.get(OzoneConfigKeys.OZONE_CONTAINER_COPY_WORKDIR);
@@ -57,6 +62,8 @@ public SimpleContainerDownloader(ConfigurationSource conf) {
} else {
workingDirectory = Paths.get(workDirString);
}
+ securityConfig = new SecurityConfig(conf);
+ this.caCert = caCert;
}
@Override
@@ -66,22 +73,27 @@ public CompletableFuture getContainerDataFromReplicas(long containerId,
CompletableFuture result = null;
for (DatanodeDetails datanode : sourceDatanodes) {
try {
-
if (result == null) {
GrpcReplicationClient grpcReplicationClient =
new GrpcReplicationClient(datanode.getIpAddress(),
datanode.getPort(Name.STANDALONE).getValue(),
- workingDirectory);
+ workingDirectory, securityConfig, caCert);
result = grpcReplicationClient.download(containerId);
} else {
result = result.thenApply(CompletableFuture::completedFuture)
.exceptionally(t -> {
LOG.error("Error on replicating container: " + containerId, t);
- GrpcReplicationClient grpcReplicationClient =
- new GrpcReplicationClient(datanode.getIpAddress(),
- datanode.getPort(Name.STANDALONE).getValue(),
- workingDirectory);
- return grpcReplicationClient.download(containerId);
+ try {
+ GrpcReplicationClient grpcReplicationClient =
+ new GrpcReplicationClient(datanode.getIpAddress(),
+ datanode.getPort(Name.STANDALONE).getValue(),
+ workingDirectory, securityConfig, caCert);
+ return grpcReplicationClient.download(containerId);
+ } catch (IOException e) {
+ LOG.error("Error on replicating container: " + containerId,
+ t);
+ return null;
+ }
}).thenCompose(Function.identity());
}
} catch (Exception ex) {
diff --git a/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/dn-overview.html b/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/dn-overview.html
new file mode 100644
index 000000000000..d4f7a17c0b76
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/dn-overview.html
@@ -0,0 +1,21 @@
+
+
+
+
+
+
diff --git a/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/dn.js b/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/dn.js
new file mode 100644
index 000000000000..c43eb42bdc25
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/dn.js
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+(function () {
+ "use strict";
+ angular.module('dn', ['ozone', 'nvd3']);
+
+ angular.module('dn').component('dnOverview', {
+ templateUrl: 'dn-overview.html',
+ require: {
+ overview: "^overview"
+ },
+ controller: function ($http) {
+ var ctrl = this;
+ $http.get("jmx?qry=Hadoop:service=HddsDatanode,name=StorageContainerMetrics")
+ .then(function (result) {
+ ctrl.dnmetrics = result.data.beans[0];
+ });
+ }
+ });
+})();
diff --git a/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/index.html b/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/index.html
new file mode 100644
index 000000000000..b1f703c0d659
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/index.html
@@ -0,0 +1,76 @@
+
+
+
+
+
+
+
+
+
+
+ HDDS Datanode Service
+
+
+
+
+
+
+
+
+
+
+
+
+
+ * 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.container.common.states.datanode;
+
+import org.apache.hadoop.ozone.container.common.statemachine.EndpointStateMachine;
+import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
+import org.apache.hadoop.util.Time;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.container.common.statemachine.EndpointStateMachine.EndPointStates.SHUTDOWN;
+import static org.mockito.Mockito.when;
+
+/**
+ * Test class for RunningDatanodeState.
+ */
+public class TestRunningDatanodeState {
+ @Test
+ public void testAwait() throws InterruptedException {
+ SCMConnectionManager connectionManager =
+ Mockito.mock(SCMConnectionManager.class);
+ List stateMachines = new ArrayList<>();
+ when(connectionManager.getValues()).thenReturn(stateMachines);
+
+ RunningDatanodeState state =
+ new RunningDatanodeState(null, connectionManager, null);
+
+ int threadPoolSize = 2;
+ ExecutorService executorService = Executors.newFixedThreadPool(
+ threadPoolSize);
+
+ ExecutorCompletionService ecs =
+ new ExecutorCompletionService<>(executorService);
+ state.setExecutorCompletionService(ecs);
+
+ for (int i = 0; i < threadPoolSize; i++) {
+ stateMachines.add(new EndpointStateMachine(null, null, null));
+ }
+
+ CompletableFuture futureOne =
+ new CompletableFuture<>();
+ for (int i = 0; i < threadPoolSize; i++) {
+ ecs.submit(() -> futureOne.get());
+ }
+
+ long startTime = Time.monotonicNow();
+ state.await(500, TimeUnit.MILLISECONDS);
+ long endTime = Time.monotonicNow();
+ Assert.assertTrue((endTime - startTime) >= 500);
+
+ futureOne.complete(SHUTDOWN);
+
+ CompletableFuture futureTwo =
+ new CompletableFuture<>();
+ for (int i = 0; i < threadPoolSize; i++) {
+ ecs.submit(() -> futureTwo.get());
+ }
+ futureTwo.complete(SHUTDOWN);
+
+ startTime = Time.monotonicNow();
+ state.await(500, TimeUnit.MILLISECONDS);
+ endTime = Time.monotonicNow();
+ Assert.assertTrue((endTime - startTime) < 500);
+
+ executorService.shutdown();
+ }
+}
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueBlockIterator.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueBlockIterator.java
index 62097b38e8b3..aff0528bdef4 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueBlockIterator.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueBlockIterator.java
@@ -45,9 +45,6 @@
import com.google.common.primitives.Longs;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_IMPL;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_ROCKSDB;
import static org.apache.hadoop.ozone.container.common.impl.ChunkLayOutVersion.FILE_PER_BLOCK;
import static org.apache.hadoop.ozone.container.common.impl.ChunkLayOutVersion.FILE_PER_CHUNK;
import org.junit.After;
@@ -71,22 +68,18 @@ public class TestKeyValueBlockIterator {
private OzoneConfiguration conf;
private File testRoot;
- private final String storeImpl;
private final ChunkLayOutVersion layout;
- public TestKeyValueBlockIterator(String metadataImpl,
- ChunkLayOutVersion layout) {
- this.storeImpl = metadataImpl;
+ public TestKeyValueBlockIterator(ChunkLayOutVersion layout) {
this.layout = layout;
}
@Parameterized.Parameters
public static Collection
-
-
-### 通用命令格式
-
-Ozone shell 命令都遵照以下格式:
-
-> _ozone sh object action url_
-
-**ozone** 脚本用来调用所有 Ozone 子命令,ozone shell 通过 ```sh``` 子命令调用。
-
-对象可以是卷、桶或键,操作一般是各种动词,比如 create、list、delete 等等。
-
-
-Ozone URL 可以指向卷、桶或键,格式如下:
-
-_\[schema\]\[server:port\]/volume/bucket/key_
-
-
-其中,
-
-1. **Schema** - 可选,默认为 `o3`,表示使用原生 RPC 协议来访问 Ozone API。
-
-2. **Server:Port** - OM 的地址,如果省略了端口, 则使用 ozone-site.xml 中的默认端口。
-
-根据具体的命令不同,卷名、桶名和键名将用来构成 URL,卷、桶和键命令的文档有更多具体的说明。
diff --git a/hadoop-hdds/docs/content/shell/KeyCommands.md b/hadoop-hdds/docs/content/shell/KeyCommands.md
deleted file mode 100644
index 11186c422184..000000000000
--- a/hadoop-hdds/docs/content/shell/KeyCommands.md
+++ /dev/null
@@ -1,177 +0,0 @@
----
-title: Key Commands
-summary: Key commands help you to manage the life cycle of
- Keys / Objects.
-weight: 4
----
-
-
-
-Ozone shell supports the following key commands.
-
- * [get](#get)
- * [put](#put)
- * [delete](#delete)
- * [info](#info)
- * [list](#list)
- * [rename](#rename)
- * [cat](#cat)
- * [copy](#cp)
-
-
-### Get
-
-The `key get` command downloads a key from Ozone cluster to local file system.
-
-***Params:***
-
-| Arguments | Comment |
-|--------------------------------|-----------------------------------------|
-| Uri | The name of the key in **/volume/bucket/key** format.
-| FileName | Local file to download the key to.
-
-
-{{< highlight bash >}}
-ozone sh key get /hive/jan/sales.orc sales.orc
-{{< /highlight >}}
-Downloads the file sales.orc from the _/hive/jan_ bucket and writes to the
-local file sales.orc.
-
-### Put
-
-The `key put` command uploads a file from the local file system to the specified bucket.
-
-***Params:***
-
-
-| Arguments | Comment |
-|--------------------------------|-----------------------------------------|
-| Uri | The name of the key in **/volume/bucket/key** format.
-| FileName | Local file to upload.
-| -r, \-\-replication | Optional, Number of copies, ONE or THREE are the options. Picks up the default from cluster configuration.
-| -t, \-\-type | Optional, replication type of the new key. RATIS and STAND_ALONE are the options. Picks up the default from cluster configuration.
-
-{{< highlight bash >}}
-ozone sh key put /hive/jan/corrected-sales.orc sales.orc
-{{< /highlight >}}
-The above command will put the sales.orc as a new key into _/hive/jan/corrected-sales.orc_.
-
-### Delete
-
-The `key delete` command removes the key from the bucket.
-
-***Params:***
-
-| Arguments | Comment |
-|--------------------------------|-----------------------------------------|
-| Uri | The name of the key.
-
-{{< highlight bash >}}
-ozone sh key delete /hive/jan/corrected-sales.orc
-{{< /highlight >}}
-
-The above command deletes the key _/hive/jan/corrected-sales.orc_.
-
-
-### Info
-
-The `key info` commands returns the information about the key.
-
-***Params:***
-
-| Arguments | Comment |
-|--------------------------------|-----------------------------------------|
-| Uri | The name of the key.
-
-{{< highlight bash >}}
-ozone sh key info /hive/jan/sales.orc
-{{< /highlight >}}
-
-The above command will print out the information about _/hive/jan/sales.orc_
-key.
-
-### List
-
-The `key list` command allows user to list all keys in a bucket.
-
-***Params:***
-
-| Arguments | Comment |
-|--------------------------------|-----------------------------------------|
-| -l, \-\-length | Maximum number of results to return. Default: 100
-| -p, \-\-prefix | Optional, Only keys that match this prefix will be returned.
-| -s, \-\-start | The listing will start from key after the start key.
-| Uri | The name of the _volume_.
-
-{{< highlight bash >}}
-ozone sh key list /hive/jan
-{{< /highlight >}}
-
-This command will list all keys in the bucket _/hive/jan_.
-
-### Rename
-
-The `key rename` command changes the name of an existing key in the specified bucket.
-
-***Params:***
-
-| Arguments | Comment |
-|--------------------------------|-----------------------------------------|
-| Uri | The name of the bucket in **/volume/bucket** format.
-| FromKey | The existing key to be renamed
-| ToKey | The new desired name of the key
-
-{{< highlight bash >}}
-ozone sh key rename /hive/jan sales.orc new_name.orc
-{{< /highlight >}}
-The above command will rename _sales.orc_ to _new\_name.orc_ in the bucket _/hive/jan_.
-
-### Cat
-
-The `key cat` command displays the contents of a specific Ozone key to standard output.
-
-***Params:***
-
-| Arguments | Comment |
-|--------------------------------|-----------------------------------------|
-| Uri | The name of the key in **/volume/bucket/key** format.
-
-
-{{< highlight bash >}}
-ozone sh key cat /hive/jan/hello.txt
-{{< /highlight >}}
-Displays the contents of the key hello.txt from the _/hive/jan_ bucket to standard output.
-
-### Cp
-
-The `key cp` command copies a key to another one in the specified bucket.
-
-***Params:***
-
-| Arguments | Comment |
-|--------------------------------|-----------------------------------------|
-| Uri | The name of the bucket in **/volume/bucket** format.
-| FromKey | The existing key to be copied
-| ToKey | The name of the new key
-| -r, \-\-replication | Optional, Number of copies, ONE or THREE are the options. Picks up the default from cluster configuration.
-| -t, \-\-type | Optional, replication type of the new key. RATIS and STAND_ALONE are the options. Picks up the default from cluster configuration.
-
-{{< highlight bash >}}
-ozone sh key cp /hive/jan sales.orc new_one.orc
-{{< /highlight >}}
-The above command will copy _sales.orc_ to _new\_one.orc_ in the bucket _/hive/jan_.
\ No newline at end of file
diff --git a/hadoop-hdds/docs/content/shell/KeyCommands.zh.md b/hadoop-hdds/docs/content/shell/KeyCommands.zh.md
deleted file mode 100644
index 2a36e7324f31..000000000000
--- a/hadoop-hdds/docs/content/shell/KeyCommands.zh.md
+++ /dev/null
@@ -1,176 +0,0 @@
----
-title: 键命令
-summary: 用键命令管理键/对象的生命周期
-weight: 4
----
-
-
-
-Ozone shell 提供以下键命令:
-
- * [下载](#下载)
- * [上传](#上传)
- * [删除](#删除)
- * [查看](#查看)
- * [列举](#列举)
- * [重命名](#重命名)
- * [Cat](#cat)
- * [Cp](#cp)
-
-
-### 下载
-
-`key get` 命令从 Ozone 集群下载一个键到本地文件系统。
-
-***参数:***
-
-| 参数名 | 说明 |
-|--------------------------------|-----------------------------------------|
-| Uri | 键名,格式为 **/volume/bucket/key**
-| FileName | 下载到本地后的文件名
-
-
-{{< highlight bash >}}
-ozone sh key get /hive/jan/sales.orc sales.orc
-{{< /highlight >}}
-
-从 _/hive/jan_ 桶中下载 sales.orc 文件,写入到本地名为 sales.orc 的文件。
-
-### 上传
-
-`key put` 命令从本地文件系统上传一个文件到指定的桶。
-
-***参数:***
-
-| 参数名 | 说明 |
-|--------------------------------|-----------------------------------------|
-| Uri | 键名,格式为 **/volume/bucket/key**
-| FileName | 待上传的本地文件
-| -r, \-\-replication | 可选,上传后的副本数,合法值为 ONE 或者 THREE,如果不设置,将采用集群配置中的默认值。
-| -t, \-\-type | 可选,副本类型,合法值为 RATIS 或 STAND_ALONE,如果不设置,将采用集群配置中的默认值。
-
-{{< highlight bash >}}
-ozone sh key put /hive/jan/corrected-sales.orc sales.orc
-{{< /highlight >}}
-
-上述命令将 sales.orc 文件作为新键上传到 _/hive/jan/corrected-sales.orc_ 。
-
-### 删除
-
-`key delete` 命令用来从桶中删除指定键。
-
-***参数:***
-
-| 参数名 | 说明 |
-|--------------------------------|-----------------------------------------|
-| Uri | 键名
-
-{{< highlight bash >}}
-ozone sh key delete /hive/jan/corrected-sales.orc
-{{< /highlight >}}
-
-上述命令会将 _/hive/jan/corrected-sales.orc_ 这个键删除。
-
-
-### 查看
-
-`key info` 命令返回指定键的信息。
-
-***参数:***
-
-| 参数名 | 说明 |
-|--------------------------------|-----------------------------------------|
-| Uri | 键名
-
-{{< highlight bash >}}
-ozone sh key info /hive/jan/sales.orc
-{{< /highlight >}}
-
-上述命令会打印出 _/hive/jan/sales.orc_ 键的相关信息。
-
-### 列举
-
-用户通过 `key list` 命令列出一个桶中的所有键。
-
-***参数:***
-
-| 参数名 | 说明 |
-|--------------------------------|-----------------------------------------|
-| -l, \-\-length | 返回结果的最大数量,默认值为 100
-| -p, \-\-prefix | 可选,只有匹配指定前缀的键会被返回
-| -s, \-\-start | 从指定键开始列举
-| Uri | 桶名
-
-{{< highlight bash >}}
-ozone sh key list /hive/jan
-{{< /highlight >}}
-
-此命令会列出 _/hive/jan_ 桶中的所有键。
-
-### 重命名
-
-`key rename` 命令用来修改指定桶中的已有键的键名。
-
-***参数:***
-
-| 参数名 | 说明 |
-|--------------------------------|-----------------------------------------|
-| Uri | 桶名,格式为 **/volume/bucket**
-| FromKey | 旧的键名
-| ToKey | 新的键名
-
-{{< highlight bash >}}
-ozone sh key rename /hive/jan sales.orc new_name.orc
-{{< /highlight >}}
-
-上述命令会将 _/hive/jan_ 桶中的 _sales.orc_ 重命名为 _new\_name.orc_ 。
-
-### Cat
-
-`key cat` 命令用来把指定的键的内容输出到终端。
-
-***参数:***
-
-| 参数名 | 说明 |
-|--------------------------------|-----------------------------------------|
-| Uri | 键名,格式为 **/volume/bucket/key**
-
-
-{{< highlight bash >}}
-ozone sh key cat /hive/jan/hello.txt
-{{< /highlight >}}
-上述命令会将 _/hive/jan_ 桶中的 hello.txt 的内容输出到标准输出中来。
-
-### Cp
-
-`key cp` 命令用来在同一个bucket下,从一个key复制出另一个key。
-
-***Params:***
-
-| 参数名 | 说明 |
-|--------------------------------|-----------------------------------------|
-| Uri | 桶名 格式为**/volume/bucket**。
-| FromKey | 现有的键名
-| ToKey | 新的键名
-| -r, \-\-replication | 可选,上传后的副本数,合法值为 ONE 或者 THREE,如果不设置,将采用集群配置中的默认值。
-| -t, \-\-type | 可选,副本类型,合法值为 RATIS 或 STAND_ALONE,如果不设置,将采用集群配置中的默认值。
-
-{{< highlight bash >}}
-ozone sh key cp /hive/jan sales.orc new_one.orc
-{{< /highlight >}}
-上述命令会将 _/hive/jan_ 桶中的 _sales.orc_ 复制到 _new\_one.orc_ 。
\ No newline at end of file
diff --git a/hadoop-hdds/docs/content/shell/VolumeCommands.md b/hadoop-hdds/docs/content/shell/VolumeCommands.md
deleted file mode 100644
index fe459f313352..000000000000
--- a/hadoop-hdds/docs/content/shell/VolumeCommands.md
+++ /dev/null
@@ -1,114 +0,0 @@
----
-title: Volume Commands
-weight: 2
-summary: Volume commands help you to manage the life cycle of a volume.
----
-
-
-Volume commands generally need administrator privileges. The ozone shell supports the following volume commands.
-
- * [create](#create)
- * [delete](#delete)
- * [info](#info)
- * [list](#list)
- * [update](#update)
-
-### Create
-
-The `volume create` command allows an administrator to create a volume and
-assign it to a user.
-
-***Params:***
-
-| Arguments | Comment |
-|--------------------------------|-----------------------------------------|
-| -q, \-\-quota | Optional, This argument that specifies the maximum size this volume can use in the Ozone cluster. |
-| -u, \-\-user | Required, The name of the user who owns this volume. This user can create, buckets and keys on this volume. |
-| Uri | The name of the volume. |
-
-{{< highlight bash >}}
-ozone sh volume create --quota=1TB --user=bilbo /hive
-{{< /highlight >}}
-
-The above command will create a volume called _hive_ on the ozone cluster. This
-volume has a quota of 1TB, and the owner is _bilbo_.
-
-### Delete
-
-The `volume delete` command allows an administrator to delete a volume. If the
-volume is not empty then this command will fail.
-
-***Params:***
-
-| Arguments | Comment |
-|--------------------------------|-----------------------------------------|
-| Uri | The name of the volume.
-
-{{< highlight bash >}}
-ozone sh volume delete /hive
-{{< /highlight >}}
-
-The above command will delete the volume hive, if the volume has no buckets
-inside it.
-
-### Info
-
-The `volume info` commands returns the information about the volume including
-quota and owner information.
-
-***Params:***
-
-| Arguments | Comment |
-|--------------------------------|-----------------------------------------|
-| Uri | The name of the volume.
-
-{{< highlight bash >}}
-ozone sh volume info /hive
-{{< /highlight >}}
-
-The above command will print out the information about hive volume.
-
-### List
-
-The `volume list` command will list the volumes accessible by a user.
-
-{{< highlight bash >}}
-ozone sh volume list --user hadoop
-{{< /highlight >}}
-
-When ACL is enabled, the above command will print out volumes that the user
-hadoop has LIST permission to. When ACL is disabled, the above command will
-print out all the volumes owned by the user hadoop.
-
-### Update
-
-The volume update command allows changing of owner and quota on a given volume.
-
-***Params:***
-
-| Arguments | Comment |
-|--------------------------------|-----------------------------------------|
-| -q, \-\-quota | Optional, This argument that specifies the maximum size this volume can use in the Ozone cluster. |
-| -u, \-\-user | Optional, The name of the user who owns this volume. This user can create, buckets and keys on this volume. |
-| Uri | The name of the volume. |
-
-{{< highlight bash >}}
-ozone sh volume update --quota=10TB /hive
-{{< /highlight >}}
-
-The above command updates the volume quota to 10TB.
diff --git a/hadoop-hdds/docs/content/shell/VolumeCommands.zh.md b/hadoop-hdds/docs/content/shell/VolumeCommands.zh.md
deleted file mode 100644
index 190e0994e74c..000000000000
--- a/hadoop-hdds/docs/content/shell/VolumeCommands.zh.md
+++ /dev/null
@@ -1,108 +0,0 @@
----
-title: 卷命令
-weight: 2
-summary: 用卷命令管理卷的生命周期
----
-
-
-卷命令通常需要管理员权限,ozone shell 支持以下卷命令:
-
- * [创建](#创建)
- * [删除](#删除)
- * [查看](#查看)
- * [列举](#列举)
- * [更新](#更新)
-
-### 创建
-
-管理员可以通过 `volume create` 命令创建一个卷并分配给一个用户。
-
-***参数:***
-
-| 参数名 | 说明 |
-|--------------------------------|-----------------------------------------|
-| -q, \-\-quota | 可选,指明该卷在 Ozone 集群所能使用的最大空间,即限额。 |
-| -u, \-\-user | 必需,指明该卷的所有者,此用户可以在该卷中创建桶和键。 |
-| Uri | 卷名 |
-
-{{< highlight bash >}}
-ozone sh volume create --quota=1TB --user=bilbo /hive
-{{< /highlight >}}
-
-上述命令会在 ozone 集群中创建名为 _hive_ 的卷,卷的限额为 1TB,所有者为 _bilbo_ 。
-
-### 删除
-
-管理员可以通过 `volume delete` 命令删除一个卷,如果卷不为空,此命令将失败。
-
-***参数***
-
-| 参数名 | 说明 |
-|--------------------------------|-----------------------------------------|
-| Uri | 卷名 |
-
-{{< highlight bash >}}
-ozone sh volume delete /hive
-{{< /highlight >}}
-
-如果 hive 卷中不包含任何桶,上述命令将删除 hive 卷。
-
-### 查看
-
-通过 `volume info` 命令可以获取卷的限额和所有者信息。
-
-***参数:***
-
-| 参数名 | 说明 |
-|--------------------------------|-----------------------------------------|
-| Uri | 卷名 |
-
-{{< highlight bash >}}
-ozone sh volume info /hive
-{{< /highlight >}}
-
-上述命令会打印出 hive 卷的相关信息。
-
-### 列举
-
-`volume list` 命令用来列举一个用户可以访问的所有卷。
-
-{{< highlight bash >}}
-ozone sh volume list --user hadoop
-{{< /highlight >}}
-
-若 ACL 已启用,上述命令会打印出 hadoop 用户有 LIST 权限的所有卷。
-若 ACL 被禁用,上述命令会打印出 hadoop 用户拥有的所有卷。
-
-### 更新
-
-`volume update` 命令用来修改卷的所有者和限额。
-
-***参数***
-
-| 参数名 | 说明 |
-|--------------------------------|-----------------------------------------|
-| -q, \-\-quota | 可选,重新指定该卷在 Ozone 集群中的限额。 |
-| -u, \-\-user | 可选,重新指定该卷的所有者 |
-| Uri | 卷名 |
-
-{{< highlight bash >}}
-ozone sh volume update --quota=10TB /hive
-{{< /highlight >}}
-
-上述命令将 hive 卷的限额更新为 10TB。
diff --git a/hadoop-hdds/docs/content/shell/_index.zh.md b/hadoop-hdds/docs/content/shell/_index.zh.md
deleted file mode 100644
index 0f6220b5f0e6..000000000000
--- a/hadoop-hdds/docs/content/shell/_index.zh.md
+++ /dev/null
@@ -1,27 +0,0 @@
----
-title: 命令行接口
-menu:
- main:
- weight: 3
----
-
-
-
-{{}}
- Ozone shell 是用户与 Ozone 进行交互的主要接口,它提供了操作 Ozone 的命令行接口。
-{{}}
diff --git a/hadoop-hdds/docs/content/start/FromSource.md b/hadoop-hdds/docs/content/start/FromSource.md
index 9ce0cc4b6a8f..80f47fb78f0b 100644
--- a/hadoop-hdds/docs/content/start/FromSource.md
+++ b/hadoop-hdds/docs/content/start/FromSource.md
@@ -22,18 +22,21 @@ weight: 30
{{< requirements >}}
* Java 1.8
* Maven
- * Protoc (2.5)
{{< /requirements >}}
-
This is a guide on how to build the ozone sources. If you are
+
+This is a guide on how to build the ozone sources. If you are not
-planning to build sources yourself, you can safely skip this page.
+planning to build sources yourself, you can safely skip this page.
+
+
If you are a Hadoop ninja, and wise in the ways of Apache, you already know
that a real Apache release is a source release.
-If you want to build from sources, Please untar the source tarball and run
-the ozone build command. This instruction assumes that you have all the
+If you want to build from sources, Please untar the source tarball (or clone the latest code
+from the [git repository](https://github.com/apache/hadoop-ozone)) and run the ozone build command. This instruction assumes that you have all the
dependencies to build Hadoop on your build machine. If you need instructions
on how to build Hadoop, please look at the Apache Hadoop Website.
@@ -41,28 +44,27 @@ on how to build Hadoop, please look at the Apache Hadoop Website.
mvn clean package -DskipTests=true
```
-This will build an ozone-\.tar.gz in your `hadoop-ozone/dist/target` directory.
+This will build an `ozone-\` directory in your `hadoop-ozone/dist/target` directory.
You can copy this tarball and use this instead of binary artifacts that are
provided along with the official release.
-## How to test the build
-
-You can run the acceptance tests in the hadoop-ozone directory to make sure
-that your build is functional. To launch the acceptance tests, please follow
- the instructions in the **README.md** in the `smoketest` directory.
+To create tar file distribution, use the `-Pdist` profile:
```bash
-cd smoketest
-./test.sh
+mvn clean package -DskipTests=true -Pdist
```
- You can also execute only a minimal subset of the tests:
+## How to run Ozone from build
+
+When you have the new distribution, you can start a local cluster [with docker-compose]({{< ref "start/RunningViaDocker.md">}}).
```bash
-cd smoketest
-./test.sh --env ozone basic
+cd hadoop-ozone/dist/target/ozone-X.X.X...
+cd compose/ozone
+docker-compose up -d
```
-Acceptance tests will start a small ozone cluster and verify that ozone shell and ozone file
- system is fully functional.
+## How to test the build
+
+`compose` subfolder contains multiple type of example setup (secure, non-secure, HA, Yarn). They can be tested with the help of [robotframework](http://robotframework.org/) with executing `test.sh` in any of the directories.
\ No newline at end of file
diff --git a/hadoop-hdds/docs/content/start/FromSource.zh.md b/hadoop-hdds/docs/content/start/FromSource.zh.md
index a1b9f372e5e8..ab740af73828 100644
--- a/hadoop-hdds/docs/content/start/FromSource.zh.md
+++ b/hadoop-hdds/docs/content/start/FromSource.zh.md
@@ -19,10 +19,15 @@ weight: 30
limitations under the License.
-->
+
本文档是关于从源码构建 Ozone 的指南,如果你}}
-This assumes that you have set up the slaves file correctly and ssh
+This assumes that you have set up the `workers` file correctly and ssh
configuration that allows ssh-ing to all data nodes. This is the same as the
HDFS configuration, so please refer to HDFS documentation on how to set this
up.
diff --git a/hadoop-hdds/docs/content/start/OnPrem.zh.md b/hadoop-hdds/docs/content/start/OnPrem.zh.md
index 2e367b8d9e39..948025e10bd5 100644
--- a/hadoop-hdds/docs/content/start/OnPrem.zh.md
+++ b/hadoop-hdds/docs/content/start/OnPrem.zh.md
@@ -151,4 +151,4 @@ ozone om --init
start-ozone.sh
{{< /highlight >}}
-这么做的前提是,slaves 文件已经正确编写,并且配置好了到各个 Datanode 的 ssh,这和 HDFS 的配置方式相同,具体方法请查看 HDFS 文档。
+这么做的前提是,`workers` 文件已经正确编写,并且配置好了到各个 Datanode 的 ssh,这和 HDFS 的配置方式相同,具体方法请查看 HDFS 文档。
diff --git a/hadoop-hdds/docs/content/start/StartFromDockerHub.md b/hadoop-hdds/docs/content/start/StartFromDockerHub.md
index c4f36aff8926..6d26dfac849a 100644
--- a/hadoop-hdds/docs/content/start/StartFromDockerHub.md
+++ b/hadoop-hdds/docs/content/start/StartFromDockerHub.md
@@ -72,11 +72,7 @@ connecting to the SCM's UI at [http://localhost:9876](http://localhost:9876).
The S3 gateway endpoint will be exposed at port 9878. You can use Ozone's S3
support as if you are working against the real S3. S3 buckets are stored under
-the `/s3v` volume, which needs to be created by an administrator first:
-
-```
-docker-compose exec scm ozone sh volume create /s3v
-```
+the `/s3v` volume.
Here is how you create buckets from command line:
diff --git a/hadoop-hdds/docs/content/tools/_index.md b/hadoop-hdds/docs/content/tools/_index.md
index c77b671dbd50..090ba357b4b3 100644
--- a/hadoop-hdds/docs/content/tools/_index.md
+++ b/hadoop-hdds/docs/content/tools/_index.md
@@ -35,8 +35,9 @@ Daemon commands:
* **om** - Ozone Manager, via daemon command can be started or stopped.
* **datanode** - Via daemon command, the HDDS data nodes can be started or
stopped.
- * **s3g** -
-
+ * **s3g** - Start the S3 compatible REST gateway
+ * **recon** - The Web UI service of Ozone can be started with this command.
+
Client commands:
* **sh** - Primary command line interface for ozone to manage volumes/buckets/keys.
@@ -48,6 +49,7 @@ Admin commands:
* **admin** - Collects admin and developer related commands related to the
ozone components.
+ * **insight** - Generic tool to display filtered log, metrics or configs to help debuging. See [the observability]({{< ref "feature/Observability.md" >}}) page for more information.
* **classpath** - Prints the class path needed to get the hadoop jar and the
required libraries.
* **dtutil** - Operations related to delegation tokens
diff --git a/hadoop-hdds/docs/pom.xml b/hadoop-hdds/docs/pom.xml
index 8c1b6225d959..404b6c2a253b 100644
--- a/hadoop-hdds/docs/pom.xml
+++ b/hadoop-hdds/docs/pom.xml
@@ -20,10 +20,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
org.apache.hadoophadoop-hdds
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOThadoop-hdds-docs
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOTApache Hadoop HDDS/Ozone DocumentationApache Hadoop HDDS/Ozone Documentationjar
@@ -55,7 +55,7 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
static/slides/*themes/ozonedoc/static/js/bootstrap.min.js
- themes/ozonedoc/static/js/jquery-3.4.1.min.js
+ themes/ozonedoc/static/js/jquery-3.5.1.min.jsthemes/ozonedoc/static/css/bootstrap-theme.min.css
themes/ozonedoc/static/css/bootstrap.min.css.map
diff --git a/hadoop-hdds/docs/themes/ozonedoc/layouts/design/section.html b/hadoop-hdds/docs/themes/ozonedoc/layouts/design/section.html
index a70151813ece..b19b83169d21 100644
--- a/hadoop-hdds/docs/themes/ozonedoc/layouts/design/section.html
+++ b/hadoop-hdds/docs/themes/ozonedoc/layouts/design/section.html
@@ -46,7 +46,7 @@
+ * 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.hdds.cli;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.util.NativeCodeLoader;
+
+import org.apache.log4j.ConsoleAppender;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.log4j.PatternLayout;
+import picocli.CommandLine;
+
+/**
+ * Ozone Admin Command line tool.
+ */
+@CommandLine.Command(name = "ozone admin",
+ hidden = true,
+ description = "Developer tools for Ozone Admin operations",
+ versionProvider = HddsVersionProvider.class,
+ mixinStandardHelpOptions = true)
+public class OzoneAdmin extends GenericCli {
+
+ private OzoneConfiguration ozoneConf;
+
+ public OzoneAdmin() {
+ super(OzoneAdmin.class);
+ }
+
+ public OzoneConfiguration getOzoneConf() {
+ if (ozoneConf == null) {
+ ozoneConf = createOzoneConfiguration();
+ }
+ return ozoneConf;
+ }
+
+ /**
+ * Main for the Ozone Admin shell Command handling.
+ *
+ * @param argv - System Args Strings[]
+ */
+ public static void main(String[] argv) {
+ LogManager.resetConfiguration();
+ Logger.getRootLogger().setLevel(Level.INFO);
+ Logger.getRootLogger()
+ .addAppender(new ConsoleAppender(new PatternLayout("%m%n")));
+ Logger.getLogger(NativeCodeLoader.class).setLevel(Level.ERROR);
+
+ new OzoneAdmin().run(argv);
+ }
+}
diff --git a/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/.gitkeep b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/cli/package-info.java
similarity index 84%
rename from hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/.gitkeep
rename to hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/cli/package-info.java
index ff1232e5fcaa..82fbd722932e 100644
--- a/hadoop-hdds/container-service/src/main/resources/webapps/hddsDatanode/.gitkeep
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/cli/package-info.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -6,12 +6,17 @@
* 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
- *
+ * 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.hdds.scm.cli;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.hdds.cli.GenericParentCommand;
+import org.apache.hadoop.hdds.conf.MutableConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import picocli.CommandLine;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY;
+import static picocli.CommandLine.Spec.Target.MIXEE;
+
+/**
+ * Defines command-line option for SCM address.
+ */
+public class ScmOption {
+
+ @CommandLine.Spec(MIXEE)
+ private CommandLine.Model.CommandSpec spec;
+
+ @CommandLine.Option(names = {"--scm"},
+ description = "The destination scm (host:port)")
+ private String scm;
+
+ public ScmClient createScmClient() {
+ try {
+ GenericParentCommand parent = (GenericParentCommand)
+ spec.root().userObject();
+ OzoneConfiguration conf = parent.createOzoneConfiguration();
+ checkAndSetSCMAddressArg(conf);
+
+ return new ContainerOperationClient(conf);
+ } catch (IOException ex) {
+ throw new IllegalArgumentException("Can't create SCM client", ex);
+ }
+ }
+
+ private void checkAndSetSCMAddressArg(MutableConfigurationSource conf) {
+ if (StringUtils.isNotEmpty(scm)) {
+ conf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, scm);
+ }
+ if (!HddsUtils.getHostNameFromConfigKeys(conf,
+ ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY).isPresent()) {
+
+ throw new IllegalArgumentException(
+ ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY
+ + " should be set in ozone-site.xml or with the --scm option");
+ }
+ }
+
+}
diff --git a/hadoop-ozone/upgrade/src/main/java/org/apache/hadoop/ozone/upgrade/Plan.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ScmSubcommand.java
similarity index 60%
rename from hadoop-ozone/upgrade/src/main/java/org/apache/hadoop/ozone/upgrade/Plan.java
rename to hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ScmSubcommand.java
index efd6092a0b16..6dc09c2cbecd 100644
--- a/hadoop-ozone/upgrade/src/main/java/org/apache/hadoop/ozone/upgrade/Plan.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ScmSubcommand.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -15,24 +15,29 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.ozone.upgrade;
+package org.apache.hadoop.hdds.scm.cli;
-import java.util.concurrent.Callable;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import picocli.CommandLine;
-import picocli.CommandLine.Command;
+import java.io.IOException;
+import java.util.concurrent.Callable;
/**
- * Command to calculate statistics and estimate the upgrade.
+ * Base class for admin commands that connect via SCM client.
*/
-@Command(name = "plan",
- description = "Plan existing HDFS block distribution and give."
- + "estimation.")
-public class Plan implements Callable {
+public abstract class ScmSubcommand implements Callable {
+
+ @CommandLine.Mixin
+ private ScmOption scmOption;
+
+ protected abstract void execute(ScmClient client) throws IOException;
@Override
- public Void call() throws Exception {
- System.err.println("[In-Place upgrade : plan] is not yet supported.");
- return null;
+ public final Void call() throws Exception {
+ try (ScmClient scmClient = scmOption.createScmClient()) {
+ execute(scmClient);
+ return null;
+ }
}
-
}
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/TopologySubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/TopologySubcommand.java
index 214da34561b5..c1aebaeec22f 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/TopologySubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/TopologySubcommand.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -18,18 +18,19 @@
package org.apache.hadoop.hdds.scm.cli;
+import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.TreeSet;
-import java.util.concurrent.Callable;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.cli.OzoneAdmin;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.scm.cli.container.WithScmClient;
import org.apache.hadoop.hdds.scm.client.ScmClient;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DEAD;
@@ -37,9 +38,9 @@
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DECOMMISSIONING;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.HEALTHY;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.STALE;
+
+import org.kohsuke.MetaInfServices;
import picocli.CommandLine;
-import picocli.CommandLine.Model.CommandSpec;
-import picocli.CommandLine.Spec;
/**
* Handler of printTopology command.
@@ -49,22 +50,18 @@
description = "Print a tree of the network topology as reported by SCM",
mixinStandardHelpOptions = true,
versionProvider = HddsVersionProvider.class)
-public class TopologySubcommand implements Callable {
-
- @Spec
- private CommandSpec spec;
-
- @CommandLine.ParentCommand
- private WithScmClient parent;
+@MetaInfServices(SubcommandWithParent.class)
+public class TopologySubcommand extends ScmSubcommand
+ implements SubcommandWithParent {
- private static List stateArray = new ArrayList<>();
+ private static final List STATES = new ArrayList<>();
static {
- stateArray.add(HEALTHY);
- stateArray.add(STALE);
- stateArray.add(DEAD);
- stateArray.add(DECOMMISSIONING);
- stateArray.add(DECOMMISSIONED);
+ STATES.add(HEALTHY);
+ STATES.add(STALE);
+ STATES.add(DEAD);
+ STATES.add(DECOMMISSIONING);
+ STATES.add(DECOMMISSIONED);
}
@CommandLine.Option(names = {"-o", "--order"},
@@ -76,22 +73,24 @@ public class TopologySubcommand implements Callable {
private boolean fullInfo;
@Override
- public Void call() throws Exception {
- try (ScmClient scmClient = parent.createScmClient()) {
- for (HddsProtos.NodeState state : stateArray) {
- List nodes = scmClient.queryNode(state,
- HddsProtos.QueryScope.CLUSTER, "");
- if (nodes != null && nodes.size() > 0) {
- // show node state
- System.out.println("State = " + state.toString());
- if (order) {
- printOrderedByLocation(nodes);
- } else {
- printNodesWithLocation(nodes);
- }
+ public Class> getParentType() {
+ return OzoneAdmin.class;
+ }
+
+ @Override
+ protected void execute(ScmClient scmClient) throws IOException {
+ for (HddsProtos.NodeState state : STATES) {
+ List nodes = scmClient.queryNode(state,
+ HddsProtos.QueryScope.CLUSTER, "");
+ if (nodes != null && !nodes.isEmpty()) {
+ // show node state
+ System.out.println("State = " + state.toString());
+ if (order) {
+ printOrderedByLocation(nodes);
+ } else {
+ printNodesWithLocation(nodes);
}
}
- return null;
}
}
@@ -124,7 +123,7 @@ private String formatPortOutput(List ports) {
StringBuilder sb = new StringBuilder();
for (int i = 0; i < ports.size(); i++) {
HddsProtos.Port port = ports.get(i);
- sb.append(port.getName() + "=" + port.getValue());
+ sb.append(port.getName()).append("=").append(port.getValue());
if (i < ports.size() - 1) {
sb.append(",");
}
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseSubcommand.java
index cd81d32b8a82..53cbd2f63da3 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseSubcommand.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -17,15 +17,15 @@
*/
package org.apache.hadoop.hdds.scm.cli.container;
-import java.util.concurrent.Callable;
+import java.io.IOException;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
import org.apache.hadoop.hdds.scm.client.ScmClient;
import static org.apache.hadoop.hdds.scm.cli.container.ContainerCommands.checkContainerExists;
import picocli.CommandLine.Command;
import picocli.CommandLine.Parameters;
-import picocli.CommandLine.ParentCommand;
/**
* The handler of close container command.
@@ -35,21 +35,15 @@
description = "close container",
mixinStandardHelpOptions = true,
versionProvider = HddsVersionProvider.class)
-public class CloseSubcommand implements Callable {
-
- @ParentCommand
- private ContainerCommands parent;
+public class CloseSubcommand extends ScmSubcommand {
@Parameters(description = "Id of the container to close")
private long containerId;
@Override
- public Void call() throws Exception {
- try (ScmClient scmClient = parent.getParent().createScmClient()) {
- checkContainerExists(scmClient, containerId);
- scmClient.closeContainer(containerId);
- return null;
- }
+ public void execute(ScmClient scmClient) throws IOException {
+ checkContainerExists(scmClient, containerId);
+ scmClient.closeContainer(containerId);
}
}
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommands.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommands.java
index cf665b008f72..de1015d141e7 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommands.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommands.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -22,12 +22,14 @@
import org.apache.hadoop.hdds.cli.GenericCli;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.cli.OzoneAdmin;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
import org.apache.hadoop.hdds.scm.client.ScmClient;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.kohsuke.MetaInfServices;
import picocli.CommandLine.Command;
import picocli.CommandLine.Model.CommandSpec;
-import picocli.CommandLine.ParentCommand;
import picocli.CommandLine.Spec;
/**
@@ -45,24 +47,23 @@
CreateSubcommand.class,
CloseSubcommand.class
})
-public class ContainerCommands implements Callable {
+@MetaInfServices(SubcommandWithParent.class)
+public class ContainerCommands implements Callable, SubcommandWithParent {
@Spec
private CommandSpec spec;
- @ParentCommand
- private WithScmClient parent;
-
- public WithScmClient getParent() {
- return parent;
- }
-
@Override
public Void call() throws Exception {
GenericCli.missingSubcommand(spec);
return null;
}
+ @Override
+ public Class> getParentType() {
+ return OzoneAdmin.class;
+ }
+
public static void checkContainerExists(ScmClient scmClient, long containerId)
throws IOException {
ContainerInfo container = scmClient.getContainer(containerId);
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateSubcommand.java
index eb79e50506e2..9eedbf858958 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateSubcommand.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -17,9 +17,10 @@
*/
package org.apache.hadoop.hdds.scm.cli.container;
-import java.util.concurrent.Callable;
+import java.io.IOException;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
import org.apache.hadoop.hdds.scm.client.ScmClient;
import org.apache.hadoop.hdds.scm.container.common.helpers
.ContainerWithPipeline;
@@ -28,7 +29,6 @@
import org.slf4j.LoggerFactory;
import picocli.CommandLine.Command;
import picocli.CommandLine.Option;
-import picocli.CommandLine.ParentCommand;
/**
* This is the handler that process container creation command.
@@ -38,27 +38,19 @@
description = "Create container",
mixinStandardHelpOptions = true,
versionProvider = HddsVersionProvider.class)
-public class CreateSubcommand implements Callable {
+public class CreateSubcommand extends ScmSubcommand {
private static final Logger LOG =
LoggerFactory.getLogger(CreateSubcommand.class);
- @ParentCommand
- private ContainerCommands parent;
-
@Option(description = "Owner of the new container", defaultValue = "OZONE",
- required = false, names = {
- "-o", "--owner"})
-
+ names = { "-o", "--owner"})
private String owner;
@Override
- public Void call() throws Exception {
- try (ScmClient scmClient = parent.getParent().createScmClient()) {
- ContainerWithPipeline container = scmClient.createContainer(owner);
- LOG.info("Container {} is created.",
- container.getContainerInfo().getContainerID());
- return null;
- }
+ public void execute(ScmClient scmClient) throws IOException {
+ ContainerWithPipeline container = scmClient.createContainer(owner);
+ LOG.info("Container {} is created.",
+ container.getContainerInfo().getContainerID());
}
}
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteSubcommand.java
index a438fe906131..62d1b8ab2ae3 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteSubcommand.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -18,16 +18,16 @@
package org.apache.hadoop.hdds.scm.cli.container;
-import java.util.concurrent.Callable;
+import java.io.IOException;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
import org.apache.hadoop.hdds.scm.client.ScmClient;
import static org.apache.hadoop.hdds.scm.cli.container.ContainerCommands.checkContainerExists;
import picocli.CommandLine.Command;
import picocli.CommandLine.Option;
import picocli.CommandLine.Parameters;
-import picocli.CommandLine.ParentCommand;
/**
* This is the handler that process delete container command.
@@ -37,7 +37,7 @@
description = "Delete container",
mixinStandardHelpOptions = true,
versionProvider = HddsVersionProvider.class)
-public class DeleteSubcommand implements Callable {
+public class DeleteSubcommand extends ScmSubcommand {
@Parameters(description = "Id of the container to close")
private long containerId;
@@ -46,15 +46,9 @@ public class DeleteSubcommand implements Callable {
"--force"}, description = "forcibly delete the container")
private boolean force;
- @ParentCommand
- private ContainerCommands parent;
-
@Override
- public Void call() throws Exception {
- try (ScmClient scmClient = parent.getParent().createScmClient()) {
- checkContainerExists(scmClient, containerId);
- scmClient.deleteContainer(containerId, force);
- return null;
- }
+ public void execute(ScmClient scmClient) throws IOException {
+ checkContainerExists(scmClient, containerId);
+ scmClient.deleteContainer(containerId, force);
}
}
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoSubcommand.java
index 31e2a45dfc58..5defc2456a42 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoSubcommand.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -17,11 +17,12 @@
*/
package org.apache.hadoop.hdds.scm.cli.container;
-import java.util.concurrent.Callable;
+import java.io.IOException;
import java.util.stream.Collectors;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
import org.apache.hadoop.hdds.scm.client.ScmClient;
import org.apache.hadoop.hdds.scm.container.common.helpers
.ContainerWithPipeline;
@@ -31,7 +32,6 @@
import org.slf4j.LoggerFactory;
import picocli.CommandLine.Command;
import picocli.CommandLine.Parameters;
-import picocli.CommandLine.ParentCommand;
/**
* This is the handler that process container info command.
@@ -41,36 +41,30 @@
description = "Show information about a specific container",
mixinStandardHelpOptions = true,
versionProvider = HddsVersionProvider.class)
-public class InfoSubcommand implements Callable {
+public class InfoSubcommand extends ScmSubcommand {
private static final Logger LOG =
LoggerFactory.getLogger(InfoSubcommand.class);
- @ParentCommand
- private ContainerCommands parent;
-
@Parameters(description = "Decimal id of the container.")
private long containerID;
@Override
- public Void call() throws Exception {
- try (ScmClient scmClient = parent.getParent().createScmClient()) {
- final ContainerWithPipeline container = scmClient.
- getContainerWithPipeline(containerID);
- Preconditions.checkNotNull(container, "Container cannot be null");
+ public void execute(ScmClient scmClient) throws IOException {
+ final ContainerWithPipeline container = scmClient.
+ getContainerWithPipeline(containerID);
+ Preconditions.checkNotNull(container, "Container cannot be null");
- // Print container report info.
- LOG.info("Container id: {}", containerID);
- LOG.info("Pipeline id: {}", container.getPipeline().getId().getId());
- LOG.info("Container State: {}", container.getContainerInfo().getState());
+ // Print container report info.
+ LOG.info("Container id: {}", containerID);
+ LOG.info("Pipeline id: {}", container.getPipeline().getId().getId());
+ LOG.info("Container State: {}", container.getContainerInfo().getState());
- // Print pipeline of an existing container.
- String machinesStr = container.getPipeline().getNodes().stream().map(
- InfoSubcommand::buildDatanodeDetails)
- .collect(Collectors.joining(",\n"));
- LOG.info("Datanodes: [{}]", machinesStr);
- return null;
- }
+ // Print pipeline of an existing container.
+ String machinesStr = container.getPipeline().getNodes().stream().map(
+ InfoSubcommand::buildDatanodeDetails)
+ .collect(Collectors.joining(",\n"));
+ LOG.info("Datanodes: [{}]", machinesStr);
}
private static String buildDatanodeDetails(DatanodeDetails details) {
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java
index 3ffc118b57be..e9b0b7dc9a50 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -19,9 +19,9 @@
import java.io.IOException;
import java.util.List;
-import java.util.concurrent.Callable;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
import org.apache.hadoop.hdds.scm.client.ScmClient;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
@@ -36,7 +36,6 @@
import picocli.CommandLine.Command;
import picocli.CommandLine.Help.Visibility;
import picocli.CommandLine.Option;
-import picocli.CommandLine.ParentCommand;
/**
* This is the handler that process container list command.
@@ -46,22 +45,19 @@
description = "List containers",
mixinStandardHelpOptions = true,
versionProvider = HddsVersionProvider.class)
-public class ListSubcommand implements Callable {
+public class ListSubcommand extends ScmSubcommand {
private static final Logger LOG =
LoggerFactory.getLogger(ListSubcommand.class);
- @ParentCommand
- private ContainerCommands parent;
-
@Option(names = {"-s", "--start"},
- description = "Container id to start the iteration", required = false)
- private long startId = 0;
+ description = "Container id to start the iteration")
+ private long startId;
@Option(names = {"-c", "--count"},
description = "Maximum number of containers to list",
defaultValue = "20", showDefaultValue = Visibility.ALWAYS)
- private int count = 20;
+ private int count;
private static final ObjectWriter WRITER;
@@ -83,17 +79,13 @@ private void outputContainerInfo(ContainerInfo containerInfo)
}
@Override
- public Void call() throws Exception {
- try (ScmClient scmClient = parent.getParent().createScmClient()) {
-
- List containerList =
- scmClient.listContainer(startId, count);
+ public void execute(ScmClient scmClient) throws IOException {
+ List containerList =
+ scmClient.listContainer(startId, count);
- // Output data list
- for (ContainerInfo container : containerList) {
- outputContainerInfo(container);
- }
- return null;
+ // Output data list
+ for (ContainerInfo container : containerList) {
+ outputContainerInfo(container);
}
}
}
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/DatanodeCommands.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/DatanodeCommands.java
index b7ba59c77604..7e77c60f6e1e 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/DatanodeCommands.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/DatanodeCommands.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -21,8 +21,10 @@
import org.apache.hadoop.hdds.cli.GenericCli;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
-import org.apache.hadoop.hdds.scm.cli.container.WithScmClient;
+import org.apache.hadoop.hdds.cli.OzoneAdmin;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.kohsuke.MetaInfServices;
import picocli.CommandLine;
import picocli.CommandLine.Model.CommandSpec;
import picocli.CommandLine.Spec;
@@ -38,21 +40,20 @@
subcommands = {
ListInfoSubcommand.class
})
-public class DatanodeCommands implements Callable {
+@MetaInfServices(SubcommandWithParent.class)
+public class DatanodeCommands implements Callable, SubcommandWithParent {
@Spec
private CommandSpec spec;
- @CommandLine.ParentCommand
- private WithScmClient parent;
-
- public WithScmClient getParent() {
- return parent;
- }
-
@Override
public Void call() throws Exception {
GenericCli.missingSubcommand(spec);
return null;
}
+
+ @Override
+ public Class> getParentType() {
+ return OzoneAdmin.class;
+ }
}
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/ListInfoSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/ListInfoSubcommand.java
index e4060b3dadaf..80c5ecaae820 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/ListInfoSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/ListInfoSubcommand.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -21,13 +21,13 @@
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
import org.apache.hadoop.hdds.scm.client.ScmClient;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import picocli.CommandLine;
import java.io.IOException;
import java.util.List;
-import java.util.concurrent.Callable;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -39,44 +39,36 @@
description = "List info of datanodes",
mixinStandardHelpOptions = true,
versionProvider = HddsVersionProvider.class)
-public class ListInfoSubcommand implements Callable {
-
- @CommandLine.ParentCommand
- private DatanodeCommands parent;
+public class ListInfoSubcommand extends ScmSubcommand {
@CommandLine.Option(names = {"--ip"},
description = "Show info by ip address.",
- defaultValue = "",
- required = false)
+ defaultValue = "")
private String ipaddress;
@CommandLine.Option(names = {"--id"},
description = "Show info by datanode UUID.",
- defaultValue = "",
- required = false)
+ defaultValue = "")
private String uuid;
private List pipelines;
@Override
- public Void call() throws Exception {
- try (ScmClient scmClient = parent.getParent().createScmClient()) {
- pipelines = scmClient.listPipelines();
- if (Strings.isNullOrEmpty(ipaddress) && Strings.isNullOrEmpty(uuid)) {
- getAllNodes(scmClient).stream().forEach(p -> printDatanodeInfo(p));
- } else {
- Stream allNodes = getAllNodes(scmClient).stream();
- if (!Strings.isNullOrEmpty(ipaddress)) {
- allNodes = allNodes.filter(p -> p.getIpAddress()
- .compareToIgnoreCase(ipaddress) == 0);
- }
- if (!Strings.isNullOrEmpty(uuid)) {
- allNodes = allNodes.filter(p -> p.getUuid().toString().equals(uuid));
- }
- allNodes.forEach(p -> printDatanodeInfo(p));
+ public void execute(ScmClient scmClient) throws IOException {
+ pipelines = scmClient.listPipelines();
+ if (Strings.isNullOrEmpty(ipaddress) && Strings.isNullOrEmpty(uuid)) {
+ getAllNodes(scmClient).forEach(this::printDatanodeInfo);
+ } else {
+ Stream allNodes = getAllNodes(scmClient).stream();
+ if (!Strings.isNullOrEmpty(ipaddress)) {
+ allNodes = allNodes.filter(p -> p.getIpAddress()
+ .compareToIgnoreCase(ipaddress) == 0);
+ }
+ if (!Strings.isNullOrEmpty(uuid)) {
+ allNodes = allNodes.filter(p -> p.getUuid().toString().equals(uuid));
}
- return null;
+ allNodes.forEach(this::printDatanodeInfo);
}
}
@@ -101,7 +93,7 @@ private void printDatanodeInfo(DatanodeDetails datanode) {
" or the node is not in Healthy state.");
} else {
relatedPipelineNum = relatedPipelines.size();
- relatedPipelines.stream().forEach(
+ relatedPipelines.forEach(
p -> pipelineListInfo.append(p.getId().getId().toString())
.append("/").append(p.getFactor().toString()).append("/")
.append(p.getType().toString()).append("/")
@@ -118,4 +110,4 @@ private void printDatanodeInfo(DatanodeDetails datanode) {
+ "/" + datanode.getHostName() + "/" + relatedPipelineNum +
" pipelines) \n" + "Related pipelines: \n" + pipelineListInfo);
}
-}
\ No newline at end of file
+}
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ActivatePipelineSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ActivatePipelineSubcommand.java
index ec4b1b789e8c..a61655dc66b0 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ActivatePipelineSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ActivatePipelineSubcommand.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -20,10 +20,11 @@
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
import org.apache.hadoop.hdds.scm.client.ScmClient;
import picocli.CommandLine;
-import java.util.concurrent.Callable;
+import java.io.IOException;
/**
* Handler of activate pipeline command.
@@ -33,20 +34,14 @@
description = "Activates the given Pipeline",
mixinStandardHelpOptions = true,
versionProvider = HddsVersionProvider.class)
-public class ActivatePipelineSubcommand implements Callable {
-
- @CommandLine.ParentCommand
- private PipelineCommands parent;
+public class ActivatePipelineSubcommand extends ScmSubcommand {
@CommandLine.Parameters(description = "ID of the pipeline to activate")
private String pipelineId;
@Override
- public Void call() throws Exception {
- try (ScmClient scmClient = parent.getParent().createScmClient()) {
- scmClient.activatePipeline(
- HddsProtos.PipelineID.newBuilder().setId(pipelineId).build());
- return null;
- }
+ public void execute(ScmClient scmClient) throws IOException {
+ scmClient.activatePipeline(
+ HddsProtos.PipelineID.newBuilder().setId(pipelineId).build());
}
}
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ClosePipelineSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ClosePipelineSubcommand.java
index 89a280e805c0..78b83e56db07 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ClosePipelineSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ClosePipelineSubcommand.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -20,10 +20,11 @@
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
import org.apache.hadoop.hdds.scm.client.ScmClient;
import picocli.CommandLine;
-import java.util.concurrent.Callable;
+import java.io.IOException;
/**
* Handler of close pipeline command.
@@ -33,20 +34,14 @@
description = "Close pipeline",
mixinStandardHelpOptions = true,
versionProvider = HddsVersionProvider.class)
-public class ClosePipelineSubcommand implements Callable {
-
- @CommandLine.ParentCommand
- private PipelineCommands parent;
+public class ClosePipelineSubcommand extends ScmSubcommand {
@CommandLine.Parameters(description = "ID of the pipeline to close")
private String pipelineId;
@Override
- public Void call() throws Exception {
- try (ScmClient scmClient = parent.getParent().createScmClient()) {
- scmClient.closePipeline(
- HddsProtos.PipelineID.newBuilder().setId(pipelineId).build());
- return null;
- }
+ public void execute(ScmClient scmClient) throws IOException {
+ scmClient.closePipeline(
+ HddsProtos.PipelineID.newBuilder().setId(pipelineId).build());
}
}
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/CreatePipelineSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/CreatePipelineSubcommand.java
index e0bdddb7797e..c784be88b376 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/CreatePipelineSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/CreatePipelineSubcommand.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -20,11 +20,12 @@
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
import org.apache.hadoop.hdds.scm.client.ScmClient;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import picocli.CommandLine;
-import java.util.concurrent.Callable;
+import java.io.IOException;
/**
* Handler of createPipeline command.
@@ -34,44 +35,37 @@
description = "create pipeline",
mixinStandardHelpOptions = true,
versionProvider = HddsVersionProvider.class)
-public class CreatePipelineSubcommand implements Callable {
- @CommandLine.ParentCommand
- private PipelineCommands parent;
+public class CreatePipelineSubcommand extends ScmSubcommand {
@CommandLine.Option(
names = {"-t", "--replicationType"},
description = "Replication type (STAND_ALONE, RATIS)",
defaultValue = "STAND_ALONE"
)
- private HddsProtos.ReplicationType type
- = HddsProtos.ReplicationType.STAND_ALONE;
+ private HddsProtos.ReplicationType type;
@CommandLine.Option(
names = {"-f", "--replicationFactor"},
description = "Replication factor (ONE, THREE)",
defaultValue = "ONE"
)
- private HddsProtos.ReplicationFactor factor
- = HddsProtos.ReplicationFactor.ONE;
+ private HddsProtos.ReplicationFactor factor;
@Override
- public Void call() throws Exception {
+ public void execute(ScmClient scmClient) throws IOException {
if (type == HddsProtos.ReplicationType.CHAINED) {
throw new IllegalArgumentException(type.name()
+ " is not supported yet.");
}
- try (ScmClient scmClient = parent.getParent().createScmClient()) {
- Pipeline pipeline = scmClient.createReplicationPipeline(
- type,
- factor,
- HddsProtos.NodePool.getDefaultInstance());
+ Pipeline pipeline = scmClient.createReplicationPipeline(
+ type,
+ factor,
+ HddsProtos.NodePool.getDefaultInstance());
- if (pipeline != null) {
- System.out.println(pipeline.getId().toString() +
- " is created. Factor: " + pipeline.getFactor() +
- ", Type: " + pipeline.getType());
- }
- return null;
+ if (pipeline != null) {
+ System.out.println(pipeline.getId().toString() +
+ " is created. Factor: " + pipeline.getFactor() +
+ ", Type: " + pipeline.getType());
}
}
-}
\ No newline at end of file
+}
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/DeactivatePipelineSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/DeactivatePipelineSubcommand.java
index 4f4f741a3647..70df4d91fae9 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/DeactivatePipelineSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/DeactivatePipelineSubcommand.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -20,10 +20,11 @@
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
import org.apache.hadoop.hdds.scm.client.ScmClient;
import picocli.CommandLine;
-import java.util.concurrent.Callable;
+import java.io.IOException;
/**
* Handler of deactivate pipeline command.
@@ -33,20 +34,14 @@
description = "Deactivates the given Pipeline",
mixinStandardHelpOptions = true,
versionProvider = HddsVersionProvider.class)
-public class DeactivatePipelineSubcommand implements Callable {
-
- @CommandLine.ParentCommand
- private PipelineCommands parent;
+public class DeactivatePipelineSubcommand extends ScmSubcommand {
@CommandLine.Parameters(description = "ID of the pipeline to deactivate")
private String pipelineId;
@Override
- public Void call() throws Exception {
- try (ScmClient scmClient = parent.getParent().createScmClient()) {
- scmClient.deactivatePipeline(
- HddsProtos.PipelineID.newBuilder().setId(pipelineId).build());
- return null;
- }
+ public void execute(ScmClient scmClient) throws IOException {
+ scmClient.deactivatePipeline(
+ HddsProtos.PipelineID.newBuilder().setId(pipelineId).build());
}
}
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ListPipelinesSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ListPipelinesSubcommand.java
index 729daeae56bc..58ae26e500e1 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ListPipelinesSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/ListPipelinesSubcommand.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -20,11 +20,12 @@
import com.google.common.base.Strings;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
import org.apache.hadoop.hdds.scm.client.ScmClient;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import picocli.CommandLine;
-import java.util.concurrent.Callable;
+import java.io.IOException;
import java.util.stream.Stream;
/**
@@ -35,38 +36,29 @@
description = "List all active pipelines",
mixinStandardHelpOptions = true,
versionProvider = HddsVersionProvider.class)
-public class ListPipelinesSubcommand implements Callable {
-
- @CommandLine.ParentCommand
- private PipelineCommands parent;
+public class ListPipelinesSubcommand extends ScmSubcommand {
@CommandLine.Option(names = {"-ffc", "--filterByFactor"},
description = "Filter listed pipelines by Factor(ONE/one)",
- defaultValue = "",
- required = false)
+ defaultValue = "")
private String factor;
@CommandLine.Option(names = {"-fst", "--filterByState"},
description = "Filter listed pipelines by State(OPEN/CLOSE)",
- defaultValue = "",
- required = false)
+ defaultValue = "")
private String state;
-
@Override
- public Void call() throws Exception {
- try (ScmClient scmClient = parent.getParent().createScmClient()) {
- Stream stream = scmClient.listPipelines().stream();
- if (!Strings.isNullOrEmpty(factor)) {
- stream = stream.filter(
- p -> p.getFactor().toString().compareToIgnoreCase(factor) == 0);
- }
- if (!Strings.isNullOrEmpty(state)) {
- stream = stream.filter(p -> p.getPipelineState().toString()
- .compareToIgnoreCase(state) == 0);
- }
- stream.forEach(System.out::println);
- return null;
+ public void execute(ScmClient scmClient) throws IOException {
+ Stream stream = scmClient.listPipelines().stream();
+ if (!Strings.isNullOrEmpty(factor)) {
+ stream = stream.filter(
+ p -> p.getFactor().toString().compareToIgnoreCase(factor) == 0);
+ }
+ if (!Strings.isNullOrEmpty(state)) {
+ stream = stream.filter(p -> p.getPipelineState().toString()
+ .compareToIgnoreCase(state) == 0);
}
+ stream.forEach(System.out::println);
}
}
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/PipelineCommands.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/PipelineCommands.java
index d5c0234d01f4..ba7371e6214a 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/PipelineCommands.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/pipeline/PipelineCommands.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -21,11 +21,12 @@
import org.apache.hadoop.hdds.cli.GenericCli;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
-import org.apache.hadoop.hdds.scm.cli.container.WithScmClient;
+import org.apache.hadoop.hdds.cli.OzoneAdmin;
+import org.apache.hadoop.hdds.cli.SubcommandWithParent;
+import org.kohsuke.MetaInfServices;
import picocli.CommandLine.Command;
import picocli.CommandLine.Model.CommandSpec;
-import picocli.CommandLine.ParentCommand;
import picocli.CommandLine.Spec;
/**
@@ -43,21 +44,20 @@
CreatePipelineSubcommand.class,
ClosePipelineSubcommand.class
})
-public class PipelineCommands implements Callable {
+@MetaInfServices(SubcommandWithParent.class)
+public class PipelineCommands implements Callable, SubcommandWithParent {
@Spec
private CommandSpec spec;
- @ParentCommand
- private WithScmClient parent;
-
- public WithScmClient getParent() {
- return parent;
- }
-
@Override
public Void call() throws Exception {
GenericCli.missingSubcommand(spec);
return null;
}
+
+ @Override
+ public Class> getParentType() {
+ return OzoneAdmin.class;
+ }
}
diff --git a/hadoop-ozone/client/pom.xml b/hadoop-ozone/client/pom.xml
index 64b0aca7006c..c512a9005dd6 100644
--- a/hadoop-ozone/client/pom.xml
+++ b/hadoop-ozone/client/pom.xml
@@ -20,10 +20,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
org.apache.hadoophadoop-ozone
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOThadoop-ozone-client
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOTApache Hadoop Ozone ClientApache Hadoop Ozone Clientjar
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
index 5bae15ddfe11..6c5d1dd909d3 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
@@ -54,6 +54,8 @@ public final class BucketArgs {
* Bucket encryption key name.
*/
private String bucketEncryptionKey;
+ private final String sourceVolume;
+ private final String sourceBucket;
/**
* Private constructor, constructed via builder.
@@ -62,15 +64,19 @@ public final class BucketArgs {
* @param acls list of ACLs.
* @param metadata map of bucket metadata
* @param bucketEncryptionKey bucket encryption key name
+ * @param sourceVolume
+ * @param sourceBucket
*/
private BucketArgs(Boolean versioning, StorageType storageType,
- List acls, Map metadata,
- String bucketEncryptionKey) {
+ List acls, Map metadata,
+ String bucketEncryptionKey, String sourceVolume, String sourceBucket) {
this.acls = acls;
this.versioning = versioning;
this.storageType = storageType;
this.metadata = metadata;
this.bucketEncryptionKey = bucketEncryptionKey;
+ this.sourceVolume = sourceVolume;
+ this.sourceBucket = sourceBucket;
}
/**
@@ -123,6 +129,14 @@ public static BucketArgs.Builder newBuilder() {
return new BucketArgs.Builder();
}
+ public String getSourceVolume() {
+ return sourceVolume;
+ }
+
+ public String getSourceBucket() {
+ return sourceBucket;
+ }
+
/**
* Builder for OmBucketInfo.
*/
@@ -132,6 +146,8 @@ public static class Builder {
private List acls;
private Map metadata;
private String bucketEncryptionKey;
+ private String sourceVolume;
+ private String sourceBucket;
public Builder() {
metadata = new HashMap<>();
@@ -161,13 +177,24 @@ public BucketArgs.Builder setBucketEncryptionKey(String bek) {
this.bucketEncryptionKey = bek;
return this;
}
+
+ public BucketArgs.Builder setSourceVolume(String volume) {
+ sourceVolume = volume;
+ return this;
+ }
+
+ public BucketArgs.Builder setSourceBucket(String bucket) {
+ sourceBucket = bucket;
+ return this;
+ }
+
/**
* Constructs the BucketArgs.
* @return instance of BucketArgs.
*/
public BucketArgs build() {
return new BucketArgs(versioning, storageType, acls, metadata,
- bucketEncryptionKey);
+ bucketEncryptionKey, sourceVolume, sourceBucket);
}
}
}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
index d22b846e1c85..d71e03c9b881 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
@@ -109,6 +109,8 @@ public class OzoneBucket extends WithMetadata {
private OzoneObj ozoneObj;
+ private String sourceVolume;
+ private String sourceBucket;
private OzoneBucket(ConfigurationSource conf, String volumeName,
String bucketName, ReplicationFactor defaultReplication,
@@ -138,11 +140,13 @@ private OzoneBucket(ConfigurationSource conf, String volumeName,
.setResType(OzoneObj.ResourceType.BUCKET)
.setStoreType(OzoneObj.StoreType.OZONE).build();
}
+
@SuppressWarnings("parameternumber")
public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
String volumeName, String bucketName, StorageType storageType,
Boolean versioning, long creationTime, Map metadata,
- String encryptionKeyName) {
+ String encryptionKeyName,
+ String sourceVolume, String sourceBucket) {
this(conf, volumeName, bucketName, null, null, proxy);
this.storageType = storageType;
this.versioning = versioning;
@@ -150,6 +154,8 @@ public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
this.creationTime = Instant.ofEpochMilli(creationTime);
this.metadata = metadata;
this.encryptionKeyName = encryptionKeyName;
+ this.sourceVolume = sourceVolume;
+ this.sourceBucket = sourceBucket;
modificationTime = Instant.now();
if (modificationTime.isBefore(this.creationTime)) {
modificationTime = Instant.ofEpochSecond(
@@ -161,9 +167,10 @@ public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
String volumeName, String bucketName, StorageType storageType,
Boolean versioning, long creationTime, long modificationTime,
- Map metadata, String encryptionKeyName) {
+ Map metadata, String encryptionKeyName,
+ String sourceVolume, String sourceBucket) {
this(conf, proxy, volumeName, bucketName, storageType, versioning,
- creationTime, metadata, encryptionKeyName);
+ creationTime, metadata, encryptionKeyName, sourceVolume, sourceBucket);
this.modificationTime = Instant.ofEpochMilli(modificationTime);
}
@@ -306,6 +313,16 @@ public String getEncryptionKeyName() {
return encryptionKeyName;
}
+ public String getSourceVolume() {
+ return sourceVolume;
+ }
+
+ public String getSourceBucket() {
+ return sourceBucket;
+ }
+
+ /**
+ * Builder for OmBucketInfo.
/**
* Adds ACLs to the Bucket.
* @param addAcl ACL to be added
@@ -455,6 +472,16 @@ public void renameKey(String fromKeyName, String toKeyName)
proxy.renameKey(volumeName, name, fromKeyName, toKeyName);
}
+ /**
+ * Rename the key by keyMap, The key is fromKeyName and value is toKeyName.
+ * @param keyMap The key is original key name nad value is new key name.
+ * @throws IOException
+ */
+ public void renameKeys(Map keyMap)
+ throws IOException {
+ proxy.renameKeys(volumeName, name, keyMap);
+ }
+
/**
* Initiate multipart upload for a specified key.
* @param keyName
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java
index 2f7b10718784..9bf3973aeaec 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientFactory.java
@@ -137,13 +137,17 @@ public static OzoneClient getRpcClient(ConfigurationSource config)
// configuration, we don't fall back to default ozone.om.address defined
// in ozone-default.xml.
- if (OmUtils.isServiceIdsDefined(config)) {
+ String[] serviceIds = config.getTrimmedStrings(OZONE_OM_SERVICE_IDS_KEY);
+ if (serviceIds.length > 1) {
throw new IOException("Following ServiceID's " +
config.getTrimmedStringCollection(OZONE_OM_SERVICE_IDS_KEY) + " are" +
" defined in the configuration. Use the method getRpcClient which " +
"takes serviceID and configuration as param");
+ } else if (serviceIds.length == 1) {
+ return getRpcClient(getClientProtocol(config, serviceIds[0]), config);
+ } else {
+ return getRpcClient(getClientProtocol(config), config);
}
- return getRpcClient(getClientProtocol(config), config);
}
/**
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java
index 3cab66465009..712d1199a335 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java
@@ -172,10 +172,8 @@ public void addPreallocateBlocks(OmKeyLocationInfoGroup version,
// server may return any number of blocks, (0 to any)
// only the blocks allocated in this open session (block createVersion
// equals to open session version)
- for (OmKeyLocationInfo subKeyInfo : version.getLocationList()) {
- if (subKeyInfo.getCreateVersion() == openVersion) {
- addKeyLocationInfo(subKeyInfo);
- }
+ for (OmKeyLocationInfo subKeyInfo : version.getLocationList(openVersion)) {
+ addKeyLocationInfo(subKeyInfo);
}
}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java
index 4af683827240..769035a5e5d0 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java
@@ -20,7 +20,6 @@
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.collections.CollectionUtils;
-import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.Seekable;
import org.apache.hadoop.hdds.client.BlockID;
@@ -35,7 +34,6 @@
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
-import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
@@ -325,62 +323,14 @@ public long getRemainingOfIndex(int index) throws IOException {
return blockStreams.get(index).getRemaining();
}
- /**
- * Copies some or all bytes from a large (over 2GB) InputStream
- * to an OutputStream, optionally skipping input bytes.
- *
- * Copy the method from IOUtils of commons-io to reimplement skip by seek
- * rather than read. The reason why IOUtils of commons-io implement skip
- * by read can be found at
- * IO-203.
- *
- *
- * This method uses the provided buffer, so there is no need to use a
- * BufferedInputStream.
- *
- *
- * @param output the OutputStream to write to
- * @param inputOffset : number of bytes to skip from input before copying
- * -ve values are ignored
- * @param length : number of bytes to copy. -ve means all
- * @param buffer the buffer to use for the copy
- * @return the number of bytes copied
- * @throws NullPointerException if the input or output is null
- * @throws IOException if an I/O error occurs
- */
- public long copyLarge(final OutputStream output,
- final long inputOffset, final long len, final byte[] buffer)
- throws IOException {
- if (inputOffset > 0) {
- seek(inputOffset);
- }
-
- if (len == 0) {
+ @Override
+ public long skip(long n) throws IOException {
+ if (n <= 0) {
return 0;
}
- final int bufferLength = buffer.length;
- int bytesToRead = bufferLength;
- if (len > 0 && len < bufferLength) {
- bytesToRead = (int) len;
- }
-
- int read;
- long totalRead = 0;
- while (bytesToRead > 0) {
- read = read(buffer, 0, bytesToRead);
- if (read == IOUtils.EOF) {
- break;
- }
-
- output.write(buffer, 0, read);
- totalRead += read;
- if (len > 0) { // only adjust len if not reading to the end
- // Note the cast must work because buffer.length is an integer
- bytesToRead = (int) Math.min(len - totalRead, bufferLength);
- }
- }
-
- return totalRead;
+ long toSkip = Math.min(n, length - getPos());
+ seek(getPos() + toSkip);
+ return toSkip;
}
}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneInputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneInputStream.java
index a69740f07952..14b28665bdb1 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneInputStream.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneInputStream.java
@@ -57,6 +57,11 @@ public int available() throws IOException {
return inputStream.available();
}
+ @Override
+ public long skip(long n) throws IOException {
+ return inputStream.skip(n);
+ }
+
public InputStream getInputStream() {
return inputStream;
}
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
index 9c662efbf000..1b8d93ac7258 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
@@ -314,7 +314,17 @@ void deleteKeys(String volumeName, String bucketName,
* @throws IOException
*/
void renameKey(String volumeName, String bucketName, String fromKeyName,
- String toKeyName) throws IOException;
+ String toKeyName) throws IOException;
+
+ /**
+ * Renames existing keys within a bucket.
+ * @param volumeName Name of the Volume
+ * @param bucketName Name of the Bucket
+ * @param keyMap The key is original key name nad value is new key name.
+ * @throws IOException
+ */
+ void renameKeys(String volumeName, String bucketName,
+ Map keyMap) throws IOException;
/**
* Returns list of Keys in {Volume/Bucket} that matches the keyPrefix,
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
index 56c867d45ea0..d72d930e54f3 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
@@ -76,6 +76,7 @@
import org.apache.hadoop.ozone.om.helpers.BucketEncryptionKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmDeleteKeys;
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
@@ -84,6 +85,7 @@
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadList;
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadListParts;
import org.apache.hadoop.ozone.om.helpers.OmPartInfo;
+import org.apache.hadoop.ozone.om.helpers.OmRenameKeys;
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
import org.apache.hadoop.ozone.om.helpers.OzoneAclUtil;
@@ -447,6 +449,8 @@ public void createBucket(
.setIsVersionEnabled(isVersionEnabled)
.addAllMetadata(bucketArgs.getMetadata())
.setStorageType(storageType)
+ .setSourceVolume(bucketArgs.getSourceVolume())
+ .setSourceBucket(bucketArgs.getSourceBucket())
.setAcls(listOfAcls.stream().distinct().collect(Collectors.toList()));
if (bek != null) {
@@ -613,7 +617,10 @@ public OzoneBucket getBucketDetails(
bucketInfo.getModificationTime(),
bucketInfo.getMetadata(),
bucketInfo.getEncryptionKeyInfo() != null ? bucketInfo
- .getEncryptionKeyInfo().getKeyName() : null);
+ .getEncryptionKeyInfo().getKeyName() : null,
+ bucketInfo.getSourceVolume(),
+ bucketInfo.getSourceBucket()
+ );
}
@Override
@@ -634,7 +641,9 @@ public List listBuckets(String volumeName, String bucketPrefix,
bucket.getModificationTime(),
bucket.getMetadata(),
bucket.getEncryptionKeyInfo() != null ? bucket
- .getEncryptionKeyInfo().getKeyName() : null))
+ .getEncryptionKeyInfo().getKeyName() : null,
+ bucket.getSourceVolume(),
+ bucket.getSourceBucket()))
.collect(Collectors.toList());
}
@@ -730,16 +739,9 @@ public void deleteKeys(
throws IOException {
HddsClientUtils.verifyResourceName(volumeName, bucketName);
Preconditions.checkNotNull(keyNameList);
- List keyArgsList = new ArrayList<>();
- for (String keyName: keyNameList) {
- OmKeyArgs keyArgs = new OmKeyArgs.Builder()
- .setVolumeName(volumeName)
- .setBucketName(bucketName)
- .setKeyName(keyName)
- .build();
- keyArgsList.add(keyArgs);
- }
- ozoneManagerClient.deleteKeys(keyArgsList);
+ OmDeleteKeys omDeleteKeys = new OmDeleteKeys(volumeName, bucketName,
+ keyNameList);
+ ozoneManagerClient.deleteKeys(omDeleteKeys);
}
@Override
@@ -759,6 +761,18 @@ public void renameKey(String volumeName, String bucketName,
ozoneManagerClient.renameKey(keyArgs, toKeyName);
}
+ @Override
+ public void renameKeys(String volumeName, String bucketName,
+ Map keyMap) throws IOException {
+ verifyVolumeName(volumeName);
+ verifyBucketName(bucketName);
+ HddsClientUtils.checkNotNull(keyMap);
+ OmRenameKeys omRenameKeys =
+ new OmRenameKeys(volumeName, bucketName, keyMap, null);
+ ozoneManagerClient.renameKeys(omRenameKeys);
+ }
+
+
@Override
public List listKeys(String volumeName, String bucketName,
String keyPrefix, String prevKey,
@@ -1016,6 +1030,7 @@ public OzoneFileStatus getOzoneFileStatus(String volumeName,
.setBucketName(bucketName)
.setKeyName(keyName)
.setRefreshPipeline(true)
+ .setSortDatanodesInPipeline(topologyAwareReadEnabled)
.build();
return ozoneManagerClient.getFileStatus(keyArgs);
}
@@ -1098,6 +1113,7 @@ public List listStatus(String volumeName, String bucketName,
.setBucketName(bucketName)
.setKeyName(keyName)
.setRefreshPipeline(true)
+ .setSortDatanodesInPipeline(topologyAwareReadEnabled)
.build();
return ozoneManagerClient
.listStatus(keyArgs, recursive, startKey, numEntries);
diff --git a/hadoop-ozone/common/pom.xml b/hadoop-ozone/common/pom.xml
index 754351b46315..78eb2e71fa4b 100644
--- a/hadoop-ozone/common/pom.xml
+++ b/hadoop-ozone/common/pom.xml
@@ -20,10 +20,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
org.apache.hadoophadoop-ozone
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOThadoop-ozone-common
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOTApache Hadoop Ozone CommonApache Hadoop Ozone Commonjar
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java
index 6eb8b18b1ee4..93e0e7f7dec0 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java
@@ -34,6 +34,7 @@
import java.util.OptionalInt;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.ozone.conf.OMClientConfig;
@@ -54,6 +55,7 @@
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_BIND_HOST_KEY;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_BIND_PORT_DEFAULT;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_INTERNAL_SERVICE_ID;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_NODES_KEY;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_PORT_DEFAULT;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SERVICE_IDS_KEY;
@@ -244,6 +246,7 @@ public static boolean isReadOnly(
case DeleteBucket:
case CreateKey:
case RenameKey:
+ case RenameKeys:
case DeleteKey:
case DeleteKeys:
case CommitKey:
@@ -527,4 +530,49 @@ public static void validateKeyName(String keyName)
OMException.ResultCodes.INVALID_KEY_NAME);
}
}
+
+ /**
+ * Return configured OzoneManager service id based on the following logic.
+ * Look at 'ozone.om.internal.service.id' first. If configured, return that.
+ * If the above is not configured, look at 'ozone.om.service.ids'.
+ * If count(ozone.om.service.ids) == 1, return that id.
+ * If count(ozone.om.service.ids) > 1 throw exception
+ * If 'ozone.om.service.ids' is not configured, return null. (Non HA)
+ * @param conf configuration
+ * @return OM service ID.
+ * @throws IOException on error.
+ */
+ public static String getOzoneManagerServiceId(OzoneConfiguration conf)
+ throws IOException {
+ String localOMServiceId = conf.get(OZONE_OM_INTERNAL_SERVICE_ID);
+ Collection omServiceIds = conf.getTrimmedStringCollection(
+ OZONE_OM_SERVICE_IDS_KEY);
+ if (localOMServiceId == null) {
+ LOG.info("{} is not defined, falling back to {} to find serviceID for "
+ + "OzoneManager if it is HA enabled cluster",
+ OZONE_OM_INTERNAL_SERVICE_ID, OZONE_OM_SERVICE_IDS_KEY);
+ if (omServiceIds.size() > 1) {
+ throw new IOException(String.format(
+ "More than 1 OzoneManager ServiceID (%s) " +
+ "configured : %s, but %s is not " +
+ "configured.", OZONE_OM_SERVICE_IDS_KEY,
+ omServiceIds.toString(), OZONE_OM_INTERNAL_SERVICE_ID));
+ }
+ } else if (!omServiceIds.contains(localOMServiceId)) {
+ throw new IOException(String.format(
+ "Cannot find the internal service id %s in %s",
+ localOMServiceId, omServiceIds.toString()));
+ } else {
+ omServiceIds = Collections.singletonList(localOMServiceId);
+ }
+
+ if (omServiceIds.isEmpty()) {
+ LOG.info("No OzoneManager ServiceID configured.");
+ return null;
+ } else {
+ String serviceId = omServiceIds.iterator().next();
+ LOG.info("Using OzoneManager ServiceID '{}'.", serviceId);
+ return serviceId;
+ }
+ }
}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java
index cd8b12614d79..3480063d1323 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java
@@ -32,16 +32,14 @@ public enum OMAction implements AuditAction {
DELETE_BUCKET,
DELETE_KEY,
RENAME_KEY,
+ RENAME_KEYS,
SET_OWNER,
SET_QUOTA,
UPDATE_VOLUME,
UPDATE_BUCKET,
UPDATE_KEY,
PURGE_KEYS,
-
- // S3 Bucket
- CREATE_S3_BUCKET,
- DELETE_S3_BUCKET,
+ DELETE_KEYS,
// READ Actions
CHECK_VOLUME_ACCESS,
@@ -52,7 +50,6 @@ public enum OMAction implements AuditAction {
READ_VOLUME,
READ_BUCKET,
READ_KEY,
- LIST_S3BUCKETS,
INITIATE_MULTIPART_UPLOAD,
COMMIT_MULTIPART_UPLOAD_PARTKEY,
COMPLETE_MULTIPART_UPLOAD,
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
index 4f512a55032d..f16679a681eb 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
@@ -239,4 +239,11 @@ private OMConfigKeys() {
"ozone.om.keyname.character.check.enabled";
public static final boolean OZONE_OM_KEYNAME_CHARACTER_CHECK_ENABLED_DEFAULT =
false;
+
+ // This config needs to be enabled, when S3G created objects used via
+ // FileSystem API.
+ public static final String OZONE_OM_ENABLE_FILESYSTEM_PATHS =
+ "ozone.om.enable.filesystem.paths";
+ public static final boolean OZONE_OM_ENABLE_FILESYSTEM_PATHS_DEFAULT =
+ false;
}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
index 58d5a02e1f30..e08dccb6a5a1 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
@@ -223,6 +223,13 @@ public enum ResultCodes {
INVALID_VOLUME_NAME,
- REPLAY // When ratis logs are replayed.
+ PARTIAL_DELETE,
+
+ DETECTED_LOOP_IN_BUCKET_LINKS,
+
+ NOT_SUPPORTED_OPERATION,
+
+ PARTIAL_RENAME
+
}
}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMReplayException.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMReplayException.java
deleted file mode 100644
index 0eeb873a58f8..000000000000
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMReplayException.java
+++ /dev/null
@@ -1,50 +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
- *
+ * 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.util;
+
+import org.apache.ratis.util.ExitUtils;
+import org.slf4j.Logger;
+
+/**
+ * An Exit Manager used to shutdown service in case of unrecoverable error.
+ * This class will be helpful to test exit functionality.
+ */
+public class ExitManager {
+
+ public void exitSystem(int status, String message, Throwable throwable,
+ Logger log) {
+ ExitUtils.terminate(1, message, throwable, log);
+ }
+}
diff --git a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/TestOmUtils.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/TestOmUtils.java
index ab9b4e1c080e..8c527cbd318a 100644
--- a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/TestOmUtils.java
+++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/TestOmUtils.java
@@ -26,10 +26,16 @@
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import static org.apache.hadoop.ozone.OmUtils.getOzoneManagerServiceId;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_INTERNAL_SERVICE_ID;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SERVICE_IDS_KEY;
+import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assume.assumeTrue;
+
+import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
@@ -99,5 +105,48 @@ public void testGetOmHAAddressesById() {
assertTrue(rpcAddrs.stream().anyMatch(
a -> a.getAddress().getHostAddress().equals("1.1.1.3")));
}
+
+ @Test
+ public void testGetOzoneManagerServiceId() throws IOException {
+
+ // If the above is not configured, look at 'ozone.om.service.ids'.
+ // If no config is set, return null. (Non HA)
+ OzoneConfiguration configuration = new OzoneConfiguration();
+ assertNull(getOzoneManagerServiceId(configuration));
+
+ // Verify 'ozone.om.internal.service.id' takes precedence
+ configuration.set(OZONE_OM_INTERNAL_SERVICE_ID, "om1");
+ configuration.set(OZONE_OM_SERVICE_IDS_KEY, "om2,om1");
+ String id = getOzoneManagerServiceId(configuration);
+ assertEquals("om1", id);
+
+ configuration.set(OZONE_OM_SERVICE_IDS_KEY, "om2,om3");
+ try {
+ getOzoneManagerServiceId(configuration);
+ Assert.fail();
+ } catch (IOException ioEx) {
+ assertTrue(ioEx.getMessage()
+ .contains("Cannot find the internal service id om1 in [om2, om3]"));
+ }
+
+ // When internal service ID is not defined.
+ // Verify if count(ozone.om.service.ids) == 1, return that id.
+ configuration = new OzoneConfiguration();
+ configuration.set(OZONE_OM_SERVICE_IDS_KEY, "om2");
+ id = getOzoneManagerServiceId(configuration);
+ assertEquals("om2", id);
+
+ // Verify if more than count(ozone.om.service.ids) > 1 and internal
+ // service id is not defined, throw exception
+ configuration.set(OZONE_OM_SERVICE_IDS_KEY, "om2,om1");
+ try {
+ getOzoneManagerServiceId(configuration);
+ Assert.fail();
+ } catch (IOException ioEx) {
+ assertTrue(ioEx.getMessage()
+ .contains("More than 1 OzoneManager ServiceID (ozone.om.service" +
+ ".ids) configured"));
+ }
+ }
}
diff --git a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketInfo.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketInfo.java
index 15468c7b2f62..650fc910289d 100644
--- a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketInfo.java
+++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmBucketInfo.java
@@ -42,10 +42,21 @@ public void protobufConversion() {
.setStorageType(StorageType.ARCHIVE)
.build();
- OmBucketInfo afterSerialization =
- OmBucketInfo.getFromProtobuf(bucket.getProtobuf());
+ Assert.assertEquals(bucket,
+ OmBucketInfo.getFromProtobuf(bucket.getProtobuf()));
+ }
+
+ @Test
+ public void protobufConversionOfBucketLink() {
+ OmBucketInfo bucket = OmBucketInfo.newBuilder()
+ .setBucketName("bucket")
+ .setVolumeName("vol1")
+ .setSourceVolume("otherVol")
+ .setSourceBucket("someBucket")
+ .build();
- Assert.assertEquals(bucket, afterSerialization);
+ Assert.assertEquals(bucket,
+ OmBucketInfo.getFromProtobuf(bucket.getProtobuf()));
}
@Test
@@ -66,7 +77,10 @@ public void testClone() {
/* Clone an omBucketInfo. */
OmBucketInfo cloneBucketInfo = omBucketInfo.copyObject();
- Assert.assertEquals(omBucketInfo, cloneBucketInfo);
+ Assert.assertNotSame(omBucketInfo, cloneBucketInfo);
+ Assert.assertEquals("Expected " + omBucketInfo + " and " + cloneBucketInfo
+ + " to be equal",
+ omBucketInfo, cloneBucketInfo);
/* Reset acl & check not equal. */
omBucketInfo.setAcls(Collections.singletonList(new OzoneAcl(
diff --git a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmKeyLocationInfoGroup.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmKeyLocationInfoGroup.java
new file mode 100644
index 000000000000..0843e0b3e882
--- /dev/null
+++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmKeyLocationInfoGroup.java
@@ -0,0 +1,59 @@
+/**
+ * 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
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.om.helpers;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Test OmKeyLocationInfoGroup.
+ */
+public class TestOmKeyLocationInfoGroup {
+
+ @Test
+ public void testCreatingAndGetLatestVersionLocations() {
+ OmKeyLocationInfoGroup testInstance = createTestInstance();
+ List latestList =
+ testInstance.getBlocksLatestVersionOnly();
+ Assert.assertEquals(1, latestList.size());
+ Assert.assertEquals(2, latestList.get(0).getCreateVersion());
+ }
+
+ @Test
+ public void testGettingPreviousVersions() {
+ OmKeyLocationInfoGroup testInstance = createTestInstance();
+ List list = testInstance.getLocationList(1L);
+ Assert.assertEquals(2, list.size());
+ }
+
+ private OmKeyLocationInfoGroup createTestInstance() {
+ OmKeyLocationInfo info1 = new OmKeyLocationInfo.Builder().build();
+ info1.setCreateVersion(1);
+ OmKeyLocationInfo info2 = new OmKeyLocationInfo.Builder().build();
+ info2.setCreateVersion(1);
+ OmKeyLocationInfo info3 = new OmKeyLocationInfo.Builder().build();
+ info3.setCreateVersion(2);
+ List locationInfoList = new ArrayList<>();
+ locationInfoList.add(info1);
+ locationInfoList.add(info2);
+ locationInfoList.add(info3);
+ return new OmKeyLocationInfoGroup(2, locationInfoList);
+ }
+}
diff --git a/hadoop-ozone/upgrade/src/main/java/org/apache/hadoop/ozone/upgrade/Balance.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOzoneFsUtils.java
similarity index 57%
rename from hadoop-ozone/upgrade/src/main/java/org/apache/hadoop/ozone/upgrade/Balance.java
rename to hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOzoneFsUtils.java
index 149273862a75..7471d539484c 100644
--- a/hadoop-ozone/upgrade/src/main/java/org/apache/hadoop/ozone/upgrade/Balance.java
+++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOzoneFsUtils.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -15,24 +15,25 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.ozone.upgrade;
-import java.util.concurrent.Callable;
+package org.apache.hadoop.ozone.om.helpers;
-import picocli.CommandLine.Command;
+import org.junit.Assert;
+import org.junit.Test;
/**
- * Command to move blocks between HDFS datanodes.
+ * Test OzoneFsUtils.
*/
-@Command(name = "balance",
- description = "Move the HDFS blocks for a better distribution "
- + "usage.")
-public class Balance implements Callable {
+public class TestOzoneFsUtils {
- @Override
- public Void call() throws Exception {
- System.err.println("[In-Place upgrade : balance] is not yet supported.");
- return null;
+ @Test
+ public void testPaths() {
+ Assert.assertTrue(OzoneFSUtils.isValidName("/a/b"));
+ Assert.assertFalse(OzoneFSUtils.isValidName("../../../a/b"));
+ Assert.assertFalse(OzoneFSUtils.isValidName("/./."));
+ Assert.assertFalse(OzoneFSUtils.isValidName("/:/"));
+ Assert.assertFalse(OzoneFSUtils.isValidName("a/b"));
+ Assert.assertFalse(OzoneFSUtils.isValidName("/a:/b"));
+ Assert.assertFalse(OzoneFSUtils.isValidName("/a//b"));
}
-
-}
\ No newline at end of file
+}
diff --git a/hadoop-ozone/csi/pom.xml b/hadoop-ozone/csi/pom.xml
index 9fd212626e78..f678280d5888 100644
--- a/hadoop-ozone/csi/pom.xml
+++ b/hadoop-ozone/csi/pom.xml
@@ -20,10 +20,10 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
org.apache.hadoophadoop-ozone
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOThadoop-ozone-csi
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOTApache Hadoop Ozone CSI serviceApache Hadoop Ozone CSI servicejar
diff --git a/hadoop-ozone/csi/src/main/proto/proto.lock b/hadoop-ozone/csi/src/main/resources/proto.lock
similarity index 100%
rename from hadoop-ozone/csi/src/main/proto/proto.lock
rename to hadoop-ozone/csi/src/main/resources/proto.lock
diff --git a/hadoop-ozone/datanode/pom.xml b/hadoop-ozone/datanode/pom.xml
index 75eaa8ccadd9..1c8d81e52fcc 100644
--- a/hadoop-ozone/datanode/pom.xml
+++ b/hadoop-ozone/datanode/pom.xml
@@ -19,12 +19,12 @@
org.apache.hadoophadoop-ozone
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOThadoop-ozone-datanodeApache Hadoop Ozone Datanodejar
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOT
diff --git a/hadoop-ozone/dev-support/checks/acceptance.sh b/hadoop-ozone/dev-support/checks/acceptance.sh
index d95c034939b9..99d8d5254504 100755
--- a/hadoop-ozone/dev-support/checks/acceptance.sh
+++ b/hadoop-ozone/dev-support/checks/acceptance.sh
@@ -28,6 +28,8 @@ fi
mkdir -p "$REPORT_DIR"
+export OZONE_ACCEPTANCE_SUITE
+
cd "$DIST_DIR/compose" || exit 1
./test-all.sh
RES=$?
diff --git a/hadoop-ozone/dev-support/checks/bats.sh b/hadoop-ozone/dev-support/checks/bats.sh
new file mode 100755
index 000000000000..2e1bbadce9aa
--- /dev/null
+++ b/hadoop-ozone/dev-support/checks/bats.sh
@@ -0,0 +1,35 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
+cd "${DIR}/../../.." || exit 1
+
+REPORT_DIR=${OUTPUT_DIR:-"${DIR}/../../../target/bats"}
+mkdir -p "${REPORT_DIR}"
+REPORT_FILE="${REPORT_DIR}/summary.txt"
+
+rm -f "${REPORT_DIR}/output.log"
+
+find * -path '*/src/test/shell/*' -name '*.bats' -print0 \
+ | xargs -0 -n1 bats --formatter tap \
+ | tee -a "${REPORT_DIR}/output.log"
+
+grep '^\(not ok\|#\)' "${REPORT_DIR}/output.log" > "${REPORT_FILE}"
+
+grep -c '^not ok' "${REPORT_FILE}" > "${REPORT_DIR}/failures"
+
+if [[ -s "${REPORT_FILE}" ]]; then
+ exit 1
+fi
diff --git a/hadoop-ozone/dev-support/checks/kubernetes.sh b/hadoop-ozone/dev-support/checks/kubernetes.sh
new file mode 100755
index 000000000000..7f68da1884ca
--- /dev/null
+++ b/hadoop-ozone/dev-support/checks/kubernetes.sh
@@ -0,0 +1,36 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
+cd "$DIR/../../.." || exit 1
+
+REPORT_DIR=${OUTPUT_DIR:-"$DIR/../../../target/kubernetes"}
+
+OZONE_VERSION=$(grep "" "pom.xml" | sed 's/<[^>]*>//g'| sed 's/^[ \t]*//')
+DIST_DIR="$DIR/../../dist/target/ozone-$OZONE_VERSION"
+
+if [ ! -d "$DIST_DIR" ]; then
+ echo "Distribution dir is missing. Doing a full build"
+ "$DIR/build.sh" -Pcoverage
+fi
+
+mkdir -p "$REPORT_DIR"
+
+cd "$DIST_DIR/kubernetes/examples" || exit 1
+./test-all.sh
+RES=$?
+cp -r result/* "$REPORT_DIR/"
+cp "$REPORT_DIR/log.html" "$REPORT_DIR/summary.html"
+exit $RES
diff --git a/hadoop-ozone/dev-support/intellij/core-site.xml b/hadoop-ozone/dev-support/intellij/core-site.xml
new file mode 100644
index 000000000000..862b32177187
--- /dev/null
+++ b/hadoop-ozone/dev-support/intellij/core-site.xml
@@ -0,0 +1,27 @@
+
+
+
+
+ fs.ofs.impl
+ org.apache.hadoop.fs.ozone.RootedOzoneFileSystem
+
+
+ fs.defaultFS
+ ofs://localhost/
+
+
diff --git a/hadoop-ozone/dist/dev-support/bin/dist-layout-stitching b/hadoop-ozone/dist/dev-support/bin/dist-layout-stitching
index e1f5c7e55c5e..a3a38209d940 100755
--- a/hadoop-ozone/dist/dev-support/bin/dist-layout-stitching
+++ b/hadoop-ozone/dist/dev-support/bin/dist-layout-stitching
@@ -108,6 +108,7 @@ run cp "${ROOT}/hadoop-ozone/dist/src/shell/hdds/hadoop-config.cmd" "libexec/"
run cp "${ROOT}/hadoop-ozone/dist/src/shell/hdds/hadoop-functions.sh" "libexec/"
run cp "${ROOT}/hadoop-ozone/dist/src/shell/ozone/ozone-config.sh" "libexec/"
run cp -r "${ROOT}/hadoop-ozone/dist/src/shell/shellprofile.d" "libexec/"
+run cp -r "${ROOT}/hadoop-ozone/dist/src/shell/upgrade" "libexec/"
run cp "${ROOT}/hadoop-ozone/dist/src/shell/hdds/hadoop-daemons.sh" "sbin/"
@@ -121,6 +122,9 @@ run cp -r "${ROOT}/hadoop-ozone/fault-injection-test/network-tests/src/test/blo
# Optional documentation, could be missing
cp -r "${ROOT}/hadoop-hdds/docs/target/classes/docs" ./
+#copy byteman helpers
+run cp -r "${ROOT}/dev-support/byteman" "share/ozone/"
+
#Copy docker compose files
#compose files are preprocessed: properties (eg. project.version) are replaced first by maven.
run cp -p -R "${ROOT}/hadoop-ozone/dist/target/compose" .
@@ -130,3 +134,4 @@ run cp -p -r "${ROOT}/hadoop-ozone/dist/target/Dockerfile" .
#workaround for https://issues.apache.org/jira/browse/MRESOURCES-236
find ./compose -name "*.sh" -exec chmod 755 {} \;
+find ./kubernetes -name "*.sh" -exec chmod 755 {} \;
diff --git a/hadoop-ozone/dist/pom.xml b/hadoop-ozone/dist/pom.xml
index 840f628899b2..a9d877170bff 100644
--- a/hadoop-ozone/dist/pom.xml
+++ b/hadoop-ozone/dist/pom.xml
@@ -19,16 +19,16 @@
org.apache.hadoophadoop-ozone
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOThadoop-ozone-distApache Hadoop Ozone Distributionjar
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOTUTF-8true
- 20200420-1
+ 20200625-1
@@ -50,8 +50,7 @@
*.classpath
hadoop-hdds-server-scm,hadoop-ozone-common,hadoop-ozone-csi,hadoop-ozone-datanode,hadoop-ozone-insight,
- hadoop-ozone-ozone-manager,hadoop-ozone-recon,hadoop-ozone-s3gateway,hadoop-ozone-tools,
- hadoop-ozone-upgrade
+ hadoop-ozone-ozone-manager,hadoop-ozone-recon,hadoop-ozone-s3gateway,hadoop-ozone-tools
@@ -219,10 +218,6 @@
org.apache.hadoophadoop-hdds-docs
-
- org.apache.hadoop
- hadoop-ozone-upgrade
- org.apache.hadoophadoop-ozone-insight
diff --git a/hadoop-ozone/dist/src/main/compose/failing1/.env b/hadoop-ozone/dist/src/main/compose/failing1/.env
new file mode 120000
index 000000000000..c9b103fa52e1
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/failing1/.env
@@ -0,0 +1 @@
+../ozone/.env
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/main/compose/failing1/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/failing1/docker-compose.yaml
new file mode 120000
index 000000000000..76acad508c42
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/failing1/docker-compose.yaml
@@ -0,0 +1 @@
+../ozone/docker-compose.yaml
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/main/compose/failing1/docker-config b/hadoop-ozone/dist/src/main/compose/failing1/docker-config
new file mode 120000
index 000000000000..49694527f1bd
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/failing1/docker-config
@@ -0,0 +1 @@
+../ozone/docker-config
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/main/compose/failing1/test.sh b/hadoop-ozone/dist/src/main/compose/failing1/test.sh
new file mode 100755
index 000000000000..cb8687f6a4b6
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/failing1/test.sh
@@ -0,0 +1,36 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+#suite:failing
+
+COMPOSE_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
+export COMPOSE_DIR
+
+export SECURITY_ENABLED=false
+export OZONE_REPLICATION_FACTOR=3
+
+# shellcheck source=/dev/null
+source "$COMPOSE_DIR/../testlib.sh"
+
+start_docker_env
+
+execute_robot_test scm failing/test1.robot
+execute_robot_test scm failing/test2.robot
+
+stop_docker_env
+
+generate_report
diff --git a/hadoop-ozone/dist/src/main/compose/failing2/.env b/hadoop-ozone/dist/src/main/compose/failing2/.env
new file mode 120000
index 000000000000..c9b103fa52e1
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/failing2/.env
@@ -0,0 +1 @@
+../ozone/.env
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/main/compose/failing2/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/failing2/docker-compose.yaml
new file mode 120000
index 000000000000..76acad508c42
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/failing2/docker-compose.yaml
@@ -0,0 +1 @@
+../ozone/docker-compose.yaml
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/main/compose/failing2/docker-config b/hadoop-ozone/dist/src/main/compose/failing2/docker-config
new file mode 120000
index 000000000000..49694527f1bd
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/failing2/docker-config
@@ -0,0 +1 @@
+../ozone/docker-config
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/main/compose/failing2/test.sh b/hadoop-ozone/dist/src/main/compose/failing2/test.sh
new file mode 100755
index 000000000000..cb8687f6a4b6
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/failing2/test.sh
@@ -0,0 +1,36 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+#suite:failing
+
+COMPOSE_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
+export COMPOSE_DIR
+
+export SECURITY_ENABLED=false
+export OZONE_REPLICATION_FACTOR=3
+
+# shellcheck source=/dev/null
+source "$COMPOSE_DIR/../testlib.sh"
+
+start_docker_env
+
+execute_robot_test scm failing/test1.robot
+execute_robot_test scm failing/test2.robot
+
+stop_docker_env
+
+generate_report
diff --git a/hadoop-ozone/dist/src/main/compose/ozone-ha/.env b/hadoop-ozone/dist/src/main/compose/ozone-ha/.env
new file mode 100644
index 000000000000..8446b4a5b015
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/ozone-ha/.env
@@ -0,0 +1,19 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+HDDS_VERSION=${hdds.version}
+OZONE_RUNNER_VERSION=${docker.ozone-runner.version}
+HADOOP_OPTS=
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/main/compose/ozone-ha/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/ozone-ha/docker-compose.yaml
new file mode 100644
index 000000000000..b9f4b60702c6
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/ozone-ha/docker-compose.yaml
@@ -0,0 +1,93 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+version: "3.4"
+
+# reusable fragments (see https://docs.docker.com/compose/compose-file/#extension-fields)
+x-common-config:
+ &common-config
+ image: apache/ozone-runner:${OZONE_RUNNER_VERSION}
+ volumes:
+ - ../..:/opt/hadoop
+ env_file:
+ - docker-config
+
+x-replication:
+ &replication
+ OZONE-SITE.XML_ozone.replication: ${OZONE_REPLICATION_FACTOR:-1}
+
+services:
+ datanode:
+ <<: *common-config
+ ports:
+ - 9864
+ - 9882
+ environment:
+ <<: *replication
+ command: ["ozone","datanode"]
+ om1:
+ <<: *common-config
+ environment:
+ ENSURE_OM_INITIALIZED: /data/metadata/om/current/VERSION
+ <<: *replication
+ ports:
+ - 9874
+ - 9862
+ hostname: om1
+ command: ["ozone","om"]
+ om2:
+ <<: *common-config
+ environment:
+ ENSURE_OM_INITIALIZED: /data/metadata/om/current/VERSION
+ <<: *replication
+ ports:
+ - 9874
+ - 9862
+ hostname: om2
+ command: ["ozone","om"]
+ om3:
+ <<: *common-config
+ environment:
+ ENSURE_OM_INITIALIZED: /data/metadata/om/current/VERSION
+ <<: *replication
+ ports:
+ - 9874
+ - 9862
+ hostname: om3
+ command: ["ozone","om"]
+ scm:
+ <<: *common-config
+ ports:
+ - 9876:9876
+ environment:
+ ENSURE_SCM_INITIALIZED: /data/metadata/scm/current/VERSION
+ OZONE-SITE.XML_hdds.scm.safemode.min.datanode: ${OZONE_SAFEMODE_MIN_DATANODES:-1}
+ <<: *replication
+ command: ["ozone","scm"]
+ s3g:
+ <<: *common-config
+ environment:
+ <<: *replication
+ ports:
+ - 9878:9878
+ command: ["ozone","s3g"]
+ recon:
+ <<: *common-config
+ ports:
+ - 9888:9888
+ environment:
+ <<: *replication
+ command: ["ozone","recon"]
diff --git a/hadoop-ozone/dist/src/main/compose/ozone-ha/docker-config b/hadoop-ozone/dist/src/main/compose/ozone-ha/docker-config
new file mode 100644
index 000000000000..d378a679f816
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/ozone-ha/docker-config
@@ -0,0 +1,35 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+CORE-SITE.XML_fs.defaultFS=o3fs://bucket1.volume1.omservice
+
+OZONE-SITE.XML_ozone.om.service.ids=omservice
+OZONE-SITE.XML_ozone.om.nodes.omservice=om1,om2,om3
+OZONE-SITE.XML_ozone.om.address.omservice.om1=om1
+OZONE-SITE.XML_ozone.om.address.omservice.om2=om2
+OZONE-SITE.XML_ozone.om.address.omservice.om3=om3
+OZONE-SITE.XML_ozone.om.ratis.enable=true
+
+OZONE-SITE.XML_ozone.scm.names=scm
+OZONE-SITE.XML_ozone.scm.datanode.id.dir=/data
+OZONE-SITE.XML_ozone.scm.block.client.address=scm
+OZONE-SITE.XML_ozone.scm.container.size=1GB
+OZONE-SITE.XML_ozone.metadata.dirs=/data/metadata
+OZONE-SITE.XML_ozone.scm.client.address=scm
+OZONE-SITE.XML_ozone.client.failover.max.attempts=6
+OZONE-SITE.XML_hdds.datanode.dir=/data/hdds
+
+no_proxy=om1,om2,om3,scm,s3g,recon,kdc,localhost,127.0.0.1
diff --git a/hadoop-ozone/dist/src/main/compose/ozone-ha/test.sh b/hadoop-ozone/dist/src/main/compose/ozone-ha/test.sh
new file mode 100755
index 000000000000..a14aa9cfd8ce
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/ozone-ha/test.sh
@@ -0,0 +1,33 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+COMPOSE_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
+export COMPOSE_DIR
+
+export SECURITY_ENABLED=false
+export OZONE_REPLICATION_FACTOR=3
+
+# shellcheck source=/dev/null
+source "$COMPOSE_DIR/../testlib.sh"
+
+start_docker_env
+
+execute_robot_test scm basic/ozone-shell-single.robot
+
+stop_docker_env
+
+generate_report
diff --git a/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop27/docker-config b/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop27/docker-config
index 8180a10f878c..5c1348422f43 100644
--- a/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop27/docker-config
+++ b/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop27/docker-config
@@ -15,6 +15,7 @@
# limitations under the License.
CORE-SITE.xml_fs.AbstractFileSystem.o3fs.impl=org.apache.hadoop.fs.ozone.OzFs
+CORE-SITE.xml_fs.AbstractFileSystem.ofs.impl=org.apache.hadoop.fs.ozone.RootedOzFs
MAPRED-SITE.XML_mapreduce.application.classpath=/opt/hadoop/share/hadoop/mapreduce/*:/opt/hadoop/share/hadoop/mapreduce/lib/*:/opt/ozone/share/ozone/lib/hadoop-ozone-filesystem-hadoop2-@project.version@.jar
no_proxy=om,scm,s3g,recon,kdc,localhost,127.0.0.1
diff --git a/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop31/docker-config b/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop31/docker-config
index af8e7f800405..e80f0284a696 100644
--- a/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop31/docker-config
+++ b/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop31/docker-config
@@ -15,6 +15,7 @@
# limitations under the License.
CORE-SITE.xml_fs.AbstractFileSystem.o3fs.impl=org.apache.hadoop.fs.ozone.OzFs
+CORE-SITE.xml_fs.AbstractFileSystem.ofs.impl=org.apache.hadoop.fs.ozone.RootedOzFs
MAPRED-SITE.XML_mapreduce.application.classpath=/opt/hadoop/share/hadoop/mapreduce/*:/opt/hadoop/share/hadoop/mapreduce/lib/*:/opt/ozone/share/ozone/lib/hadoop-ozone-filesystem-hadoop3-@project.version@.jar
no_proxy=om,scm,s3g,recon,kdc,localhost,127.0.0.1
diff --git a/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop32/docker-config b/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop32/docker-config
index af8e7f800405..e80f0284a696 100644
--- a/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop32/docker-config
+++ b/hadoop-ozone/dist/src/main/compose/ozone-mr/hadoop32/docker-config
@@ -15,6 +15,7 @@
# limitations under the License.
CORE-SITE.xml_fs.AbstractFileSystem.o3fs.impl=org.apache.hadoop.fs.ozone.OzFs
+CORE-SITE.xml_fs.AbstractFileSystem.ofs.impl=org.apache.hadoop.fs.ozone.RootedOzFs
MAPRED-SITE.XML_mapreduce.application.classpath=/opt/hadoop/share/hadoop/mapreduce/*:/opt/hadoop/share/hadoop/mapreduce/lib/*:/opt/ozone/share/ozone/lib/hadoop-ozone-filesystem-hadoop3-@project.version@.jar
no_proxy=om,scm,s3g,recon,kdc,localhost,127.0.0.1
diff --git a/hadoop-ozone/dist/src/main/compose/ozone-mr/test.sh b/hadoop-ozone/dist/src/main/compose/ozone-mr/test.sh
new file mode 100644
index 000000000000..3a18d4df2860
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/ozone-mr/test.sh
@@ -0,0 +1,37 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+SCRIPT_DIR=$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null && pwd )
+ALL_RESULT_DIR="$SCRIPT_DIR/result"
+mkdir -p "$ALL_RESULT_DIR"
+rm "$ALL_RESULT_DIR/*" || true
+source "$SCRIPT_DIR/../testlib.sh"
+
+tests=$(find_tests)
+cd "$SCRIPT_DIR"
+
+RESULT=0
+# shellcheck disable=SC2044
+for t in ${tests}; do
+ d="$(dirname "${t}")"
+
+ if ! run_test_script "${d}"; then
+ RESULT=1
+ fi
+
+ copy_results "${d}" "${ALL_RESULT_DIR}"
+done
+
diff --git a/hadoop-ozone/dist/src/main/compose/ozone-om-ha-s3/docker-config b/hadoop-ozone/dist/src/main/compose/ozone-om-ha-s3/docker-config
index b835147ee261..4d47bf0b6f19 100644
--- a/hadoop-ozone/dist/src/main/compose/ozone-om-ha-s3/docker-config
+++ b/hadoop-ozone/dist/src/main/compose/ozone-om-ha-s3/docker-config
@@ -14,7 +14,6 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-CORE-SITE.XML_fs.o3fs.impl=org.apache.hadoop.fs.ozone.OzoneFileSystem
CORE-SITE.XML_fs.defaultFS=o3fs://bucket.volume.id1
OZONE-SITE.XML_ozone.om.service.ids=id1
OZONE-SITE.XML_ozone.om.nodes.id1=om1,om2,om3
diff --git a/hadoop-ozone/dist/src/main/compose/ozone-om-ha/docker-config b/hadoop-ozone/dist/src/main/compose/ozone-om-ha/docker-config
index 4eb1c8a5b6b7..008c684aeb99 100644
--- a/hadoop-ozone/dist/src/main/compose/ozone-om-ha/docker-config
+++ b/hadoop-ozone/dist/src/main/compose/ozone-om-ha/docker-config
@@ -14,7 +14,6 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-CORE-SITE.XML_fs.o3fs.impl=org.apache.hadoop.fs.ozone.OzoneFileSystem
CORE-SITE.XML_fs.defaultFS=o3fs://bucket1.volume1.omservice
OZONE-SITE.XML_ozone.om.service.ids=omservice
OZONE-SITE.XML_ozone.om.nodes.omservice=om1,om2,om3
diff --git a/hadoop-ozone/dist/src/main/compose/ozone-topology/test.sh b/hadoop-ozone/dist/src/main/compose/ozone-topology/test.sh
index d4efa4f4af00..392112ba5313 100755
--- a/hadoop-ozone/dist/src/main/compose/ozone-topology/test.sh
+++ b/hadoop-ozone/dist/src/main/compose/ozone-topology/test.sh
@@ -39,7 +39,7 @@ execute_robot_test scm topology/loaddata.robot
stop_containers datanode_1 datanode_2 datanode_3
-execute_robot_test scm topology/readdata.robot
+execute_robot_test scm -N readdata-first-half topology/readdata.robot
start_containers datanode_1 datanode_2 datanode_3
@@ -49,7 +49,7 @@ wait_for_port datanode_3 9858 60
stop_containers datanode_4 datanode_5 datanode_6
-execute_robot_test scm topology/readdata.robot
+execute_robot_test scm -N readdata-second-half topology/readdata.robot
stop_docker_env
diff --git a/hadoop-ozone/dist/src/main/compose/ozone/docker-config b/hadoop-ozone/dist/src/main/compose/ozone/docker-config
index 4d53acbad793..f100e823517a 100644
--- a/hadoop-ozone/dist/src/main/compose/ozone/docker-config
+++ b/hadoop-ozone/dist/src/main/compose/ozone/docker-config
@@ -14,8 +14,6 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-CORE-SITE.XML_fs.ofs.impl=org.apache.hadoop.fs.ozone.RootedOzoneFileSystem
-CORE-SITE.XML_fs.o3fs.impl=org.apache.hadoop.fs.ozone.OzoneFileSystem
OZONE-SITE.XML_ozone.om.address=om
OZONE-SITE.XML_ozone.om.http-address=om:9874
OZONE-SITE.XML_ozone.scm.container.size=1GB
diff --git a/hadoop-ozone/dist/src/main/compose/ozone/test.sh b/hadoop-ozone/dist/src/main/compose/ozone/test.sh
index e0b1d62ade08..2f57831f88bf 100755
--- a/hadoop-ozone/dist/src/main/compose/ozone/test.sh
+++ b/hadoop-ozone/dist/src/main/compose/ozone/test.sh
@@ -15,6 +15,8 @@
# See the License for the specific language governing permissions and
# limitations under the License.
+#suite:unsecure
+
COMPOSE_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
export COMPOSE_DIR
@@ -26,24 +28,24 @@ source "$COMPOSE_DIR/../testlib.sh"
start_docker_env
-#Due to the limitation of the current auditparser test, it should be the
-#first test in a clean cluster.
-
-#Disabling for now, audit parser tool during parse getting exception.
-#execute_robot_test om auditparser
-
execute_robot_test scm lib
+execute_robot_test scm ozone-lib
execute_robot_test scm basic
execute_robot_test scm gdpr
-execute_robot_test scm -v SCHEME:ofs ozonefs/ozonefs.robot
-execute_robot_test scm -v SCHEME:o3fs ozonefs/ozonefs.robot
+for scheme in ofs o3fs; do
+ for bucket in link bucket; do
+ execute_robot_test scm -v SCHEME:${scheme} -v BUCKET_TYPE:${bucket} -N ozonefs-${scheme}-${bucket} ozonefs/ozonefs.robot
+ done
+done
execute_robot_test scm security/ozone-secure-token.robot
-execute_robot_test scm s3
+for bucket in link generated; do
+ execute_robot_test scm -v BUCKET:${bucket} -N s3-${bucket} s3
+done
execute_robot_test scm recon
diff --git a/hadoop-ozone/dist/src/main/compose/ozonesecure-mr/docker-config b/hadoop-ozone/dist/src/main/compose/ozonesecure-mr/docker-config
index 0adafdfd60ea..7f1050db8b90 100644
--- a/hadoop-ozone/dist/src/main/compose/ozonesecure-mr/docker-config
+++ b/hadoop-ozone/dist/src/main/compose/ozonesecure-mr/docker-config
@@ -67,6 +67,7 @@ HDFS-SITE.XML_rpc.metrics.quantile.enable=true
HDFS-SITE.XML_rpc.metrics.percentiles.intervals=60,300
CORE-SITE.XML_fs.AbstractFileSystem.o3fs.impl=org.apache.hadoop.fs.ozone.OzFs
+CORE-SITE.XML_fs.AbstractFileSystem.ofs.impl=org.apache.hadoop.fs.ozone.RootedOzFs
CORE-SITE.XML_fs.defaultFS=o3fs://bucket1.volume1/
MAPRED-SITE.XML_mapreduce.framework.name=yarn
diff --git a/hadoop-ozone/dist/src/main/compose/ozonesecure-om-ha/docker-config b/hadoop-ozone/dist/src/main/compose/ozonesecure-om-ha/docker-config
index e245b7e01998..c332448a9508 100644
--- a/hadoop-ozone/dist/src/main/compose/ozonesecure-om-ha/docker-config
+++ b/hadoop-ozone/dist/src/main/compose/ozonesecure-om-ha/docker-config
@@ -14,7 +14,6 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-CORE-SITE.XML_fs.o3fs.impl=org.apache.hadoop.fs.ozone.OzoneFileSystem
CORE-SITE.XML_fs.defaultFS=o3fs://bucket.volume.id1
OZONE-SITE.XML_ozone.om.service.ids=id1
OZONE-SITE.XML_ozone.om.internal.service.id=id1
diff --git a/hadoop-ozone/dist/src/main/compose/ozonesecure/docker-config b/hadoop-ozone/dist/src/main/compose/ozonesecure/docker-config
index abb30453e9c7..30e1816ab8c1 100644
--- a/hadoop-ozone/dist/src/main/compose/ozonesecure/docker-config
+++ b/hadoop-ozone/dist/src/main/compose/ozonesecure/docker-config
@@ -14,8 +14,6 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-CORE-SITE.XML_fs.ofs.impl=org.apache.hadoop.fs.ozone.RootedOzoneFileSystem
-CORE-SITE.XML_fs.o3fs.impl=org.apache.hadoop.fs.ozone.OzoneFileSystem
OZONE-SITE.XML_ozone.om.volume.listall.allowed=false
OZONE-SITE.XML_ozone.om.address=om
diff --git a/hadoop-ozone/dist/src/main/compose/ozonesecure/test.sh b/hadoop-ozone/dist/src/main/compose/ozonesecure/test.sh
index 9c3f3ab83cc7..eeccb849eedc 100755
--- a/hadoop-ozone/dist/src/main/compose/ozonesecure/test.sh
+++ b/hadoop-ozone/dist/src/main/compose/ozonesecure/test.sh
@@ -15,6 +15,8 @@
# See the License for the specific language governing permissions and
# limitations under the License.
+#suite:secure
+
COMPOSE_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
export COMPOSE_DIR
@@ -23,18 +25,27 @@ source "$COMPOSE_DIR/../testlib.sh"
export SECURITY_ENABLED=true
+: ${OZONE_BUCKET_KEY_NAME:=key1}
+
start_docker_env
+execute_command_in_container kms hadoop key create ${OZONE_BUCKET_KEY_NAME}
+
execute_robot_test scm kinit.robot
execute_robot_test scm basic
execute_robot_test scm security
-execute_robot_test scm -v SCHEME:ofs ozonefs/ozonefs.robot
-execute_robot_test scm -v SCHEME:o3fs ozonefs/ozonefs.robot
+for scheme in ofs o3fs; do
+ for bucket in link bucket; do
+ execute_robot_test scm -v SCHEME:${scheme} -v BUCKET_TYPE:${bucket} -N ozonefs-${scheme}-${bucket} ozonefs/ozonefs.robot
+ done
+done
-execute_robot_test s3g s3
+for bucket in link generated; do
+ execute_robot_test s3g -v BUCKET:${bucket} -N s3-${bucket} s3
+done
#expects 4 pipelines, should be run before
#admincli which creates STANDALONE pipeline
diff --git a/hadoop-ozone/dist/src/main/compose/test-all.sh b/hadoop-ozone/dist/src/main/compose/test-all.sh
index e7f6f7123f09..45a3c52d52f1 100755
--- a/hadoop-ozone/dist/src/main/compose/test-all.sh
+++ b/hadoop-ozone/dist/src/main/compose/test-all.sh
@@ -25,31 +25,31 @@ PROJECT_DIR="$SCRIPT_DIR/.."
mkdir -p "$ALL_RESULT_DIR"
rm "$ALL_RESULT_DIR/*" || true
+source "$SCRIPT_DIR"/testlib.sh
+
if [ "$OZONE_WITH_COVERAGE" ]; then
java -cp "$PROJECT_DIR"/share/coverage/$(ls "$PROJECT_DIR"/share/coverage | grep test-util):"$PROJECT_DIR"/share/coverage/jacoco-core.jar org.apache.hadoop.test.JacocoServer &
DOCKER_BRIDGE_IP=$(docker network inspect bridge --format='{{(index .IPAM.Config 0).Gateway}}')
export HADOOP_OPTS="-javaagent:share/coverage/jacoco-agent.jar=output=tcpclient,address=$DOCKER_BRIDGE_IP,includes=org.apache.hadoop.ozone.*:org.apache.hadoop.hdds.*:org.apache.hadoop.fs.ozone.*"
fi
+tests=$(find_tests)
+cd "$SCRIPT_DIR"
+
RESULT=0
-IFS=$'\n'
# shellcheck disable=SC2044
-for test in $(find "$SCRIPT_DIR" -name test.sh | grep "${OZONE_TEST_SELECTOR:-""}" |sort); do
- echo "Executing test in $(dirname "$test")"
-
- #required to read the .env file from the right location
- cd "$(dirname "$test")" || continue
- ./test.sh
- ret=$?
- if [[ $ret -ne 0 ]]; then
- RESULT=1
- echo "ERROR: Test execution of $(dirname "$test") is FAILED!!!!"
+for t in ${tests}; do
+ d="$(dirname "${t}")"
+
+ if ! run_test_script "${d}"; then
+ RESULT=1
fi
- RESULT_DIR="$(dirname "$test")/result"
- cp "$RESULT_DIR"/robot-*.xml "$RESULT_DIR"/docker-*.log "$RESULT_DIR"/*.out* "$ALL_RESULT_DIR"/
+
+ copy_results "${d}" "${ALL_RESULT_DIR}"
done
-rebot -N "smoketests" -d "$SCRIPT_DIR/result" "$SCRIPT_DIR/result/robot-*.xml"
+rebot --nostatusrc -N acceptance -d "$ALL_RESULT_DIR" "$ALL_RESULT_DIR"/*.xml
+
if [ "$OZONE_WITH_COVERAGE" ]; then
pkill -f JacocoServer
cp /tmp/jacoco-combined.exec "$SCRIPT_DIR"/result
diff --git a/hadoop-ozone/dist/src/main/compose/testlib.sh b/hadoop-ozone/dist/src/main/compose/testlib.sh
index 15d1664ed80c..db449b90ad9c 100755
--- a/hadoop-ozone/dist/src/main/compose/testlib.sh
+++ b/hadoop-ozone/dist/src/main/compose/testlib.sh
@@ -17,7 +17,6 @@
set -e
COMPOSE_ENV_NAME=$(basename "$COMPOSE_DIR")
-COMPOSE_FILE=$COMPOSE_DIR/docker-compose.yaml
RESULT_DIR=${RESULT_DIR:-"$COMPOSE_DIR/result"}
RESULT_DIR_INSIDE="/tmp/smoketest/$(basename "$COMPOSE_ENV_NAME")/result"
SMOKETEST_DIR_INSIDE="${OZONE_DIR:-/opt/hadoop}/smoketest"
@@ -32,17 +31,39 @@ fi
## @description create results directory, purging any prior data
create_results_dir() {
#delete previous results
- rm -rf "$RESULT_DIR"
+ [[ "${OZONE_KEEP_RESULTS:-}" == "true" ]] || rm -rf "$RESULT_DIR"
mkdir -p "$RESULT_DIR"
#Should be writeable from the docker containers where user is different.
chmod ogu+w "$RESULT_DIR"
}
+## @description find all the test.sh scripts in the immediate child dirs
+find_tests(){
+ if [[ -n "${OZONE_ACCEPTANCE_SUITE}" ]]; then
+ tests=$(find . -mindepth 2 -maxdepth 2 -name test.sh | xargs grep -l "^#suite:${OZONE_ACCEPTANCE_SUITE}$" | sort)
+
+ # 'misc' is default suite, add untagged tests, too
+ if [[ "misc" == "${OZONE_ACCEPTANCE_SUITE}" ]]; then
+ untagged="$(find . -mindepth 2 -maxdepth 2 -name test.sh | xargs grep -L "^#suite:")"
+ if [[ -n "${untagged}" ]]; then
+ tests=$(echo ${tests} ${untagged} | xargs -n1 | sort)
+ fi
+ fi
+
+ if [[ -z "${tests}" ]]; then
+ echo "No tests found for suite ${OZONE_ACCEPTANCE_SUITE}"
+ exit 1
+ fi
+ else
+ tests=$(find . -mindepth 2 -maxdepth 2 -name test.sh | grep "${OZONE_TEST_SELECTOR:-""}" | sort)
+ fi
+ echo $tests
+}
## @description wait until safemode exit (or 180 seconds)
-## @param the docker-compose file
wait_for_safemode_exit(){
- local compose_file=$1
+ # version-dependent
+ : ${OZONE_SAFEMODE_STATUS_COMMAND:=ozone admin safemode status --verbose}
#Reset the timer
SECONDS=0
@@ -51,11 +72,11 @@ wait_for_safemode_exit(){
while [[ $SECONDS -lt 180 ]]; do
#This line checks the safemode status in scm
- local command="ozone admin safemode status"
+ local command="${OZONE_SAFEMODE_STATUS_COMMAND}"
if [[ "${SECURITY_ENABLED}" == 'true' ]]; then
- status=$(docker-compose -f "${compose_file}" exec -T scm bash -c "kinit -k HTTP/scm@EXAMPLE.COM -t /etc/security/keytabs/HTTP.keytab && $command" || true)
+ status=$(docker-compose exec -T scm bash -c "kinit -k HTTP/scm@EXAMPLE.COM -t /etc/security/keytabs/HTTP.keytab && $command" || true)
else
- status=$(docker-compose -f "${compose_file}" exec -T scm bash -c "$command")
+ status=$(docker-compose exec -T scm bash -c "$command")
fi
echo $status
@@ -80,9 +101,9 @@ start_docker_env(){
create_results_dir
export OZONE_SAFEMODE_MIN_DATANODES="${datanode_count}"
- docker-compose -f "$COMPOSE_FILE" --no-ansi down
- if ! { docker-compose -f "$COMPOSE_FILE" --no-ansi up -d --scale datanode="${datanode_count}" \
- && wait_for_safemode_exit "$COMPOSE_FILE"; }; then
+ docker-compose --no-ansi down
+ if ! { docker-compose --no-ansi up -d --scale datanode="${datanode_count}" \
+ && wait_for_safemode_exit ; }; then
OUTPUT_NAME="$COMPOSE_ENV_NAME"
stop_docker_env
return 1
@@ -114,11 +135,11 @@ execute_robot_test(){
OUTPUT_PATH="$RESULT_DIR_INSIDE/${OUTPUT_FILE}"
# shellcheck disable=SC2068
- docker-compose -f "$COMPOSE_FILE" exec -T "$CONTAINER" mkdir -p "$RESULT_DIR_INSIDE" \
- && docker-compose -f "$COMPOSE_FILE" exec -T "$CONTAINER" robot -v OM_SERVICE_ID:"${OM_SERVICE_ID}" -v SECURITY_ENABLED:"${SECURITY_ENABLED}" -v OM_HA_PARAM:"${OM_HA_PARAM}" ${ARGUMENTS[@]} --log NONE -N "$TEST_NAME" --report NONE "${OZONE_ROBOT_OPTS[@]}" --output "$OUTPUT_PATH" "$SMOKETEST_DIR_INSIDE/$TEST"
+ docker-compose exec -T "$CONTAINER" mkdir -p "$RESULT_DIR_INSIDE" \
+ && docker-compose exec -T "$CONTAINER" robot -v OM_SERVICE_ID:"${OM_SERVICE_ID}" -v SECURITY_ENABLED:"${SECURITY_ENABLED}" -v OM_HA_PARAM:"${OM_HA_PARAM}" -v KEY_NAME:"${OZONE_BUCKET_KEY_NAME}" ${ARGUMENTS[@]} --log NONE --report NONE "${OZONE_ROBOT_OPTS[@]}" --output "$OUTPUT_PATH" "$SMOKETEST_DIR_INSIDE/$TEST"
local -i rc=$?
- FULL_CONTAINER_NAME=$(docker-compose -f "$COMPOSE_FILE" ps | grep "_${CONTAINER}_" | head -n 1 | awk '{print $1}')
+ FULL_CONTAINER_NAME=$(docker-compose ps | grep "_${CONTAINER}_" | head -n 1 | awk '{print $1}')
docker cp "$FULL_CONTAINER_NAME:$OUTPUT_PATH" "$RESULT_DIR/"
copy_daemon_logs
@@ -135,7 +156,7 @@ execute_robot_test(){
## @description Copy any 'out' files for daemon processes to the result dir
copy_daemon_logs() {
local c f
- for c in $(docker-compose -f "$COMPOSE_FILE" ps | grep "^${COMPOSE_ENV_NAME}_" | awk '{print $1}'); do
+ for c in $(docker-compose ps | grep "^${COMPOSE_ENV_NAME}_" | awk '{print $1}'); do
for f in $(docker exec "${c}" ls -1 /var/log/hadoop | grep -F '.out'); do
docker cp "${c}:/var/log/hadoop/${f}" "$RESULT_DIR/"
done
@@ -149,7 +170,7 @@ copy_daemon_logs() {
execute_command_in_container(){
set -e
# shellcheck disable=SC2068
- docker-compose -f "$COMPOSE_FILE" exec -T "$@"
+ docker-compose exec -T "$@"
set +e
}
@@ -157,7 +178,7 @@ execute_command_in_container(){
## @param List of container names, eg datanode_1 datanode_2
stop_containers() {
set -e
- docker-compose -f "$COMPOSE_FILE" --no-ansi stop $@
+ docker-compose --no-ansi stop $@
set +e
}
@@ -166,7 +187,7 @@ stop_containers() {
## @param List of container names, eg datanode_1 datanode_2
start_containers() {
set -e
- docker-compose -f "$COMPOSE_FILE" --no-ansi start $@
+ docker-compose --no-ansi start $@
set +e
}
@@ -185,7 +206,7 @@ wait_for_port(){
while [[ $SECONDS -lt $timeout ]]; do
set +e
- docker-compose -f "${COMPOSE_FILE}" exec -T scm /bin/bash -c "nc -z $host $port"
+ docker-compose exec -T scm /bin/bash -c "nc -z $host $port"
status=$?
set -e
if [ $status -eq 0 ] ; then
@@ -202,9 +223,9 @@ wait_for_port(){
## @description Stops a docker-compose based test environment (with saving the logs)
stop_docker_env(){
- docker-compose -f "$COMPOSE_FILE" --no-ansi logs > "$RESULT_DIR/docker-$OUTPUT_NAME.log"
+ docker-compose --no-ansi logs > "$RESULT_DIR/docker-$OUTPUT_NAME.log"
if [ "${KEEP_RUNNING:-false}" = false ]; then
- docker-compose -f "$COMPOSE_FILE" --no-ansi down
+ docker-compose --no-ansi down
fi
}
@@ -226,3 +247,39 @@ generate_report(){
exit 1
fi
}
+
+## @description Copy results of a single test environment to the "all tests" dir.
+copy_results() {
+ local test_dir="$1"
+ local all_result_dir="$2"
+
+ local result_dir="${test_dir}/result"
+ local test_dir_name=$(basename ${test_dir})
+ if [[ -n "$(find "${result_dir}" -name "*.xml")" ]]; then
+ rebot --nostatusrc -N "${test_dir_name}" -o "${all_result_dir}/${test_dir_name}.xml" "${result_dir}/*.xml"
+ fi
+
+ cp "${result_dir}"/docker-*.log "${all_result_dir}"/
+ if [[ -n "$(find "${result_dir}" -name "*.out")" ]]; then
+ cp "${result_dir}"/*.out* "${all_result_dir}"/
+ fi
+}
+
+run_test_script() {
+ local d="$1"
+
+ echo "Executing test in ${d}"
+
+ #required to read the .env file from the right location
+ cd "${d}" || return
+
+ ret=0
+ if ! ./test.sh; then
+ ret=1
+ echo "ERROR: Test execution of ${d} is FAILED!!!!"
+ fi
+
+ cd - > /dev/null
+
+ return ${ret}
+}
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/.env b/hadoop-ozone/dist/src/main/compose/upgrade/.env
new file mode 100644
index 000000000000..6f757c552b2e
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/.env
@@ -0,0 +1,21 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+HDDS_VERSION=${hdds.version}
+OZONE_RUNNER_VERSION=${docker.ozone-runner.version}
+OZONE_IMAGE=apache/ozone-runner:${docker.ozone-runner.version}
+OZONE_DIR=/opt/hadoop
+OZONE_VOLUME=.
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/README.md b/hadoop-ozone/dist/src/main/compose/upgrade/README.md
new file mode 100644
index 000000000000..5d844e006b50
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/README.md
@@ -0,0 +1,29 @@
+
+
+# Compose file for upgrade
+
+This directory contains a sample cluster definition and script for
+testing upgrade from previous version to the current one.
+
+Data for each container is persisted in mounted volume (by default it's
+`data` under the `compose/upgrade` directory, but can be overridden via
+`OZONE_VOLUME` environment variable).
+
+Prior version is run using an official `apache/ozone` image, while the
+current version is run with the `ozone-runner` image using locally built
+source code.
+
+Currently the test script only supports a single version upgrade (eg.
+from 0.5.0 to 1.0.0).
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/upgrade/docker-compose.yaml
new file mode 100644
index 000000000000..ae527593dcd4
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/docker-compose.yaml
@@ -0,0 +1,127 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+version: "3.4"
+
+# reusable fragments (see https://docs.docker.com/compose/compose-file/#extension-fields)
+x-common-config:
+ &common-config
+ env_file:
+ - docker-config
+ image: ${OZONE_IMAGE}
+
+x-replication:
+ &replication
+ OZONE-SITE.XML_ozone.replication: ${OZONE_REPLICATION_FACTOR:-1}
+
+x-datanode:
+ &datanode
+ command: ["ozone","datanode"]
+ <<: *common-config
+ environment:
+ <<: *replication
+ ports:
+ - 9864
+ - 9882
+
+services:
+ dn1:
+ <<: *datanode
+ networks:
+ net:
+ ipv4_address: 10.9.0.11
+ volumes:
+ - ${OZONE_VOLUME}/dn1:/data
+ - ../..:${OZONE_DIR}
+ dn2:
+ <<: *datanode
+ networks:
+ net:
+ ipv4_address: 10.9.0.12
+ volumes:
+ - ${OZONE_VOLUME}/dn2:/data
+ - ../..:${OZONE_DIR}
+ dn3:
+ <<: *datanode
+ networks:
+ net:
+ ipv4_address: 10.9.0.13
+ volumes:
+ - ${OZONE_VOLUME}/dn3:/data
+ - ../..:${OZONE_DIR}
+ om:
+ command: ["ozone","om"]
+ <<: *common-config
+ environment:
+ ENSURE_OM_INITIALIZED: /data/metadata/om/current/VERSION
+ <<: *replication
+ networks:
+ net:
+ ipv4_address: 10.9.0.14
+ ports:
+ - 9874:9874
+ - 9862:9862
+ volumes:
+ - ${OZONE_VOLUME}/om:/data
+ - ../..:${OZONE_DIR}
+ recon:
+ command: ["ozone","recon"]
+ <<: *common-config
+ environment:
+ <<: *replication
+ networks:
+ net:
+ ipv4_address: 10.9.0.15
+ ports:
+ - 9888:9888
+ volumes:
+ - ${OZONE_VOLUME}/recon:/data
+ - ../..:${OZONE_DIR}
+ s3g:
+ command: ["ozone","s3g"]
+ <<: *common-config
+ environment:
+ <<: *replication
+ networks:
+ net:
+ ipv4_address: 10.9.0.16
+ ports:
+ - 9878:9878
+ volumes:
+ - ${OZONE_VOLUME}/s3g:/data
+ - ../..:${OZONE_DIR}
+ scm:
+ command: ["ozone","scm"]
+ <<: *common-config
+ environment:
+ ENSURE_SCM_INITIALIZED: /data/metadata/scm/current/VERSION
+ OZONE-SITE.XML_hdds.scm.safemode.min.datanode: ${OZONE_SAFEMODE_MIN_DATANODES:-1}
+ <<: *replication
+ networks:
+ net:
+ ipv4_address: 10.9.0.17
+ ports:
+ - 9876:9876
+ volumes:
+ - ${OZONE_VOLUME}/scm:/data
+ - ../..:${OZONE_DIR}
+
+networks:
+ net:
+ driver: bridge
+ ipam:
+ config:
+ - subnet: 10.9.0.0/16
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/docker-config b/hadoop-ozone/dist/src/main/compose/upgrade/docker-config
new file mode 100644
index 000000000000..ecc994b4ac26
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/docker-config
@@ -0,0 +1,33 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+CORE-SITE.XML_fs.ofs.impl=org.apache.hadoop.fs.ozone.RootedOzoneFileSystem
+CORE-SITE.XML_fs.o3fs.impl=org.apache.hadoop.fs.ozone.OzoneFileSystem
+OZONE-SITE.XML_ozone.om.address=om
+OZONE-SITE.XML_ozone.om.http-address=om:9874
+OZONE-SITE.XML_ozone.scm.container.size=1GB
+OZONE-SITE.XML_ozone.scm.pipeline.owner.container.count=1
+OZONE-SITE.XML_ozone.scm.names=scm
+OZONE-SITE.XML_ozone.scm.datanode.id.dir=/data
+OZONE-SITE.XML_ozone.scm.block.client.address=scm
+OZONE-SITE.XML_ozone.metadata.dirs=/data/metadata
+OZONE-SITE.XML_ozone.recon.db.dir=/data/metadata/recon
+OZONE-SITE.XML_ozone.scm.client.address=scm
+OZONE-SITE.XML_hdds.datanode.dir=/data/hdds
+OZONE-SITE.XML_ozone.recon.address=recon:9891
+OZONE-SITE.XML_recon.om.snapshot.task.interval.delay=1m
+
+no_proxy=om,scm,s3g,kdc,localhost,127.0.0.1
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/test.sh b/hadoop-ozone/dist/src/main/compose/upgrade/test.sh
new file mode 100644
index 000000000000..0c51325b7c2d
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/test.sh
@@ -0,0 +1,70 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+COMPOSE_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
+export COMPOSE_DIR
+
+: "${OZONE_REPLICATION_FACTOR:=3}"
+: "${OZONE_UPGRADE_FROM:="0.5.0"}"
+: "${OZONE_UPGRADE_TO:="1.0.0"}"
+: "${OZONE_VOLUME:="${COMPOSE_DIR}/data"}"
+
+export OZONE_VOLUME
+
+mkdir -p "${OZONE_VOLUME}"/{dn1,dn2,dn3,om,recon,s3g,scm}
+mkdir -p "${OZONE_VOLUME}/debug"
+
+if [[ -n "${OZONE_VOLUME_OWNER}" ]]; then
+ current_user=$(whoami)
+ if [[ "${OZONE_VOLUME_OWNER}" != "${current_user}" ]]; then
+ chown -R "${OZONE_VOLUME_OWNER}" "${OZONE_VOLUME}" \
+ || sudo chown -R "${OZONE_VOLUME_OWNER}" "${OZONE_VOLUME}"
+ fi
+fi
+
+# define version-specifics
+export OZONE_DIR=/opt/ozone
+export OZONE_IMAGE="apache/ozone:${OZONE_UPGRADE_FROM}"
+# shellcheck source=/dev/null
+source "${COMPOSE_DIR}/versions/ozone-${OZONE_UPGRADE_FROM}.sh"
+# shellcheck source=/dev/null
+source "${COMPOSE_DIR}/../testlib.sh"
+
+# prepare pre-upgrade cluster
+start_docker_env
+execute_robot_test scm topology/loaddata.robot
+stop_docker_env
+
+# run upgrade scripts
+SCRIPT_DIR=../../libexec/upgrade
+[[ -f "${SCRIPT_DIR}/${OZONE_UPGRADE_TO}.sh" ]] && "${SCRIPT_DIR}/${OZONE_UPGRADE_TO}.sh"
+
+# update version-specifics
+export OZONE_DIR=/opt/hadoop
+unset OZONE_IMAGE # use apache/ozone-runner defined in docker-compose.yaml
+# shellcheck source=/dev/null
+source "${COMPOSE_DIR}/versions/ozone-${OZONE_UPGRADE_TO}.sh"
+# shellcheck source=/dev/null
+source "${COMPOSE_DIR}/../testlib.sh"
+
+# re-start cluster with new version and check after upgrade
+export OZONE_KEEP_RESULTS=true
+start_docker_env
+execute_robot_test scm topology/readdata.robot
+stop_docker_env
+
+generate_report
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/versions/README.md b/hadoop-ozone/dist/src/main/compose/upgrade/versions/README.md
new file mode 100644
index 000000000000..24cd113469a6
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/versions/README.md
@@ -0,0 +1,15 @@
+
+
+The scripts in this directory define version-specific behavior required for [`testlib.sh`](../../testlib.sh). For example the `ozone admin` command was renamed from `ozone scmcli` in 1.0.0.
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/versions/ozone-0.5.0.sh b/hadoop-ozone/dist/src/main/compose/upgrade/versions/ozone-0.5.0.sh
new file mode 100644
index 000000000000..667ce959026a
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/versions/ozone-0.5.0.sh
@@ -0,0 +1,19 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+export OZONE_ADMIN_COMMAND=scmcli
+export OZONE_SAFEMODE_STATUS_COMMAND='ozone scmcli safemode status'
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/versions/ozone-1.0.0.sh b/hadoop-ozone/dist/src/main/compose/upgrade/versions/ozone-1.0.0.sh
new file mode 100644
index 000000000000..3ff23e0441bc
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/versions/ozone-1.0.0.sh
@@ -0,0 +1,19 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+export OZONE_ADMIN_COMMAND=admin
+export OZONE_SAFEMODE_STATUS_COMMAND='ozone admin safemode status --verbose'
diff --git a/hadoop-ozone/dist/src/main/k8s/.gitignore b/hadoop-ozone/dist/src/main/k8s/.gitignore
new file mode 100644
index 000000000000..bb9ee6087c6f
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/k8s/.gitignore
@@ -0,0 +1,15 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+result
diff --git a/hadoop-ozone/dist/src/main/k8s/definitions/ozone/config.yaml b/hadoop-ozone/dist/src/main/k8s/definitions/ozone/config.yaml
index 6e638915a247..124f72ff5e58 100644
--- a/hadoop-ozone/dist/src/main/k8s/definitions/ozone/config.yaml
+++ b/hadoop-ozone/dist/src/main/k8s/definitions/ozone/config.yaml
@@ -26,6 +26,7 @@ data:
OZONE-SITE.XML_ozone.om.address: "om-0.om"
OZONE-SITE.XML_ozone.scm.client.address: "scm-0.scm"
OZONE-SITE.XML_ozone.scm.names: "scm-0.scm"
+ OZONE-SITE.XML_hdds.scm.safemode.min.datanode: "3"
LOG4J.PROPERTIES_log4j.rootLogger: "INFO, stdout"
LOG4J.PROPERTIES_log4j.appender.stdout: "org.apache.log4j.ConsoleAppender"
LOG4J.PROPERTIES_log4j.appender.stdout.layout: "org.apache.log4j.PatternLayout"
diff --git a/hadoop-ozone/dist/src/main/k8s/definitions/ozone/definitions/onenode.yaml b/hadoop-ozone/dist/src/main/k8s/definitions/ozone/definitions/onenode.yaml
index 882477936adf..19a3e1dd7409 100644
--- a/hadoop-ozone/dist/src/main/k8s/definitions/ozone/definitions/onenode.yaml
+++ b/hadoop-ozone/dist/src/main/k8s/definitions/ozone/definitions/onenode.yaml
@@ -19,7 +19,7 @@ description: remove scheduling rules to make it possible to run multiple datanod
- type: Remove
trigger:
metadata:
- name: ozone-datanode
+ name: datanode
path:
- spec
- template
diff --git a/hadoop-ozone/dist/src/main/k8s/definitions/pv-test/flekszible.yaml b/hadoop-ozone/dist/src/main/k8s/definitions/test-webserver/flekszible.yaml
similarity index 100%
rename from hadoop-ozone/dist/src/main/k8s/definitions/pv-test/flekszible.yaml
rename to hadoop-ozone/dist/src/main/k8s/definitions/test-webserver/flekszible.yaml
diff --git a/hadoop-ozone/dist/src/main/k8s/definitions/pv-test/webserver-deployment.yaml b/hadoop-ozone/dist/src/main/k8s/definitions/test-webserver/webserver-deployment.yaml
similarity index 100%
rename from hadoop-ozone/dist/src/main/k8s/definitions/pv-test/webserver-deployment.yaml
rename to hadoop-ozone/dist/src/main/k8s/definitions/test-webserver/webserver-deployment.yaml
diff --git a/hadoop-ozone/dist/src/main/k8s/definitions/pv-test/webserver-service.yaml b/hadoop-ozone/dist/src/main/k8s/definitions/test-webserver/webserver-service.yaml
similarity index 100%
rename from hadoop-ozone/dist/src/main/k8s/definitions/pv-test/webserver-service.yaml
rename to hadoop-ozone/dist/src/main/k8s/definitions/test-webserver/webserver-service.yaml
diff --git a/hadoop-ozone/dist/src/main/k8s/definitions/pv-test/webserver-volume.yaml b/hadoop-ozone/dist/src/main/k8s/definitions/test-webserver/webserver-volume.yaml
similarity index 100%
rename from hadoop-ozone/dist/src/main/k8s/definitions/pv-test/webserver-volume.yaml
rename to hadoop-ozone/dist/src/main/k8s/definitions/test-webserver/webserver-volume.yaml
diff --git a/hadoop-ozone/dist/src/main/k8s/examples/getting-started/config-configmap.yaml b/hadoop-ozone/dist/src/main/k8s/examples/getting-started/config-configmap.yaml
index 00fb72b14e50..f02fb56f089c 100644
--- a/hadoop-ozone/dist/src/main/k8s/examples/getting-started/config-configmap.yaml
+++ b/hadoop-ozone/dist/src/main/k8s/examples/getting-started/config-configmap.yaml
@@ -26,6 +26,7 @@ data:
OZONE-SITE.XML_ozone.om.address: om-0.om
OZONE-SITE.XML_ozone.scm.client.address: scm-0.scm
OZONE-SITE.XML_ozone.scm.names: scm-0.scm
+ OZONE-SITE.XML_hdds.scm.safemode.min.datanode: "3"
LOG4J.PROPERTIES_log4j.rootLogger: INFO, stdout
LOG4J.PROPERTIES_log4j.appender.stdout: org.apache.log4j.ConsoleAppender
LOG4J.PROPERTIES_log4j.appender.stdout.layout: org.apache.log4j.PatternLayout
diff --git a/hadoop-ozone/dist/src/main/k8s/examples/getting-started/datanode-statefulset.yaml b/hadoop-ozone/dist/src/main/k8s/examples/getting-started/datanode-statefulset.yaml
index c393eada79d6..db91864bdaf3 100644
--- a/hadoop-ozone/dist/src/main/k8s/examples/getting-started/datanode-statefulset.yaml
+++ b/hadoop-ozone/dist/src/main/k8s/examples/getting-started/datanode-statefulset.yaml
@@ -37,16 +37,6 @@ spec:
prometheus.io/port: "9882"
prometheus.io/path: /prom
spec:
- affinity:
- podAntiAffinity:
- requiredDuringSchedulingIgnoredDuringExecution:
- - labelSelector:
- matchExpressions:
- - key: component
- operator: In
- values:
- - datanode
- topologyKey: kubernetes.io/hostname
securityContext:
fsGroup: 1000
containers:
diff --git a/hadoop-ozone/dist/src/main/k8s/examples/getting-started/test.sh b/hadoop-ozone/dist/src/main/k8s/examples/getting-started/test.sh
new file mode 100755
index 000000000000..7d6bdfb981e0
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/k8s/examples/getting-started/test.sh
@@ -0,0 +1,39 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+export K8S_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
+
+cd "$K8S_DIR"
+
+# shellcheck source=/dev/null
+source "../testlib.sh"
+
+rm -rf result
+
+regenerate_resources
+
+start_k8s_env
+
+execute_robot_test scm-0 smoketest/basic/basic.robot
+
+combine_reports
+
+get_logs
+
+stop_k8s_env
+
+revert_resources
diff --git a/hadoop-ozone/dist/src/main/k8s/examples/minikube/config-configmap.yaml b/hadoop-ozone/dist/src/main/k8s/examples/minikube/config-configmap.yaml
index 00fb72b14e50..f02fb56f089c 100644
--- a/hadoop-ozone/dist/src/main/k8s/examples/minikube/config-configmap.yaml
+++ b/hadoop-ozone/dist/src/main/k8s/examples/minikube/config-configmap.yaml
@@ -26,6 +26,7 @@ data:
OZONE-SITE.XML_ozone.om.address: om-0.om
OZONE-SITE.XML_ozone.scm.client.address: scm-0.scm
OZONE-SITE.XML_ozone.scm.names: scm-0.scm
+ OZONE-SITE.XML_hdds.scm.safemode.min.datanode: "3"
LOG4J.PROPERTIES_log4j.rootLogger: INFO, stdout
LOG4J.PROPERTIES_log4j.appender.stdout: org.apache.log4j.ConsoleAppender
LOG4J.PROPERTIES_log4j.appender.stdout.layout: org.apache.log4j.PatternLayout
diff --git a/hadoop-ozone/dist/src/main/k8s/examples/minikube/test.sh b/hadoop-ozone/dist/src/main/k8s/examples/minikube/test.sh
new file mode 100755
index 000000000000..7d6bdfb981e0
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/k8s/examples/minikube/test.sh
@@ -0,0 +1,39 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+export K8S_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
+
+cd "$K8S_DIR"
+
+# shellcheck source=/dev/null
+source "../testlib.sh"
+
+rm -rf result
+
+regenerate_resources
+
+start_k8s_env
+
+execute_robot_test scm-0 smoketest/basic/basic.robot
+
+combine_reports
+
+get_logs
+
+stop_k8s_env
+
+revert_resources
diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dev/Flekszible b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dev/Flekszible
index 350ea73c50b7..3d9bfcd6ce2d 100644
--- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dev/Flekszible
+++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dev/Flekszible
@@ -37,11 +37,11 @@ import:
- type: Image
image: "@docker.image@"
- type: ozone/tracing
- - path: pv-test
+ - path: test-webserver
destination: pv-test
- path: ozone-csi
destination: csi
- - path: pv-test
+ - path: test-webserver
destination: pv-test
transformations:
- type: Namespace
diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dev/config-configmap.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dev/config-configmap.yaml
index 82f303fafe4e..58076303fbe3 100644
--- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dev/config-configmap.yaml
+++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dev/config-configmap.yaml
@@ -26,6 +26,7 @@ data:
OZONE-SITE.XML_ozone.om.address: om-0.om
OZONE-SITE.XML_ozone.scm.client.address: scm-0.scm
OZONE-SITE.XML_ozone.scm.names: scm-0.scm
+ OZONE-SITE.XML_hdds.scm.safemode.min.datanode: "3"
LOG4J.PROPERTIES_log4j.rootLogger: INFO, stdout
LOG4J.PROPERTIES_log4j.appender.stdout: org.apache.log4j.ConsoleAppender
LOG4J.PROPERTIES_log4j.appender.stdout.layout: org.apache.log4j.PatternLayout
diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dev/datanode-statefulset.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dev/datanode-statefulset.yaml
index 475ce690b643..b22212ff79ea 100644
--- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dev/datanode-statefulset.yaml
+++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dev/datanode-statefulset.yaml
@@ -37,16 +37,6 @@ spec:
prometheus.io/port: "9882"
prometheus.io/path: /prom
spec:
- affinity:
- podAntiAffinity:
- requiredDuringSchedulingIgnoredDuringExecution:
- - labelSelector:
- matchExpressions:
- - key: component
- operator: In
- values:
- - datanode
- topologyKey: kubernetes.io/hostname
securityContext:
fsGroup: 1000
containers:
diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dev/test.sh b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dev/test.sh
new file mode 100755
index 000000000000..7d6bdfb981e0
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dev/test.sh
@@ -0,0 +1,39 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+export K8S_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
+
+cd "$K8S_DIR"
+
+# shellcheck source=/dev/null
+source "../testlib.sh"
+
+rm -rf result
+
+regenerate_resources
+
+start_k8s_env
+
+execute_robot_test scm-0 smoketest/basic/basic.robot
+
+combine_reports
+
+get_logs
+
+stop_k8s_env
+
+revert_resources
diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone/Flekszible b/hadoop-ozone/dist/src/main/k8s/examples/ozone/Flekszible
index 2fb527c0a45f..ec6d74533baa 100644
--- a/hadoop-ozone/dist/src/main/k8s/examples/ozone/Flekszible
+++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone/Flekszible
@@ -31,7 +31,7 @@ import:
destination: pv-test
- path: ozone-csi
destination: csi
- - path: pv-test
+ - path: test-webserver
destination: pv-test
transformations:
- type: Namespace
diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone/config-configmap.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone/config-configmap.yaml
index c7e8f486e89f..820c1977b4eb 100644
--- a/hadoop-ozone/dist/src/main/k8s/examples/ozone/config-configmap.yaml
+++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone/config-configmap.yaml
@@ -26,6 +26,7 @@ data:
OZONE-SITE.XML_ozone.om.address: om-0.om
OZONE-SITE.XML_ozone.scm.client.address: scm-0.scm
OZONE-SITE.XML_ozone.scm.names: scm-0.scm
+ OZONE-SITE.XML_hdds.scm.safemode.min.datanode: "3"
LOG4J.PROPERTIES_log4j.rootLogger: INFO, stdout
LOG4J.PROPERTIES_log4j.appender.stdout: org.apache.log4j.ConsoleAppender
LOG4J.PROPERTIES_log4j.appender.stdout.layout: org.apache.log4j.PatternLayout
diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone/test.sh b/hadoop-ozone/dist/src/main/k8s/examples/ozone/test.sh
new file mode 100755
index 000000000000..7d6bdfb981e0
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone/test.sh
@@ -0,0 +1,39 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+export K8S_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
+
+cd "$K8S_DIR"
+
+# shellcheck source=/dev/null
+source "../testlib.sh"
+
+rm -rf result
+
+regenerate_resources
+
+start_k8s_env
+
+execute_robot_test scm-0 smoketest/basic/basic.robot
+
+combine_reports
+
+get_logs
+
+stop_k8s_env
+
+revert_resources
diff --git a/hadoop-ozone/dist/src/main/k8s/examples/test-all.sh b/hadoop-ozone/dist/src/main/k8s/examples/test-all.sh
new file mode 100755
index 000000000000..ae810c9f679c
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/k8s/examples/test-all.sh
@@ -0,0 +1,49 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+
+#
+# Test executor to test all the compose/*/test.sh test scripts.
+#
+SCRIPT_DIR=$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null && pwd )
+
+set -ex
+
+ALL_RESULT_DIR="$SCRIPT_DIR/result"
+rm "$ALL_RESULT_DIR/*" || true
+mkdir -p "$ALL_RESULT_DIR"
+
+RESULT=0
+IFS=$'\n'
+# shellcheck disable=SC2044
+for test in $(find "$SCRIPT_DIR" -name test.sh | grep "${OZONE_TEST_SELECTOR:-""}" |sort); do
+ TEST_DIR="$(dirname $test)"
+ TEST_NAME="$(basename "$TEST_DIR")"
+
+ echo ""
+ echo "#### Executing tests of ${TEST_DIR} #####"
+ echo ""
+ cd "$TEST_DIR" || continue
+ ./test.sh
+
+ cp "$TEST_DIR"/result/output.xml "$ALL_RESULT_DIR"/"${TEST_NAME}".xml
+ mkdir -p "$ALL_RESULT_DIR"/"${TEST_NAME}"
+ mv "$TEST_DIR"/logs/*log "$ALL_RESULT_DIR"/"${TEST_NAME}"/
+done
+
+rebot -N "smoketests" -d "$ALL_RESULT_DIR/" "$ALL_RESULT_DIR/*.xml"
+
diff --git a/hadoop-ozone/dist/src/main/k8s/examples/testlib.sh b/hadoop-ozone/dist/src/main/k8s/examples/testlib.sh
new file mode 100644
index 000000000000..2442cb9a70fe
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/k8s/examples/testlib.sh
@@ -0,0 +1,144 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+retry() {
+ n=0
+ until [ $n -ge 100 ]
+ do
+ "$@" && break
+ n=$[$n+1]
+ echo "$n '$@' is failed..."
+ sleep ${RETRY_SLEEP:-3}
+ done
+ if [ $n -eq 100 ]; then
+ return 255
+ fi
+}
+
+grep_log() {
+ CONTAINER="$1"
+ PATTERN="$2"
+ kubectl logs "$1" | grep "$PATTERN"
+}
+
+wait_for_startup(){
+ print_phase "Waiting until the k8s cluster is running"
+ retry all_pods_are_running
+ retry grep_log scm-0 "SCM exiting safe mode."
+ retry grep_log om-0 "HTTP server of ozoneManager listening"
+ print_phase "Cluster is up and running"
+}
+
+all_pods_are_running() {
+ RUNNING_COUNT=$(kubectl get pod --field-selector status.phase=Running | wc -l)
+ ALL_COUNT=$(kubectl get pod | wc -l)
+ RUNNING_COUNT=$((RUNNING_COUNT - 1))
+ ALL_COUNT=$((ALL_COUNT - 1))
+ if [ "$RUNNING_COUNT" -lt "3" ]; then
+ echo "$RUNNING_COUNT pods are running. Waiting for more."
+ return 1
+ elif [ "$RUNNING_COUNT" -ne "$ALL_COUNT" ]; then
+ echo "$RUNNING_COUNT pods are running out from the $ALL_COUNT"
+ return 2
+ else
+ STARTED=true
+ return 0
+ fi
+}
+
+start_k8s_env() {
+ print_phase "Deleting existing k8s resources"
+ #reset environment
+ kubectl delete statefulset --all
+ kubectl delete daemonset --all
+ kubectl delete deployment --all
+ kubectl delete service --all
+ kubectl delete configmap --all
+ kubectl delete pod --all
+ kubectl delete pvc --all
+ kubectl delete pv --all
+
+ print_phase "Applying k8s resources from $1"
+ kubectl apply -f .
+ wait_for_startup
+}
+
+get_logs() {
+ mkdir -p logs
+ for pod in $(kubectl get pods -o custom-columns=NAME:.metadata.name | tail -n +2); do
+ kubectl logs "${pod}" > "logs/pod-${pod}.log"
+ done
+}
+
+stop_k8s_env() {
+ if [ ! "$KEEP_RUNNING" ]; then
+ kubectl delete -f .
+ fi
+}
+
+regenerate_resources() {
+ print_phase "Modifying Kubernetes resources file for test"
+ echo " (mounting current Ozone directory to the containers, scheduling containers to one node, ...)"
+ echo ""
+ echo "WARNING: this test can be executed only with local Kubernetes cluster"
+ echo " (source dir should be available from K8s nodes)"
+ echo ""
+
+ PARENT_OF_PARENT=$(realpath ../..)
+
+ if [ $(basename $PARENT_OF_PARENT) == "k8s" ]; then
+ #running from src dir
+ OZONE_ROOT=$(realpath ../../../../../target/ozone-0.6.0-SNAPSHOT)
+ else
+ #running from dist
+ OZONE_ROOT=$(realpath ../../..)
+ fi
+
+ flekszible generate -t mount:hostPath="$OZONE_ROOT",path=/opt/hadoop -t image:image=apache/ozone-runner:20200420-1 -t ozone/onenode
+}
+
+revert_resources() {
+ print_phase "Regenerating original Kubernetes resource files"
+ flekszible generate
+}
+
+execute_robot_test() {
+ print_phase "Executing robot tests $@"
+ mkdir -p result
+
+ CONTAINER="$1"
+ shift 1 #Remove first argument which was the container name
+
+ # shellcheck disable=SC2206
+ ARGUMENTS=($@)
+
+ kubectl exec -it "${CONTAINER}" -- bash -c 'rm -rf /tmp/report'
+ kubectl exec -it "${CONTAINER}" -- bash -c 'mkdir -p /tmp/report'
+ kubectl exec -it "${CONTAINER}" -- robot --nostatusrc -d /tmp/report ${ARGUMENTS[@]} || true
+ kubectl cp "${CONTAINER}":/tmp/report/output.xml "result/$CONTAINER-$RANDOM.xml" || true
+}
+
+combine_reports() {
+ rm result/output.xml || true
+ rebot -d result --nostatusrc -o output.xml -N $(basename "$(pwd)") result/*.xml
+}
+
+print_phase() {
+ echo ""
+ echo "**** $1 ****"
+ echo ""
+}
diff --git a/hadoop-ozone/dist/src/main/license/bin/LICENSE.txt b/hadoop-ozone/dist/src/main/license/bin/LICENSE.txt
index 54c8fe2ce206..df63c6e7debb 100644
--- a/hadoop-ozone/dist/src/main/license/bin/LICENSE.txt
+++ b/hadoop-ozone/dist/src/main/license/bin/LICENSE.txt
@@ -424,7 +424,7 @@ bootstrap.min.js
angular-route-1.7.9.min.js
angular-nvd3-1.0.9.min.js
angular-1.7.9.min.js
-jquery-3.4.1.min.js
+jquery-3.5.1.min.js
--------------------------------------------------------------------------------
recon server uses a huge number of javascript and css dependencies. See the
diff --git a/hadoop-ozone/dist/src/main/smoketest/admincli/admin.robot b/hadoop-ozone/dist/src/main/smoketest/admincli/admin.robot
new file mode 100644
index 000000000000..a28888b23f4b
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/admincli/admin.robot
@@ -0,0 +1,32 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+*** Settings ***
+Documentation Test ozone admin command
+Library BuiltIn
+Resource ../commonlib.robot
+Test Timeout 5 minutes
+
+*** Test Cases ***
+Incomplete command
+ ${output} = Execute And Ignore Error ozone admin
+ Should contain ${output} Incomplete command
+ Should contain ${output} container
+ Should contain ${output} datanode
+ Should contain ${output} om
+ Should contain ${output} pipeline
+ Should contain ${output} replicationmanager
+ Should contain ${output} safemode
+ Should contain ${output} printTopology
diff --git a/hadoop-ozone/dist/src/main/smoketest/admincli/container.robot b/hadoop-ozone/dist/src/main/smoketest/admincli/container.robot
new file mode 100644
index 000000000000..0560880de79d
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/admincli/container.robot
@@ -0,0 +1,68 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+*** Settings ***
+Documentation Test ozone admin container command
+Library BuiltIn
+Resource ../commonlib.robot
+Test Timeout 5 minutes
+Suite Setup Create test data
+
+*** Variables ***
+${CONTAINER}
+
+*** Keywords ***
+Create test data
+ Run Keyword if '${SECURITY_ENABLED}' == 'true' Kinit test user testuser testuser.keytab
+ Execute ozone freon ockg -n1 -t1 -p container
+
+*** Test Cases ***
+Create container
+ ${output} = Execute ozone admin container create
+ Should contain ${output} is created
+ ${container} = Execute echo "${output}" | grep 'is created' | cut -f2 -d' '
+ Set Suite Variable ${CONTAINER} ${container}
+
+List containers
+ ${output} = Execute ozone admin container list
+ Should contain ${output} OPEN
+
+List containers with explicit host
+ ${output} = Execute ozone admin container list --scm scm
+ Should contain ${output} OPEN
+
+Container info
+ ${output} = Execute ozone admin container info "${CONTAINER}"
+ Should contain ${output} Container id: ${CONTAINER}
+ Should contain ${output} Datanodes
+
+Close container
+ Execute ozone admin container close "${CONTAINER}"
+ ${output} = Execute ozone admin container info "${CONTAINER}"
+ Should contain ${output} CLOS
+
+Incomplete command
+ ${output} = Execute And Ignore Error ozone admin container
+ Should contain ${output} Incomplete command
+ Should contain ${output} list
+ Should contain ${output} info
+ Should contain ${output} delete
+ Should contain ${output} create
+ Should contain ${output} close
+
+List containers on unknown host
+ ${output} = Execute And Ignore Error ozone admin --verbose container list --scm unknown-host
+ Should contain ${output} Invalid host name
+
diff --git a/hadoop-ozone/dist/src/main/smoketest/admincli/datanode.robot b/hadoop-ozone/dist/src/main/smoketest/admincli/datanode.robot
index cb16bc8bc86a..b34f3af6255a 100644
--- a/hadoop-ozone/dist/src/main/smoketest/admincli/datanode.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/admincli/datanode.robot
@@ -14,17 +14,22 @@
# limitations under the License.
*** Settings ***
-Documentation Smoketest ozone cluster startup
-Library OperatingSystem
+Documentation Test ozone admin datanode command
Library BuiltIn
Resource ../commonlib.robot
Test Timeout 5 minutes
-*** Variables ***
-
-
*** Test Cases ***
-Run list datanodes
+List datanodes
${output} = Execute ozone admin datanode list
Should contain ${output} Datanode:
- Should contain ${output} Related pipelines:
\ No newline at end of file
+ Should contain ${output} Related pipelines:
+
+Incomplete command
+ ${output} = Execute And Ignore Error ozone admin datanode
+ Should contain ${output} Incomplete command
+ Should contain ${output} list
+
+List datanodes on unknown host
+ ${output} = Execute And Ignore Error ozone admin --verbose datanode list --scm unknown-host
+ Should contain ${output} Invalid host name
diff --git a/hadoop-ozone/dist/src/main/smoketest/admincli/pipeline.robot b/hadoop-ozone/dist/src/main/smoketest/admincli/pipeline.robot
index b514ae7b07ad..3a97f8394977 100644
--- a/hadoop-ozone/dist/src/main/smoketest/admincli/pipeline.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/admincli/pipeline.robot
@@ -14,21 +14,52 @@
# limitations under the License.
*** Settings ***
-Documentation Smoketest ozone cluster startup
-Library OperatingSystem
+Documentation Test ozone admin pipeline command
Library BuiltIn
Resource ../commonlib.robot
Test Timeout 5 minutes
*** Variables ***
-
+${PIPELINE}
*** Test Cases ***
-Run list pipeline
+Create pipeline
+ ${output} = Execute ozone admin pipeline create
+ Should contain ${output} is created. Factor: ONE, Type: STAND_ALONE
+ ${pipeline} = Execute echo "${output}" | grep 'is created' | cut -f1 -d' ' | cut -f2 -d'='
+ Set Suite Variable ${PIPELINE} ${pipeline}
+
+List pipelines
${output} = Execute ozone admin pipeline list
- Should contain ${output} Type:
- Should contain ${output} Factor:ONE, State:
+ Should contain ${output} Factor:ONE
-Run create pipeline
- ${output} = Execute ozone admin pipeline create
- Should contain ${output} is created. Factor: ONE, Type: STAND_ALONE
\ No newline at end of file
+List pipelines with explicit host
+ ${output} = Execute ozone admin pipeline list --scm scm
+ Should contain ${output} Factor:ONE
+
+Deactivate pipeline
+ Execute ozone admin pipeline deactivate "${PIPELINE}"
+ ${output} = Execute ozone admin pipeline list | grep "${PIPELINE}"
+ Should contain ${output} DORMANT
+
+Activate pipeline
+ Execute ozone admin pipeline activate "${PIPELINE}"
+ ${output} = Execute ozone admin pipeline list | grep "${PIPELINE}"
+ Should contain ${output} OPEN
+
+Close pipeline
+ Execute ozone admin pipeline close "${PIPELINE}"
+ ${output} = Execute ozone admin pipeline list | grep "${PIPELINE}"
+ Should contain ${output} CLOSED
+
+Incomplete command
+ ${output} = Execute And Ignore Error ozone admin pipeline
+ Should contain ${output} Incomplete command
+ Should contain ${output} close
+ Should contain ${output} create
+ Should contain ${output} deactivate
+ Should contain ${output} list
+
+List pipelines on unknown host
+ ${output} = Execute And Ignore Error ozone admin --verbose pipeline list --scm unknown-host
+ Should contain ${output} Invalid host name
diff --git a/hadoop-ozone/dist/src/main/smoketest/admincli/replicationmanager.robot b/hadoop-ozone/dist/src/main/smoketest/admincli/replicationmanager.robot
new file mode 100644
index 000000000000..cef294f1e8d7
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/admincli/replicationmanager.robot
@@ -0,0 +1,53 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+*** Settings ***
+Documentation Test ozone admin replicationmanager command
+Library BuiltIn
+Resource ../commonlib.robot
+Test Timeout 5 minutes
+
+*** Test Cases ***
+Check replicationmanager
+ ${output} = Execute ozone admin replicationmanager status
+ Should contain ${output} ReplicationManager
+ Should contain ${output} Running
+
+Check replicationmanager with explicit host
+ ${output} = Execute ozone admin replicationmanager status --scm scm
+ Should contain ${output} ReplicationManager
+ Should contain ${output} Running
+
+Start replicationmanager
+ ${output} = Execute ozone admin replicationmanager start
+ Should contain ${output} Starting ReplicationManager
+ Wait Until Keyword Succeeds 30sec 5sec Execute ozone admin replicationmanager status | grep -q 'is Running'
+
+Stop replicationmanager
+ ${output} = Execute ozone admin replicationmanager stop
+ Should contain ${output} Stopping ReplicationManager
+ Wait Until Keyword Succeeds 30sec 5sec Execute ozone admin replicationmanager status | grep -q 'is Not Running'
+
+Incomplete command
+ ${output} = Execute And Ignore Error ozone admin replicationmanager
+ Should contain ${output} Incomplete command
+ Should contain ${output} start
+ Should contain ${output} stop
+ Should contain ${output} status
+
+Check replicationmanager on unknown host
+ ${output} = Execute And Ignore Error ozone admin --verbose replicationmanager status --scm unknown-host
+ Should contain ${output} Invalid host name
+
diff --git a/hadoop-ozone/dist/src/main/smoketest/admincli/safemode.robot b/hadoop-ozone/dist/src/main/smoketest/admincli/safemode.robot
new file mode 100644
index 000000000000..114d846e0e07
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/admincli/safemode.robot
@@ -0,0 +1,45 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+*** Settings ***
+Documentation Test ozone admin safemode command
+Library BuiltIn
+Resource ../commonlib.robot
+Test Timeout 5 minutes
+
+*** Test Cases ***
+Check safemode
+ ${output} = Execute ozone admin safemode status
+ Should contain ${output} SCM is out of safe mode
+
+Check safemode with explicit host
+ ${output} = Execute ozone admin safemode status --scm scm
+ Should contain ${output} SCM is out of safe mode
+
+Wait for safemode exit
+ ${output} = Execute ozone admin safemode wait -t 2
+ Should contain ${output} SCM is out of safe mode
+
+Incomplete command
+ ${output} = Execute And Ignore Error ozone admin safemode
+ Should contain ${output} Incomplete command
+ Should contain ${output} status
+ Should contain ${output} exit
+ Should contain ${output} wait
+
+Check safemode on unknown host
+ ${output} = Execute And Ignore Error ozone admin --verbose safemode status --scm unknown-host
+ Should contain ${output} Invalid host name
+
diff --git a/hadoop-ozone/dist/src/main/smoketest/basic/links.robot b/hadoop-ozone/dist/src/main/smoketest/basic/links.robot
new file mode 100644
index 000000000000..71c046e18a25
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/basic/links.robot
@@ -0,0 +1,152 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+*** Settings ***
+Documentation Test bucket links via Ozone CLI
+Library OperatingSystem
+Resource ../commonlib.robot
+Resource ../ozone-lib/shell.robot
+Test Setup Run Keyword if '${SECURITY_ENABLED}' == 'true' Kinit test user testuser testuser.keytab
+Test Timeout 2 minute
+Suite Setup Create volumes
+
+*** Variables ***
+${prefix} generated
+
+*** Keywords ***
+Create volumes
+ ${random} = Generate Random String 5 [NUMBERS]
+ Set Suite Variable ${source} ${random}-source
+ Set Suite Variable ${target} ${random}-target
+ Execute ozone sh volume create ${source}
+ Execute ozone sh volume create ${target}
+ Run Keyword if '${SECURITY_ENABLED}' == 'true' Setup ACL tests
+
+Setup ACL tests
+ Execute ozone sh bucket create ${source}/readable-bucket
+ Execute ozone sh key put ${source}/readable-bucket/key-in-readable-bucket /etc/passwd
+ Execute ozone sh bucket create ${source}/unreadable-bucket
+ Execute ozone sh bucket link ${source}/readable-bucket ${target}/readable-link
+ Execute ozone sh bucket link ${source}/readable-bucket ${target}/unreadable-link
+ Execute ozone sh bucket link ${source}/unreadable-bucket ${target}/link-to-unreadable-bucket
+ Execute ozone sh volume addacl --acl user:testuser2/scm@EXAMPLE.COM:r ${target}
+ Execute ozone sh volume addacl --acl user:testuser2/scm@EXAMPLE.COM:rl ${source}
+ Execute ozone sh bucket addacl --acl user:testuser2/scm@EXAMPLE.COM:rl ${source}/readable-bucket
+ Execute ozone sh bucket addacl --acl user:testuser2/scm@EXAMPLE.COM:r ${target}/readable-link
+ Execute ozone sh bucket addacl --acl user:testuser2/scm@EXAMPLE.COM:r ${target}/link-to-unreadable-bucket
+
+Can follow link with read access
+ Execute kdestroy
+ Run Keyword Kinit test user testuser2 testuser2.keytab
+ ${result} = Execute And Ignore Error ozone sh key list ${target}/readable-link
+ Should Contain ${result} key-in-readable-bucket
+
+Cannot follow link without read access
+ Execute kdestroy
+ Run Keyword Kinit test user testuser2 testuser2.keytab
+ ${result} = Execute And Ignore Error ozone sh key list ${target}/unreadable-link
+ Should Contain ${result} PERMISSION_DENIED
+
+ACL verified on source bucket
+ Execute kdestroy
+ Run Keyword Kinit test user testuser2 testuser2.keytab
+ ${result} = Execute ozone sh bucket info ${target}/link-to-unreadable-bucket
+ Should Contain ${result} link-to-unreadable-bucket
+ Should Not Contain ${result} PERMISSION_DENIED
+ ${result} = Execute And Ignore Error ozone sh key list ${target}/link-to-unreadable-bucket
+ Should Contain ${result} PERMISSION_DENIED
+
+*** Test Cases ***
+Link to non-existent bucket
+ Execute ozone sh bucket link ${source}/no-such-bucket ${target}/dangling-link
+ ${result} = Execute And Ignore Error ozone sh key list ${target}/dangling-link
+ Should Contain ${result} BUCKET_NOT_FOUND
+
+Key create passthrough
+ Execute ozone sh bucket link ${source}/bucket1 ${target}/link1
+ Execute ozone sh bucket create ${source}/bucket1
+ Execute ozone sh key put ${target}/link1/key1 /etc/passwd
+ Key Should Match Local File ${target}/link1/key1 /etc/passwd
+
+Key read passthrough
+ Execute ozone sh key put ${source}/bucket1/key2 /opt/hadoop/NOTICE.txt
+ Key Should Match Local File ${source}/bucket1/key2 /opt/hadoop/NOTICE.txt
+
+Key list passthrough
+ ${target_list} = Execute ozone sh key list ${target}/link1 | jq -r '.name'
+ ${source_list} = Execute ozone sh key list ${source}/bucket1 | jq -r '.name'
+ Should Be Equal ${target_list} ${source_list}
+ Should Contain ${source_list} key1
+ Should Contain ${source_list} key2
+
+Key delete passthrough
+ Execute ozone sh key delete ${target}/link1/key2
+ ${source_list} = Execute ozone sh key list ${source}/bucket1 | jq -r '.name'
+ Should Not Contain ${source_list} key2
+
+Bucket list contains links
+ ${result} = Execute ozone sh bucket list ${target}
+ Should Contain ${result} link1
+ Should Contain ${result} dangling-link
+
+Bucket info shows source
+ ${result} = Execute ozone sh bucket info ${target}/link1 | jq -r '.sourceVolume, .sourceBucket' | xargs
+ Should Be Equal ${result} ${source} bucket1
+
+Source and target have separate ACLs
+ Execute ozone sh bucket addacl --acl user:user1:rwxy ${target}/link1
+ Verify ACL bucket ${target}/link1 USER user1 READ WRITE READ_ACL WRITE_ACL
+ Verify ACL bucket ${source}/bucket1 USER user1 ${EMPTY}
+
+ Execute ozone sh bucket addacl --acl group:group2:r ${source}/bucket1
+ Verify ACL bucket ${target}/link1 GROUP group2 ${EMPTY}
+ Verify ACL bucket ${source}/bucket1 GROUP group2 READ
+
+Buckets and links share namespace
+ Execute ozone sh bucket link ${source}/bucket2 ${target}/link2
+ ${result} = Execute And Ignore Error ozone sh bucket create ${target}/link2
+ Should Contain ${result} BUCKET_ALREADY_EXISTS
+
+ Execute ozone sh bucket create ${target}/bucket3
+ ${result} = Execute And Ignore Error ozone sh bucket link ${source}/bucket1 ${target}/bucket3
+ Should Contain ${result} BUCKET_ALREADY_EXISTS
+
+Can follow link with read access
+ Run Keyword if '${SECURITY_ENABLED}' == 'true' Can follow link with read access
+
+Cannot follow link without read access
+ Run Keyword if '${SECURITY_ENABLED}' == 'true' Cannot follow link without read access
+
+ACL verified on source bucket
+ Run Keyword if '${SECURITY_ENABLED}' == 'true' ACL verified on source bucket
+
+Loop in link chain is detected
+ Execute ozone sh bucket link ${target}/loop1 ${target}/loop2
+ Execute ozone sh bucket link ${target}/loop2 ${target}/loop3
+ Execute ozone sh bucket link ${target}/loop3 ${target}/loop1
+ ${result} = Execute And Ignore Error ozone sh key list ${target}/loop2
+ Should Contain ${result} DETECTED_LOOP
+
+Multiple links to same bucket are allowed
+ Execute ozone sh bucket link ${source}/bucket1 ${target}/link3
+ Execute ozone sh key put ${target}/link3/key3 /etc/group
+ Key Should Match Local File ${target}/link1/key3 /etc/group
+
+Source bucket not affected by deleting link
+ Execute ozone sh bucket delete ${target}/link1
+ ${bucket_list} = Execute ozone sh bucket list ${target}
+ Should Not Contain ${bucket_list} link1
+ ${source_list} = Execute ozone sh key list ${source}/bucket1 | jq -r '.name'
+ Should Contain ${source_list} key1
diff --git a/hadoop-ozone/dist/src/main/smoketest/basic/ozone-shell-lib.robot b/hadoop-ozone/dist/src/main/smoketest/basic/ozone-shell-lib.robot
new file mode 100644
index 000000000000..ba0006153d29
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/basic/ozone-shell-lib.robot
@@ -0,0 +1,136 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+*** Settings ***
+Documentation Test ozone shell CLI usage
+Library OperatingSystem
+Resource ../commonlib.robot
+
+*** Variables ***
+${prefix} generated
+
+*** Keywords ***
+
+Generate prefix
+ ${random} = Generate Random String 5 [NUMBERS]
+ Set Suite Variable ${prefix} ${random}
+
+Test ozone shell
+ [arguments] ${protocol} ${server} ${volume}
+ ${result} = Execute And Ignore Error ozone sh volume info ${protocol}${server}/${volume}
+ Should contain ${result} VOLUME_NOT_FOUND
+ ${result} = Execute ozone sh volume create ${protocol}${server}/${volume} --quota 100TB
+ Should not contain ${result} Failed
+ ${result} = Execute ozone sh volume list ${protocol}${server}/ | jq -r '. | select(.name=="${volume}")'
+ Should contain ${result} creationTime
+ ${result} = Execute ozone sh volume list | jq -r '. | select(.name=="${volume}")'
+ Should contain ${result} creationTime
+# TODO: Disable updating the owner, acls should be used to give access to other user.
+ Execute ozone sh volume update ${protocol}${server}/${volume} --quota 10TB
+# ${result} = Execute ozone sh volume info ${protocol}${server}/${volume} | jq -r '. | select(.volumeName=="${volume}") | .owner | .name'
+# Should Be Equal ${result} bill
+ ${result} = Execute ozone sh volume info ${protocol}${server}/${volume} | jq -r '. | select(.name=="${volume}") | .quota'
+ Should Be Equal ${result} 10995116277760
+ Execute ozone sh bucket create ${protocol}${server}/${volume}/bb1
+ ${result} = Execute ozone sh bucket info ${protocol}${server}/${volume}/bb1 | jq -r '. | select(.name=="bb1") | .storageType'
+ Should Be Equal ${result} DISK
+ ${result} = Execute ozone sh bucket list ${protocol}${server}/${volume}/ | jq -r '. | select(.name=="bb1") | .volumeName'
+ Should Be Equal ${result} ${volume}
+ Run Keyword Test key handling ${protocol} ${server} ${volume}
+ Execute ozone sh bucket delete ${protocol}${server}/${volume}/bb1
+ Execute ozone sh volume delete ${protocol}${server}/${volume}
+
+Test Volume Acls
+ [arguments] ${protocol} ${server} ${volume}
+ Execute ozone sh volume create ${protocol}${server}/${volume}
+ ${result} = Execute ozone sh volume getacl ${protocol}${server}/${volume}
+ Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \".*\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"ALL\" .
+ ${result} = Execute ozone sh volume addacl ${protocol}${server}/${volume} -a user:superuser1:rwxy[DEFAULT]
+ ${result} = Execute ozone sh volume getacl ${protocol}${server}/${volume}
+ Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \"superuser1*\",\n.*\"aclScope\" : \"DEFAULT\",\n.*\"aclList\" : . \"READ\", \"WRITE\", \"READ_ACL\", \"WRITE_ACL\" .
+ ${result} = Execute ozone sh volume removeacl ${protocol}${server}/${volume} -a user:superuser1:xy
+ ${result} = Execute ozone sh volume getacl ${protocol}${server}/${volume}
+ Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \"superuser1\",\n.*\"aclScope\" : \"DEFAULT\",\n.*\"aclList\" : . \"READ\", \"WRITE\", \"READ_ACL\", \"WRITE_ACL\" .
+ ${result} = Execute ozone sh volume setacl ${protocol}${server}/${volume} -al user:superuser1:rwxy,group:superuser1:a,user:testuser/scm@EXAMPLE.COM:rwxyc,group:superuser1:a[DEFAULT]
+ ${result} = Execute ozone sh volume getacl ${protocol}${server}/${volume}
+ Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \"superuser1*\",\n.*\"aclScope\" : \"DEFAULT\",\n.*\"aclList\" : . \"READ\", \"WRITE\", \"READ_ACL\", \"WRITE_ACL\" .
+ Should Match Regexp ${result} \"type\" : \"GROUP\",\n.*\"name\" : \"superuser1\",\n.*\"aclScope\" : \"DEFAULT\",\n.*\"aclList\" : . \"ALL\" .
+
+Test Bucket Acls
+ [arguments] ${protocol} ${server} ${volume}
+ Execute ozone sh bucket create ${protocol}${server}/${volume}/bb1
+ ${result} = Execute ozone sh bucket getacl ${protocol}${server}/${volume}/bb1
+ Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \".*\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"ALL\" .
+ ${result} = Execute ozone sh bucket addacl ${protocol}${server}/${volume}/bb1 -a user:superuser1:rwxy
+ ${result} = Execute ozone sh bucket getacl ${protocol}${server}/${volume}/bb1
+ Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \"superuser1*\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"READ\", \"WRITE\", \"READ_ACL\", \"WRITE_ACL\"
+ ${result} = Execute ozone sh bucket removeacl ${protocol}${server}/${volume}/bb1 -a user:superuser1:xy
+ ${result} = Execute ozone sh bucket getacl ${protocol}${server}/${volume}/bb1
+ Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \"superuser1\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"READ\", \"WRITE\"
+ ${result} = Execute ozone sh bucket setacl ${protocol}${server}/${volume}/bb1 -al user:superuser1:rwxy,group:superuser1:a,user:testuser/scm@EXAMPLE.COM:rwxyc,group:superuser1:a[DEFAULT]
+ ${result} = Execute ozone sh bucket getacl ${protocol}${server}/${volume}/bb1
+ Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \"superuser1*\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"READ\", \"WRITE\", \"READ_ACL\", \"WRITE_ACL\"
+ Should Match Regexp ${result} \"type\" : \"GROUP\",\n.*\"name\" : \"superuser1\",\n.*\"aclScope\" : \"DEFAULT\",\n.*\"aclList\" : . \"ALL\" .
+
+
+Test key handling
+ [arguments] ${protocol} ${server} ${volume}
+ Execute ozone sh key put ${protocol}${server}/${volume}/bb1/key1 /opt/hadoop/NOTICE.txt
+ Execute rm -f /tmp/NOTICE.txt.1
+ Execute ozone sh key get ${protocol}${server}/${volume}/bb1/key1 /tmp/NOTICE.txt.1
+ Execute diff -q /opt/hadoop/NOTICE.txt /tmp/NOTICE.txt.1
+
+ Execute ozone sh key put -t RATIS ${protocol}${server}/${volume}/bb1/key1_RATIS /opt/hadoop/NOTICE.txt
+ Execute rm -f /tmp/key1_RATIS
+ Execute ozone sh key get ${protocol}${server}/${volume}/bb1/key1_RATIS /tmp/key1_RATIS
+ Execute diff -q /opt/hadoop/NOTICE.txt /tmp/key1_RATIS
+ ${result} = Execute ozone sh key info ${protocol}${server}/${volume}/bb1/key1_RATIS | jq -r '. | select(.name=="key1_RATIS")'
+ Should contain ${result} RATIS
+ Execute ozone sh key delete ${protocol}${server}/${volume}/bb1/key1_RATIS
+
+ Execute ozone sh key cp ${protocol}${server}/${volume}/bb1 key1 key1-copy
+ Execute rm -f /tmp/key1-copy
+ Execute ozone sh key get ${protocol}${server}/${volume}/bb1/key1-copy /tmp/key1-copy
+ Execute diff -q /opt/hadoop/NOTICE.txt /tmp/key1-copy
+ Execute ozone sh key delete ${protocol}${server}/${volume}/bb1/key1-copy
+
+ ${result} = Execute And Ignore Error ozone sh key get ${protocol}${server}/${volume}/bb1/key1 /tmp/NOTICE.txt.1
+ Should Contain ${result} NOTICE.txt.1 exists
+ ${result} = Execute ozone sh key get --force ${protocol}${server}/${volume}/bb1/key1 /tmp/NOTICE.txt.1
+ Should Not Contain ${result} NOTICE.txt.1 exists
+ ${result} = Execute ozone sh key info ${protocol}${server}/${volume}/bb1/key1 | jq -r '. | select(.name=="key1")'
+ Should contain ${result} creationTime
+ ${result} = Execute ozone sh key list ${protocol}${server}/${volume}/bb1 | jq -r '. | select(.name=="key1") | .name'
+ Should Be Equal ${result} key1
+ Execute ozone sh key rename ${protocol}${server}/${volume}/bb1 key1 key2
+ ${result} = Execute ozone sh key list ${protocol}${server}/${volume}/bb1 | jq -r '.name'
+ Should Be Equal ${result} key2
+ Execute ozone sh key delete ${protocol}${server}/${volume}/bb1/key2
+
+Test key Acls
+ [arguments] ${protocol} ${server} ${volume}
+ Execute ozone sh key put ${protocol}${server}/${volume}/bb1/key2 /opt/hadoop/NOTICE.txt
+ ${result} = Execute ozone sh key getacl ${protocol}${server}/${volume}/bb1/key2
+ Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \".*\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"ALL\" .
+ ${result} = Execute ozone sh key addacl ${protocol}${server}/${volume}/bb1/key2 -a user:superuser1:rwxy
+ ${result} = Execute ozone sh key getacl ${protocol}${server}/${volume}/bb1/key2
+ Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \"superuser1\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"READ\", \"WRITE\", \"READ_ACL\", \"WRITE_ACL\"
+ ${result} = Execute ozone sh key removeacl ${protocol}${server}/${volume}/bb1/key2 -a user:superuser1:xy
+ ${result} = Execute ozone sh key getacl ${protocol}${server}/${volume}/bb1/key2
+ Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \"superuser1\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"READ\", \"WRITE\"
+ ${result} = Execute ozone sh key setacl ${protocol}${server}/${volume}/bb1/key2 -al user:superuser1:rwxy,group:superuser1:a,user:testuser/scm@EXAMPLE.COM:rwxyc
+ ${result} = Execute ozone sh key getacl ${protocol}${server}/${volume}/bb1/key2
+ Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \"superuser1\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"READ\", \"WRITE\", \"READ_ACL\", \"WRITE_ACL\"
+ Should Match Regexp ${result} \"type\" : \"GROUP\",\n.*\"name\" : \"superuser1\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"ALL\" .
diff --git a/hadoop-ozone/dist/src/main/smoketest/basic/ozone-shell-single.robot b/hadoop-ozone/dist/src/main/smoketest/basic/ozone-shell-single.robot
new file mode 100644
index 000000000000..e08ee0933f98
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/basic/ozone-shell-single.robot
@@ -0,0 +1,27 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+*** Settings ***
+Documentation Test ozone shell CLI usage
+Library OperatingSystem
+Resource ../commonlib.robot
+Resource ozone-shell-lib.robot
+Test Timeout 2 minute
+Suite Setup Generate prefix
+
+*** Test Cases ***
+
+Test ozone shell
+ Test ozone shell ${EMPTY} ${EMPTY} ${prefix}-rpcbasic
diff --git a/hadoop-ozone/dist/src/main/smoketest/basic/ozone-shell.robot b/hadoop-ozone/dist/src/main/smoketest/basic/ozone-shell.robot
index 9143f38dcf36..45b2d35a6cee 100644
--- a/hadoop-ozone/dist/src/main/smoketest/basic/ozone-shell.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/basic/ozone-shell.robot
@@ -17,18 +17,11 @@
Documentation Test ozone shell CLI usage
Library OperatingSystem
Resource ../commonlib.robot
+Resource ozone-shell-lib.robot
Test Setup Run Keyword if '${SECURITY_ENABLED}' == 'true' Kinit test user testuser testuser.keytab
Test Timeout 2 minute
Suite Setup Generate prefix
-*** Variables ***
-${prefix} generated
-
-*** Keywords ***
-Generate prefix
- ${random} = Generate Random String 5 [NUMBERS]
- Set Suite Variable ${prefix} ${random}
-
*** Test Cases ***
RpcClient with port
Test ozone shell o3:// om:9862 ${prefix}-rpcwoport
@@ -47,113 +40,3 @@ RpcClient without host
RpcClient without scheme
Test ozone shell ${EMPTY} ${EMPTY} ${prefix}-rpcwoscheme
-
-
-*** Keywords ***
-Test ozone shell
- [arguments] ${protocol} ${server} ${volume}
- ${result} = Execute And Ignore Error ozone sh volume info ${protocol}${server}/${volume}
- Should contain ${result} VOLUME_NOT_FOUND
- ${result} = Execute ozone sh volume create ${protocol}${server}/${volume} --quota 100TB
- Should not contain ${result} Failed
- ${result} = Execute ozone sh volume list ${protocol}${server}/ | jq -r '. | select(.name=="${volume}")'
- Should contain ${result} creationTime
- ${result} = Execute ozone sh volume list | jq -r '. | select(.name=="${volume}")'
- Should contain ${result} creationTime
-# TODO: Disable updating the owner, acls should be used to give access to other user.
- Execute ozone sh volume update ${protocol}${server}/${volume} --quota 10TB
-# ${result} = Execute ozone sh volume info ${protocol}${server}/${volume} | jq -r '. | select(.volumeName=="${volume}") | .owner | .name'
-# Should Be Equal ${result} bill
- ${result} = Execute ozone sh volume info ${protocol}${server}/${volume} | jq -r '. | select(.name=="${volume}") | .quota'
- Should Be Equal ${result} 10995116277760
- Execute ozone sh bucket create ${protocol}${server}/${volume}/bb1
- ${result} = Execute ozone sh bucket info ${protocol}${server}/${volume}/bb1 | jq -r '. | select(.name=="bb1") | .storageType'
- Should Be Equal ${result} DISK
- ${result} = Execute ozone sh bucket list ${protocol}${server}/${volume}/ | jq -r '. | select(.name=="bb1") | .volumeName'
- Should Be Equal ${result} ${volume}
- Run Keyword Test key handling ${protocol} ${server} ${volume}
- Execute ozone sh bucket delete ${protocol}${server}/${volume}/bb1
- Execute ozone sh volume delete ${protocol}${server}/${volume}
-
-Test Volume Acls
- [arguments] ${protocol} ${server} ${volume}
- Execute ozone sh volume create ${protocol}${server}/${volume}
- ${result} = Execute ozone sh volume getacl ${protocol}${server}/${volume}
- Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \".*\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"ALL\" .
- ${result} = Execute ozone sh volume addacl ${protocol}${server}/${volume} -a user:superuser1:rwxy[DEFAULT]
- ${result} = Execute ozone sh volume getacl ${protocol}${server}/${volume}
- Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \"superuser1*\",\n.*\"aclScope\" : \"DEFAULT\",\n.*\"aclList\" : . \"READ\", \"WRITE\", \"READ_ACL\", \"WRITE_ACL\" .
- ${result} = Execute ozone sh volume removeacl ${protocol}${server}/${volume} -a user:superuser1:xy
- ${result} = Execute ozone sh volume getacl ${protocol}${server}/${volume}
- Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \"superuser1\",\n.*\"aclScope\" : \"DEFAULT\",\n.*\"aclList\" : . \"READ\", \"WRITE\", \"READ_ACL\", \"WRITE_ACL\" .
- ${result} = Execute ozone sh volume setacl ${protocol}${server}/${volume} -al user:superuser1:rwxy,group:superuser1:a,user:testuser/scm@EXAMPLE.COM:rwxyc,group:superuser1:a[DEFAULT]
- ${result} = Execute ozone sh volume getacl ${protocol}${server}/${volume}
- Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \"superuser1*\",\n.*\"aclScope\" : \"DEFAULT\",\n.*\"aclList\" : . \"READ\", \"WRITE\", \"READ_ACL\", \"WRITE_ACL\" .
- Should Match Regexp ${result} \"type\" : \"GROUP\",\n.*\"name\" : \"superuser1\",\n.*\"aclScope\" : \"DEFAULT\",\n.*\"aclList\" : . \"ALL\" .
-
-Test Bucket Acls
- [arguments] ${protocol} ${server} ${volume}
- Execute ozone sh bucket create ${protocol}${server}/${volume}/bb1
- ${result} = Execute ozone sh bucket getacl ${protocol}${server}/${volume}/bb1
- Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \".*\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"ALL\" .
- ${result} = Execute ozone sh bucket addacl ${protocol}${server}/${volume}/bb1 -a user:superuser1:rwxy
- ${result} = Execute ozone sh bucket getacl ${protocol}${server}/${volume}/bb1
- Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \"superuser1*\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"READ\", \"WRITE\", \"READ_ACL\", \"WRITE_ACL\"
- ${result} = Execute ozone sh bucket removeacl ${protocol}${server}/${volume}/bb1 -a user:superuser1:xy
- ${result} = Execute ozone sh bucket getacl ${protocol}${server}/${volume}/bb1
- Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \"superuser1\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"READ\", \"WRITE\"
- ${result} = Execute ozone sh bucket setacl ${protocol}${server}/${volume}/bb1 -al user:superuser1:rwxy,group:superuser1:a,user:testuser/scm@EXAMPLE.COM:rwxyc,group:superuser1:a[DEFAULT]
- ${result} = Execute ozone sh bucket getacl ${protocol}${server}/${volume}/bb1
- Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \"superuser1*\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"READ\", \"WRITE\", \"READ_ACL\", \"WRITE_ACL\"
- Should Match Regexp ${result} \"type\" : \"GROUP\",\n.*\"name\" : \"superuser1\",\n.*\"aclScope\" : \"DEFAULT\",\n.*\"aclList\" : . \"ALL\" .
-
-
-Test key handling
- [arguments] ${protocol} ${server} ${volume}
- Execute ozone sh key put ${protocol}${server}/${volume}/bb1/key1 /opt/hadoop/NOTICE.txt
- Execute rm -f /tmp/NOTICE.txt.1
- Execute ozone sh key get ${protocol}${server}/${volume}/bb1/key1 /tmp/NOTICE.txt.1
- Execute diff -q /opt/hadoop/NOTICE.txt /tmp/NOTICE.txt.1
-
- Execute ozone sh key put -t RATIS ${protocol}${server}/${volume}/bb1/key1_RATIS /opt/hadoop/NOTICE.txt
- Execute rm -f /tmp/key1_RATIS
- Execute ozone sh key get ${protocol}${server}/${volume}/bb1/key1_RATIS /tmp/key1_RATIS
- Execute diff -q /opt/hadoop/NOTICE.txt /tmp/key1_RATIS
- ${result} = Execute ozone sh key info ${protocol}${server}/${volume}/bb1/key1_RATIS | jq -r '. | select(.name=="key1_RATIS")'
- Should contain ${result} RATIS
- Execute ozone sh key delete ${protocol}${server}/${volume}/bb1/key1_RATIS
-
- Execute ozone sh key cp ${protocol}${server}/${volume}/bb1 key1 key1-copy
- Execute rm -f /tmp/key1-copy
- Execute ozone sh key get ${protocol}${server}/${volume}/bb1/key1-copy /tmp/key1-copy
- Execute diff -q /opt/hadoop/NOTICE.txt /tmp/key1-copy
- Execute ozone sh key delete ${protocol}${server}/${volume}/bb1/key1-copy
-
- ${result} = Execute And Ignore Error ozone sh key get ${protocol}${server}/${volume}/bb1/key1 /tmp/NOTICE.txt.1
- Should Contain ${result} NOTICE.txt.1 exists
- ${result} = Execute ozone sh key get --force ${protocol}${server}/${volume}/bb1/key1 /tmp/NOTICE.txt.1
- Should Not Contain ${result} NOTICE.txt.1 exists
- ${result} = Execute ozone sh key info ${protocol}${server}/${volume}/bb1/key1 | jq -r '. | select(.name=="key1")'
- Should contain ${result} creationTime
- ${result} = Execute ozone sh key list ${protocol}${server}/${volume}/bb1 | jq -r '. | select(.name=="key1") | .name'
- Should Be Equal ${result} key1
- Execute ozone sh key rename ${protocol}${server}/${volume}/bb1 key1 key2
- ${result} = Execute ozone sh key list ${protocol}${server}/${volume}/bb1 | jq -r '.name'
- Should Be Equal ${result} key2
- Execute ozone sh key delete ${protocol}${server}/${volume}/bb1/key2
-
-Test key Acls
- [arguments] ${protocol} ${server} ${volume}
- Execute ozone sh key put ${protocol}${server}/${volume}/bb1/key2 /opt/hadoop/NOTICE.txt
- ${result} = Execute ozone sh key getacl ${protocol}${server}/${volume}/bb1/key2
- Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \".*\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"ALL\" .
- ${result} = Execute ozone sh key addacl ${protocol}${server}/${volume}/bb1/key2 -a user:superuser1:rwxy
- ${result} = Execute ozone sh key getacl ${protocol}${server}/${volume}/bb1/key2
- Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \"superuser1\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"READ\", \"WRITE\", \"READ_ACL\", \"WRITE_ACL\"
- ${result} = Execute ozone sh key removeacl ${protocol}${server}/${volume}/bb1/key2 -a user:superuser1:xy
- ${result} = Execute ozone sh key getacl ${protocol}${server}/${volume}/bb1/key2
- Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \"superuser1\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"READ\", \"WRITE\"
- ${result} = Execute ozone sh key setacl ${protocol}${server}/${volume}/bb1/key2 -al user:superuser1:rwxy,group:superuser1:a,user:testuser/scm@EXAMPLE.COM:rwxyc
- ${result} = Execute ozone sh key getacl ${protocol}${server}/${volume}/bb1/key2
- Should Match Regexp ${result} \"type\" : \"USER\",\n.*\"name\" : \"superuser1\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"READ\", \"WRITE\", \"READ_ACL\", \"WRITE_ACL\"
- Should Match Regexp ${result} \"type\" : \"GROUP\",\n.*\"name\" : \"superuser1\",\n.*\"aclScope\" : \"ACCESS\",\n.*\"aclList\" : . \"ALL\" .
diff --git a/hadoop-ozone/dist/src/main/smoketest/commonlib.robot b/hadoop-ozone/dist/src/main/smoketest/commonlib.robot
index 407111a8030c..bf3b3e92d708 100644
--- a/hadoop-ozone/dist/src/main/smoketest/commonlib.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/commonlib.robot
@@ -18,44 +18,14 @@ Library OperatingSystem
Library String
Library BuiltIn
+Resource lib/os.robot
+
*** Variables ***
${SECURITY_ENABLED} false
${OM_HA_PARAM} ${EMPTY}
${OM_SERVICE_ID} om
*** Keywords ***
-Execute
- [arguments] ${command}
- ${rc} ${output} = Run And Return Rc And Output ${command}
- Log ${output}
- Should Be Equal As Integers ${rc} 0
- [return] ${output}
-
-Execute And Ignore Error
- [arguments] ${command}
- ${rc} ${output} = Run And Return Rc And Output ${command}
- Log ${output}
- [return] ${output}
-
-Execute and checkrc
- [arguments] ${command} ${expected_error_code}
- ${rc} ${output} = Run And Return Rc And Output ${command}
- Log ${output}
- Should Be Equal As Integers ${rc} ${expected_error_code}
- [return] ${output}
-
-Compare files
- [arguments] ${file1} ${file2}
- ${checksumbefore} = Execute md5sum ${file1} | awk '{print $1}'
- ${checksumafter} = Execute md5sum ${file2} | awk '{print $1}'
- Should Be Equal ${checksumbefore} ${checksumafter}
-
-Install aws cli
- ${rc} ${output} = Run And Return Rc And Output which apt-get
- Run Keyword if '${rc}' == '0' Install aws cli s3 debian
- ${rc} ${output} = Run And Return Rc And Output yum --help
- Run Keyword if '${rc}' == '0' Install aws cli s3 centos
-
Kinit HTTP user
${hostname} = Execute hostname
Wait Until Keyword Succeeds 2min 10sec Execute kinit -k HTTP/${hostname}@EXAMPLE.COM -t /etc/security/keytabs/HTTP.keytab
diff --git a/hadoop-ozone/dist/src/main/smoketest/debug/ozone-debug.robot b/hadoop-ozone/dist/src/main/smoketest/debug/ozone-debug.robot
index 39e561af6cf0..f7e3274fcedd 100644
--- a/hadoop-ozone/dist/src/main/smoketest/debug/ozone-debug.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/debug/ozone-debug.robot
@@ -29,8 +29,8 @@ Write key
*** Test Cases ***
Test ozone debug
- ${result} = Execute ozone debug chunkinfo o3://om/vol1/bucket1/debugKey | jq -r '.[]'
+ ${result} = Execute ozone debug chunkinfo o3://om/vol1/bucket1/debugKey | jq -r '.KeyLocations[0][0].Locations'
Should contain ${result} files
- ${result} = Execute ozone debug chunkinfo o3://om/vol1/bucket1/debugKey | jq -r '.[].files[0]'
+ ${result} = Execute ozone debug chunkinfo o3://om/vol1/bucket1/debugKey | jq -r '.KeyLocations[0][0].Locations.files[0]'
File Should Exist ${result}
diff --git a/hadoop-ozone/dist/src/main/smoketest/failing/test1.robot b/hadoop-ozone/dist/src/main/smoketest/failing/test1.robot
new file mode 100644
index 000000000000..c1602496c8eb
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/failing/test1.robot
@@ -0,0 +1,21 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+*** Settings ***
+Documentation This test always fails
+
+*** Test Cases ***
+Failure 1
+ Fail
diff --git a/hadoop-ozone/dist/src/main/smoketest/failing/test2.robot b/hadoop-ozone/dist/src/main/smoketest/failing/test2.robot
new file mode 100644
index 000000000000..d161ffe4bf6d
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/failing/test2.robot
@@ -0,0 +1,21 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+*** Settings ***
+Documentation This test always fails
+
+*** Test Cases ***
+Failure 2
+ Fail
diff --git a/hadoop-ozone/dist/src/main/smoketest/lib/os.robot b/hadoop-ozone/dist/src/main/smoketest/lib/os.robot
new file mode 100644
index 000000000000..af927f9af7c0
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/lib/os.robot
@@ -0,0 +1,49 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+*** Settings ***
+Library OperatingSystem
+
+*** Keywords ***
+Execute
+ [arguments] ${command}
+ Run Keyword And Return Execute and checkrc ${command} 0
+
+Execute And Ignore Error
+ [arguments] ${command}
+ ${rc} ${output} = Run And Return Rc And Output ${command}
+ Log ${output}
+ [return] ${output}
+
+Execute and checkrc
+ [arguments] ${command} ${expected_error_code}
+ ${rc} ${output} = Run And Return Rc And Output ${command}
+ Log ${output}
+ Should Be Equal As Integers ${rc} ${expected_error_code}
+ [return] ${output}
+
+Compare files
+ [arguments] ${file1} ${file2}
+ ${checksumbefore} = Execute md5sum ${file1} | awk '{print $1}'
+ ${checksumafter} = Execute md5sum ${file2} | awk '{print $1}'
+ Should Be Equal ${checksumbefore} ${checksumafter}
+
+Create Random File
+ ${postfix} = Generate Random String 5 [NUMBERS]
+ ${tmpfile} = Set Variable /tmp/tempfile-${postfix}
+ File Should Not Exist ${tmpfile}
+ ${content} = Set Variable "Random string"
+ Create File ${tmpfile} ${content}
+ [Return] ${tmpfile}
diff --git a/hadoop-ozone/dist/src/main/smoketest/lib/os_tests.robot b/hadoop-ozone/dist/src/main/smoketest/lib/os_tests.robot
new file mode 100644
index 000000000000..dd4beaf3c161
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/lib/os_tests.robot
@@ -0,0 +1,38 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+*** Settings ***
+Resource os.robot
+
+
+*** Test Cases ***
+
+Execute
+ ${output} = Execute echo 42
+ Should Be Equal ${output} 42
+
+Execute failing command
+ Run Keyword And Expect Error * Execute false
+
+Execute And Ignore Error
+ ${output} = Execute And Ignore Error echo 123 && false
+ Should Be Equal ${output} 123
+
+Execute and checkrc
+ ${output} = Execute and checkrc echo failure && exit 1 1
+ Should Be Equal ${output} failure
+
+Execute and checkrc RC mismatch
+ Run Keyword And Expect Error * Execute and checkrc echo failure && exit 3 1
diff --git a/hadoop-ozone/dist/src/main/smoketest/ozone-lib/shell.robot b/hadoop-ozone/dist/src/main/smoketest/ozone-lib/shell.robot
new file mode 100644
index 000000000000..9afc6df2eabb
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/ozone-lib/shell.robot
@@ -0,0 +1,53 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+*** Settings ***
+Resource ../lib/os.robot
+Library String
+
+
+*** Keywords ***
+Bucket Exists
+ [arguments] ${bucket}
+ ${rc} ${output} = Run And Return Rc And Output timeout 15 ozone sh bucket info ${bucket}
+ Return From Keyword If ${rc} != 0 ${FALSE}
+ Return From Keyword If 'VOLUME_NOT_FOUND' in '''${output}''' ${FALSE}
+ Return From Keyword If 'BUCKET_NOT_FOUND' in '''${output}''' ${FALSE}
+ [Return] ${TRUE}
+
+Compare Key With Local File
+ [arguments] ${key} ${file}
+ ${postfix} = Generate Random String 5 [NUMBERS]
+ ${tmpfile} = Set Variable /tmp/tempkey-${postfix}
+ Execute ozone sh key get -f ${key} ${tmpfile}
+ ${rc} = Run And Return Rc diff -q ${file} ${tmpfile}
+ Execute rm -f ${tmpfile}
+ ${result} = Set Variable If ${rc} == 0 ${TRUE} ${FALSE}
+ [Return] ${result}
+
+Key Should Match Local File
+ [arguments] ${key} ${file}
+ ${matches} = Compare Key With Local File ${key} ${file}
+ Should Be True ${matches}
+
+Verify ACL
+ [arguments] ${object_type} ${object} ${type} ${name} ${acls}
+ ${actual_acls} = Execute ozone sh ${object_type} getacl ${object} | jq -r '.[] | select(.type == "${type}") | select(.name == "${name}") | .aclList[]' | xargs
+ Should Be Equal ${acls} ${actual_acls}
+
+Create Random Volume
+ ${random} = Generate Random String 5 [LOWER]
+ Execute ozone sh volume create o3://${OM_SERVICE_ID}/vol-${random}
+ [return] vol-${random}
diff --git a/hadoop-ozone/dist/src/main/smoketest/ozone-lib/shell_tests.robot b/hadoop-ozone/dist/src/main/smoketest/ozone-lib/shell_tests.robot
new file mode 100644
index 000000000000..56fbcf8b61f0
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/ozone-lib/shell_tests.robot
@@ -0,0 +1,58 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+*** Settings ***
+Resource ../lib/os.robot
+Resource shell.robot
+
+
+*** Variables ***
+${OM_SERVICE_ID} om
+
+
+*** Test Cases ***
+
+Bucket Exists should not if No Such Volume
+ ${exists} = Bucket Exists o3://${OM_SERVICE_ID}/no-such-volume/any-bucket
+ Should Be Equal ${exists} ${FALSE}
+
+Bucket Exists should not if No Such Bucket
+ Execute And Ignore Error ozone sh volume create o3://${OM_SERVICE_ID}/vol1
+ ${exists} = Bucket Exists o3://${OM_SERVICE_ID}/vol1/no-such-bucket
+ Should Be Equal ${exists} ${FALSE}
+
+Bucket Exists
+ Execute And Ignore Error ozone sh bucket create o3://${OM_SERVICE_ID}/vol1/bucket
+ ${exists} = Bucket Exists o3://${OM_SERVICE_ID}/vol1/bucket
+ Should Be Equal ${exists} ${TRUE}
+
+Bucket Exists should not if No Such OM service
+ ${exists} = Bucket Exists o3://no-such-host/any-volume/any-bucket
+ Should Be Equal ${exists} ${FALSE}
+
+
+Key Should Match Local File
+ [Setup] Execute ozone sh key put o3://${OM_SERVICE_ID}/vol1/bucket/passwd /etc/passwd
+ Key Should Match Local File o3://${OM_SERVICE_ID}/vol1/bucket/passwd /etc/passwd
+
+Compare Key With Local File with Different File
+ ${random_file} = Create Random File
+ ${matches} = Compare Key With Local File o3://${OM_SERVICE_ID}/vol1/bucket/passwd ${random_file}
+ Should Be Equal ${matches} ${FALSE}
+ [Teardown] Remove File ${random_file}
+
+Compare Key With Local File if File Does Not Exist
+ ${matches} = Compare Key With Local File o3://${OM_SERVICE_ID}/vol1/bucket/passwd /no-such-file
+ Should Be Equal ${matches} ${FALSE}
diff --git a/hadoop-ozone/dist/src/main/smoketest/ozonefs/ozonefs.robot b/hadoop-ozone/dist/src/main/smoketest/ozonefs/ozonefs.robot
index 6d0042b30496..450f1b6d9efc 100644
--- a/hadoop-ozone/dist/src/main/smoketest/ozonefs/ozonefs.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/ozonefs/ozonefs.robot
@@ -19,7 +19,7 @@ Library OperatingSystem
Resource ../commonlib.robot
Resource setup.robot
Test Timeout 5 minutes
-Suite Setup Setup ${BUCKET_TYPE}s for FS test
+Suite Setup Setup for FS test
*** Test Cases ***
List root
diff --git a/hadoop-ozone/dist/src/main/smoketest/ozonefs/setup.robot b/hadoop-ozone/dist/src/main/smoketest/ozonefs/setup.robot
index 16e059ede721..441822d7fb3b 100644
--- a/hadoop-ozone/dist/src/main/smoketest/ozonefs/setup.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/ozonefs/setup.robot
@@ -29,12 +29,12 @@ ${BUCKET_IN_VOL2} ${BUCKET_TYPE}3-${SCHEME}
${DEEP_DIR} test/${SCHEME}/dir
*** Keywords ***
-Setup buckets for FS test
+Setup for FS test
Create volumes for FS test
- Create buckets for FS test
+ Run Keyword Create ${BUCKET_TYPE}s for FS test
Sanity check for FS test
Assign suite vars for FS test
- Log Completed setup for ${SCHEME} tests in ${VOLUME}/${BUCKET} using FS base URL: ${BASE_URL}
+ Log Completed setup for ${SCHEME} tests with ${BUCKET_TYPE}s in ${VOLUME}/${BUCKET} using FS base URL: ${BASE_URL}
Create volumes for FS test
Execute And Ignore Error ozone sh volume create ${VOLUME} --quota 100TB
@@ -45,6 +45,16 @@ Create buckets for FS test
Execute ozone sh bucket create ${VOLUME}/${BUCKET2}
Execute ozone sh bucket create ${VOL2}/${BUCKET_IN_VOL2}
+Create links for FS test
+ Execute And Ignore Error ozone sh volume create ${VOLUME}-src --quota 100TB
+ Execute And Ignore Error ozone sh volume create ${VOL2}-src --quota 100TB
+ Execute ozone sh bucket create ${VOLUME}-src/${BUCKET}-src
+ Execute ozone sh bucket create ${VOLUME}-src/${BUCKET2}-src
+ Execute ozone sh bucket create ${VOL2}-src/${BUCKET_IN_VOL2}-src
+ Execute ozone sh bucket link ${VOLUME}-src/${BUCKET}-src ${VOLUME}/${BUCKET}
+ Execute ozone sh bucket link ${VOLUME}-src/${BUCKET2}-src ${VOLUME}/${BUCKET2}
+ Execute ozone sh bucket link ${VOL2}-src/${BUCKET_IN_VOL2}-src ${VOL2}/${BUCKET_IN_VOL2}
+
Sanity check for FS test
${result} = Execute ozone sh volume list
Should contain ${result} ${VOLUME}
diff --git a/hadoop-ozone/dist/src/main/smoketest/recon/recon-api.robot b/hadoop-ozone/dist/src/main/smoketest/recon/recon-api.robot
index 707384902b75..ada2dd8daf6a 100644
--- a/hadoop-ozone/dist/src/main/smoketest/recon/recon-api.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/recon/recon-api.robot
@@ -28,10 +28,10 @@ ${API_ENDPOINT_URL} http://recon:9888/api/v1
*** Keywords ***
Check if Recon picks up container from OM
Run Keyword if '${SECURITY_ENABLED}' == 'true' Kinit HTTP user
- ${result} = Execute curl --negotiate -u : -v ${API_ENDPOINT_URL}/containers
+ ${result} = Execute curl --negotiate -u : -LSs ${API_ENDPOINT_URL}/containers
Should contain ${result} \"ContainerID\":1
- ${result} = Execute curl --negotiate -u : -v ${API_ENDPOINT_URL}/utilization/fileCount
+ ${result} = Execute curl --negotiate -u : -LSs ${API_ENDPOINT_URL}/utilization/fileCount
Should contain ${result} \"fileSize\":2048,\"count\":10
*** Test Cases ***
@@ -43,13 +43,13 @@ Check if Recon picks up OM data
Wait Until Keyword Succeeds 90sec 10sec Check if Recon picks up container from OM
Check if Recon picks up DN heartbeats
- ${result} = Execute curl --negotiate -u : -v ${API_ENDPOINT_URL}/datanodes
+ ${result} = Execute curl --negotiate -u : -LSs ${API_ENDPOINT_URL}/datanodes
Should contain ${result} datanodes
Should contain ${result} datanode_1
Should contain ${result} datanode_2
Should contain ${result} datanode_3
- ${result} = Execute curl --negotiate -u : -v ${API_ENDPOINT_URL}/pipelines
+ ${result} = Execute curl --negotiate -u : -LSs ${API_ENDPOINT_URL}/pipelines
Should contain ${result} pipelines
Should contain ${result} RATIS
Should contain ${result} OPEN
@@ -57,15 +57,15 @@ Check if Recon picks up DN heartbeats
Should contain ${result} datanode_2
Should contain ${result} datanode_3
- ${result} = Execute curl --negotiate -u : -v ${API_ENDPOINT_URL}/clusterState
+ ${result} = Execute curl --negotiate -u : -LSs ${API_ENDPOINT_URL}/clusterState
Should contain ${result} \"totalDatanodes\":3
Should contain ${result} \"healthyDatanodes\":3
Should contain ${result} \"pipelines\":4
- ${result} = Execute curl --negotiate -u : -v ${API_ENDPOINT_URL}/containers/1/replicaHistory
+ ${result} = Execute curl --negotiate -u : -LSs ${API_ENDPOINT_URL}/containers/1/replicaHistory
Should contain ${result} \"containerId\":1
Check if Recon Web UI is up
Run Keyword if '${SECURITY_ENABLED}' == 'true' Kinit HTTP user
- ${result} = Execute curl --negotiate -u : -v ${ENDPOINT_URL}
- Should contain ${result} Ozone Recon
\ No newline at end of file
+ ${result} = Execute curl --negotiate -u : -LSs ${ENDPOINT_URL}
+ Should contain ${result} Ozone Recon
diff --git a/hadoop-ozone/dist/src/main/smoketest/s3/MultipartUpload.robot b/hadoop-ozone/dist/src/main/smoketest/s3/MultipartUpload.robot
index 004a49645918..1c6827a16560 100644
--- a/hadoop-ozone/dist/src/main/smoketest/s3/MultipartUpload.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/s3/MultipartUpload.robot
@@ -88,7 +88,7 @@ Test Multipart Upload Complete
#read file and check the key
${result} = Execute AWSS3ApiCli get-object --bucket ${BUCKET} --key multipartKey1 /tmp/multipartKey1.result
- Execute cat /tmp/part1 /tmp/part2 >> /tmp/multipartKey1
+ Execute cat /tmp/part1 /tmp/part2 > /tmp/multipartKey1
Compare files /tmp/multipartKey1 /tmp/multipartKey1.result
Test Multipart Upload Complete Entity too small
@@ -156,7 +156,7 @@ Test Multipart Upload Complete Invalid part errors and complete mpu with few par
Should contain ${result} ETag
${result} = Execute AWSS3ApiCli get-object --bucket ${BUCKET} --key multipartKey3 /tmp/multipartKey3.result
- Execute cat /tmp/part1 /tmp/part3 >> /tmp/multipartKey3
+ Execute cat /tmp/part1 /tmp/part3 > /tmp/multipartKey3
Compare files /tmp/multipartKey3 /tmp/multipartKey3.result
Test abort Multipart upload
@@ -237,7 +237,6 @@ Test Multipart Upload Put With Copy
Should contain ${result} UploadId
${result} = Execute AWSS3APICli upload-part-copy --bucket ${BUCKET} --key copytest/destination --upload-id ${uploadID} --part-number 1 --copy-source ${BUCKET}/copytest/source
- Should contain ${result} ${BUCKET}
Should contain ${result} ETag
Should contain ${result} LastModified
${eTag1} = Execute and checkrc echo '${result}' | jq -r '.CopyPartResult.ETag' 0
@@ -260,13 +259,11 @@ Test Multipart Upload Put With Copy and range
Should contain ${result} UploadId
${result} = Execute AWSS3APICli upload-part-copy --bucket ${BUCKET} --key copyrange/destination --upload-id ${uploadID} --part-number 1 --copy-source ${BUCKET}/copyrange/source --copy-source-range bytes=0-10485758
- Should contain ${result} ${BUCKET}
Should contain ${result} ETag
Should contain ${result} LastModified
${eTag1} = Execute and checkrc echo '${result}' | jq -r '.CopyPartResult.ETag' 0
${result} = Execute AWSS3APICli upload-part-copy --bucket ${BUCKET} --key copyrange/destination --upload-id ${uploadID} --part-number 2 --copy-source ${BUCKET}/copyrange/source --copy-source-range bytes=10485758-10485760
- Should contain ${result} ${BUCKET}
Should contain ${result} ETag
Should contain ${result} LastModified
${eTag2} = Execute and checkrc echo '${result}' | jq -r '.CopyPartResult.ETag' 0
diff --git a/hadoop-ozone/dist/src/main/smoketest/s3/bucketdelete.robot b/hadoop-ozone/dist/src/main/smoketest/s3/bucketdelete.robot
index bcba30db94e3..ce7b8254b0d0 100644
--- a/hadoop-ozone/dist/src/main/smoketest/s3/bucketdelete.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/s3/bucketdelete.robot
@@ -23,14 +23,20 @@ Test Timeout 5 minutes
Suite Setup Setup s3 tests
*** Variables ***
-${ENDPOINT_URL} http://s3g:9878
${BUCKET} generated
+${ENDPOINT_URL} http://s3g:9878
+
+*** Keywords ***
+Create bucket to be deleted
+ ${bucket} = Run Keyword if '${BUCKET}' == 'link' Create link to-be-deleted
+ ... ELSE Run Keyword Create bucket
+ [return] ${bucket}
*** Test Cases ***
Delete existing bucket
-# Bucket already is created in Test Setup.
- Execute AWSS3APICli delete-bucket --bucket ${BUCKET}
+ ${bucket} = Create bucket to be deleted
+ Execute AWSS3APICli delete-bucket --bucket ${bucket}
Delete non-existent bucket
${result} = Execute AWSS3APICli and checkrc delete-bucket --bucket nosuchbucket 255
diff --git a/hadoop-ozone/dist/src/main/smoketest/s3/buckethead.robot b/hadoop-ozone/dist/src/main/smoketest/s3/buckethead.robot
index 76668716cdba..f3ecd011ceb3 100644
--- a/hadoop-ozone/dist/src/main/smoketest/s3/buckethead.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/s3/buckethead.robot
@@ -31,5 +31,6 @@ ${BUCKET} generated
Head Bucket not existent
${result} = Execute AWSS3APICli head-bucket --bucket ${BUCKET}
${result} = Execute AWSS3APICli and checkrc head-bucket --bucket ozonenosuchbucketqqweqwe 255
- Should contain ${result} Bad Request
- Should contain ${result} 400
+ Should contain ${result} 404
+ Should contain ${result} Not Found
+
diff --git a/hadoop-ozone/dist/src/main/smoketest/s3/commonawslib.robot b/hadoop-ozone/dist/src/main/smoketest/s3/commonawslib.robot
index 4595587c91af..74dba38657d6 100644
--- a/hadoop-ozone/dist/src/main/smoketest/s3/commonawslib.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/s3/commonawslib.robot
@@ -15,12 +15,13 @@
*** Settings ***
Resource ../commonlib.robot
-Resource ../commonlib.robot
+Resource ../ozone-lib/shell.robot
*** Variables ***
+${ENDPOINT_URL} http://s3g:9878
${OZONE_S3_HEADER_VERSION} v4
${OZONE_S3_SET_CREDENTIALS} true
-${BUCKET} bucket-999
+${BUCKET} generated
*** Keywords ***
Execute AWSS3APICli
@@ -38,6 +39,12 @@ Execute AWSS3Cli
${output} = Execute aws s3 --endpoint-url ${ENDPOINT_URL} ${command}
[return] ${output}
+Install aws cli
+ ${rc} ${output} = Run And Return Rc And Output which apt-get
+ Run Keyword if '${rc}' == '0' Install aws cli s3 debian
+ ${rc} ${output} = Run And Return Rc And Output yum --help
+ Run Keyword if '${rc}' == '0' Install aws cli s3 centos
+
Install aws cli s3 centos
Execute sudo -E yum install -y awscli
@@ -73,8 +80,9 @@ Setup dummy credentials for S3
Create bucket
${postfix} = Generate Random String 5 [NUMBERS]
- Set Suite Variable ${BUCKET} bucket-${postfix}
- Create bucket with name ${BUCKET}
+ ${bucket} = Set Variable bucket-${postfix}
+ Create bucket with name ${bucket}
+ [Return] ${bucket}
Create bucket with name
[Arguments] ${bucket}
@@ -85,6 +93,19 @@ Create bucket with name
Setup s3 tests
Run Keyword Install aws cli
Run Keyword if '${OZONE_S3_SET_CREDENTIALS}' == 'true' Setup v4 headers
- ${result} = Execute And Ignore Error ozone sh volume create o3://${OM_SERVICE_ID}/s3v
- Should not contain ${result} Failed
- Run Keyword if '${BUCKET}' == 'generated' Create bucket
+ ${BUCKET} = Run Keyword if '${BUCKET}' == 'generated' Create bucket
+ ... ELSE Set Variable ${BUCKET}
+ Set Suite Variable ${BUCKET}
+ Run Keyword if '${BUCKET}' == 'link' Setup links for S3 tests
+
+Setup links for S3 tests
+ ${exists} = Bucket Exists o3://${OM_SERVICE_ID}/s3v/link
+ Return From Keyword If ${exists}
+ Execute ozone sh volume create o3://${OM_SERVICE_ID}/legacy
+ Execute ozone sh bucket create o3://${OM_SERVICE_ID}/legacy/source-bucket
+ Create link link
+
+Create link
+ [arguments] ${bucket}
+ Execute ozone sh bucket link o3://${OM_SERVICE_ID}/legacy/source-bucket o3://${OM_SERVICE_ID}/s3v/${bucket}
+ [return] ${bucket}
diff --git a/hadoop-ozone/dist/src/main/smoketest/security/bucket-encryption.robot b/hadoop-ozone/dist/src/main/smoketest/security/bucket-encryption.robot
new file mode 100644
index 000000000000..a78f94e5fa97
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/security/bucket-encryption.robot
@@ -0,0 +1,45 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+*** Settings ***
+Documentation Test for bucket encryption
+Library BuiltIn
+Library String
+Resource ../commonlib.robot
+Resource ../lib/os.robot
+Resource ../ozone-lib/shell.robot
+Suite Setup Setup Test
+Test Timeout 5 minutes
+
+*** Variables ***
+${KEY_NAME} key1
+${VOLUME}
+
+*** Keywords ***
+Setup Test
+ ${volume} = Create Random Volume
+ Set Suite Variable ${VOLUME} ${volume}
+
+
+*** Test Cases ***
+Create Encrypted Bucket
+ ${output} = Execute ozone sh bucket create -k ${KEY_NAME} o3://${OM_SERVICE_ID}/${VOLUME}/encrypted-bucket
+ Should Not Contain ${output} INVALID_REQUEST
+ Bucket Exists o3://${OM_SERVICE_ID}/${VOLUME}/encrypted-bucket
+
+Create Key in Encrypted Bucket
+ ${key} = Set Variable o3://${OM_SERVICE_ID}/${VOLUME}/encrypted-bucket/passwd
+ ${output} = Execute ozone sh key put ${key} /etc/passwd
+ Key Should Match Local File ${key} /etc/passwd
diff --git a/hadoop-ozone/dist/src/main/smoketest/security/ozone-secure-s3.robot b/hadoop-ozone/dist/src/main/smoketest/security/ozone-secure-s3.robot
index 5103e80279c2..70bade5f1e85 100644
--- a/hadoop-ozone/dist/src/main/smoketest/security/ozone-secure-s3.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/security/ozone-secure-s3.robot
@@ -24,6 +24,8 @@ Test Timeout 5 minutes
*** Variables ***
${ENDPOINT_URL} http://s3g:9878
+${TEMPDIR} /tmp
+${TEST_FILE} NOTICE.txt
*** Keywords ***
Setup volume names
@@ -38,6 +40,19 @@ Secure S3 test Success
${output} = Execute aws s3api --endpoint-url ${ENDPOINT_URL} list-buckets
Should contain ${output} bucket-test123
+Secure S3 put-object test
+ ${testFilePath} = Set Variable ${TEMPDIR}/${TEST_FILE}
+ Copy File ${TEST_FILE} ${testFilePath}
+ ${output} = Execute aws s3api --endpoint ${ENDPOINT_URL} put-object --bucket=bucket-test123 --key=tmp1/tmp2/NOTICE.txt --body=${testFilePath}
+ ${output} = Execute aws s3api --endpoint ${ENDPOINT_URL} list-objects --bucket=bucket-test123
+ Should contain ${output} tmp1/tmp2/NOTICE.txt
+ ${output} = Execute aws s3api --endpoint ${ENDPOINT_URL} put-object --bucket=bucket-test123 --key=tmp3//tmp4/NOTICE.txt --body=${testFilePath}
+ ${output} = Execute aws s3api --endpoint ${ENDPOINT_URL} list-objects --bucket=bucket-test123
+ Should contain ${output} tmp3//tmp4/NOTICE.txt
+ ${output} = Execute aws s3api --endpoint ${ENDPOINT_URL} put-object --bucket=bucket-test123 --key=//tmp5/tmp6/NOTICE.txt --body=${testFilePath}
+ ${output} = Execute aws s3api --endpoint ${ENDPOINT_URL} list-objects --bucket=bucket-test123
+ Should contain ${output} //tmp5/tmp6/NOTICE.txt
+
Secure S3 test Failure
Run Keyword Setup dummy credentials for S3
${rc} ${result} = Run And Return Rc And Output aws s3api --endpoint-url ${ENDPOINT_URL} create-bucket --bucket bucket-test123
diff --git a/hadoop-ozone/dist/src/main/smoketest/spnego/web.robot b/hadoop-ozone/dist/src/main/smoketest/spnego/web.robot
index 9c4156fcd6e4..065e390e5b84 100644
--- a/hadoop-ozone/dist/src/main/smoketest/spnego/web.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/spnego/web.robot
@@ -30,6 +30,11 @@ ${OM_SERVICE_LIST_URL} http://om:9874/serviceList
${SCM_URL} http://scm:9876
${RECON_URL} http://recon:9888
+${SCM_CONF_URL} http://scm:9876/conf
+${SCM_JMX_URL} http://scm:9876/jmx
+${SCM_STACKS_URL} http://scm:9876/stacks
+
+
*** Keywords ***
Verify SPNEGO enabled URL
[arguments] ${url}
@@ -60,6 +65,15 @@ Test OM Service List
Test SCM portal
Verify SPNEGO enabled URL ${SCM_URL}
+Test SCM conf
+ Verify SPNEGO enabled URL ${SCM_CONF_URL}
+
+Test SCM jmx
+ Verify SPNEGO enabled URL ${SCM_JMX_URL}
+
+Test SCM stacks
+ Verify SPNEGO enabled URL ${SCM_STACKS_URL}
+
Test Recon portal
Verify SPNEGO enabled URL ${RECON_URL}
diff --git a/hadoop-ozone/dist/src/shell/hdds/hadoop-daemons.sh b/hadoop-ozone/dist/src/shell/hdds/hadoop-daemons.sh
index 55304916ad1f..1d8096b4baae 100755
--- a/hadoop-ozone/dist/src/shell/hdds/hadoop-daemons.sh
+++ b/hadoop-ozone/dist/src/shell/hdds/hadoop-daemons.sh
@@ -16,7 +16,7 @@
# limitations under the License.
-# Run a Hadoop command on all slave hosts.
+# Run a Hadoop command on all worker hosts.
function hadoop_usage
{
diff --git a/hadoop-ozone/dist/src/shell/hdds/hadoop-functions.sh b/hadoop-ozone/dist/src/shell/hdds/hadoop-functions.sh
index b46045b2d8c0..769af336cac8 100755
--- a/hadoop-ozone/dist/src/shell/hdds/hadoop-functions.sh
+++ b/hadoop-ozone/dist/src/shell/hdds/hadoop-functions.sh
@@ -999,7 +999,7 @@ function hadoop_connect_to_hosts
# shellcheck disable=SC2124
local params="$@"
local worker_file
- local tmpslvnames
+ local tmp_worker_names
#
# ssh (or whatever) to a host
@@ -1030,10 +1030,10 @@ function hadoop_connect_to_hosts
else
# no spaces allowed in the pdsh arg host list
# shellcheck disable=SC2086
- tmpslvnames=$(echo ${HADOOP_WORKER_NAMES} | tr -s ' ' ,)
+ tmp_worker_names=$(echo ${HADOOP_WORKER_NAMES} | tr -s ' ' ,)
PDSH_SSH_ARGS_APPEND="${HADOOP_SSH_OPTS}" pdsh \
-f "${HADOOP_SSH_PARALLEL}" \
- -w "${tmpslvnames}" $"${@// /\\ }" 2>&1
+ -w "${tmp_worker_names}" $"${@// /\\ }" 2>&1
fi
else
if [[ -z "${HADOOP_WORKER_NAMES}" ]]; then
diff --git a/hadoop-ozone/dist/src/shell/ozone/ozone b/hadoop-ozone/dist/src/shell/ozone/ozone
index 42e8dcaf28e1..c536484e9b56 100755
--- a/hadoop-ozone/dist/src/shell/ozone/ozone
+++ b/hadoop-ozone/dist/src/shell/ozone/ozone
@@ -54,7 +54,6 @@ function hadoop_usage
hadoop_add_subcommand "insight" client "tool to get runtime operation information"
hadoop_add_subcommand "version" client "print the version"
hadoop_add_subcommand "dtutil" client "operations related to delegation tokens"
- hadoop_add_subcommand "upgrade" client "HDFS to Ozone in-place upgrade tool"
hadoop_add_subcommand "admin" client "Ozone admin tool"
hadoop_add_subcommand "debug" client "Ozone debug tool"
@@ -214,12 +213,8 @@ function ozonecmd_case
HADOOP_CLASSNAME=org.apache.hadoop.security.token.DtUtilShell
OZONE_RUN_ARTIFACT_NAME="hadoop-ozone-tools"
;;
- upgrade)
- HADOOP_CLASSNAME=org.apache.hadoop.ozone.upgrade.InPlaceUpgrade
- OZONE_RUN_ARTIFACT_NAME="hadoop-ozone-upgrade"
- ;;
admin)
- HADOOP_CLASSNAME=org.apache.hadoop.ozone.admin.OzoneAdmin
+ HADOOP_CLASSNAME=org.apache.hadoop.hdds.cli.OzoneAdmin
OZONE_RUN_ARTIFACT_NAME="hadoop-ozone-tools"
;;
debug)
diff --git a/hadoop-ozone/dist/src/shell/upgrade/1.0.0.sh b/hadoop-ozone/dist/src/shell/upgrade/1.0.0.sh
new file mode 100755
index 000000000000..65739787ee67
--- /dev/null
+++ b/hadoop-ozone/dist/src/shell/upgrade/1.0.0.sh
@@ -0,0 +1,23 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
+
+: "${SCM_DIR:="${OZONE_VOLUME}/scm"}"
+: "${OZONE_RUNNER_VERSION:="20200625-1"}"
+
+docker run --rm -v "${SCM_DIR}":/scm -v "${SCRIPT_DIR}/1.0.0":/upgrade -w /scm/metadata apache/ozone-runner:"${OZONE_RUNNER_VERSION}" /upgrade/01-migrate-scm-db.sh
diff --git a/hadoop-ozone/dist/src/shell/upgrade/1.0.0/01-migrate-scm-db.sh b/hadoop-ozone/dist/src/shell/upgrade/1.0.0/01-migrate-scm-db.sh
new file mode 100755
index 000000000000..dee676841972
--- /dev/null
+++ b/hadoop-ozone/dist/src/shell/upgrade/1.0.0/01-migrate-scm-db.sh
@@ -0,0 +1,24 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+echo Running upgrade script for HDDS-3499
+
+ldb --db=scm.db create_column_family containers
+ldb --db=scm.db create_column_family pipelines
+
+ldb --db=scm-container.db --key_hex --value_hex dump | ldb --db=scm.db --key_hex --value_hex --column_family=containers load
+ldb --db=scm-pipeline.db --key_hex --value_hex dump | ldb --db=scm.db --key_hex --value_hex --column_family=pipelines load
diff --git a/hadoop-ozone/dist/src/test/shell/compose_testlib.bats b/hadoop-ozone/dist/src/test/shell/compose_testlib.bats
new file mode 100644
index 000000000000..058da64f30d7
--- /dev/null
+++ b/hadoop-ozone/dist/src/test/shell/compose_testlib.bats
@@ -0,0 +1,37 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+
+load ../../main/compose/testlib.sh
+@test "Find test recursive, only on one level" {
+ cd $BATS_TEST_DIRNAME
+ run find_tests
+ [[ "$output" == "./test1/test.sh ./test2/test.sh ./test4/test.sh" ]]
+}
+
+@test "Find test by suite" {
+ OZONE_ACCEPTANCE_SUITE=one
+ cd $BATS_TEST_DIRNAME
+ run find_tests
+ [[ "$output" == "./test4/test.sh" ]]
+}
+
+@test "Find test default suite" {
+ OZONE_ACCEPTANCE_SUITE=misc
+ cd $BATS_TEST_DIRNAME
+ run find_tests
+ [[ "$output" == "./test1/test.sh ./test2/test.sh" ]]
+}
diff --git a/hadoop-ozone/dist/src/test/shell/gc_opts.bats b/hadoop-ozone/dist/src/test/shell/gc_opts.bats
index 1400a4058f78..feb29af0e35c 100644
--- a/hadoop-ozone/dist/src/test/shell/gc_opts.bats
+++ b/hadoop-ozone/dist/src/test/shell/gc_opts.bats
@@ -14,14 +14,12 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-
-
#
# Can be executed with bats (https://github.com/bats-core/bats-core)
-# bats gc_opts.bats (FROM THE CURRENT DIRECTORY)
+# bats gc_opts.bats
#
-source ../../shell/hdds/hadoop-functions.sh
+load ../../shell/hdds/hadoop-functions.sh
@test "Setting Hadoop GC parameters: add GC params for server" {
export HADOOP_SUBCMD_SUPPORTDAEMONIZATION=true
export HADOOP_OPTS="Test"
diff --git a/hadoop-ozone/dist/src/test/shell/k8s_testlib.bats b/hadoop-ozone/dist/src/test/shell/k8s_testlib.bats
new file mode 100644
index 000000000000..4558a1e114c4
--- /dev/null
+++ b/hadoop-ozone/dist/src/test/shell/k8s_testlib.bats
@@ -0,0 +1,55 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+export COUNTER=1
+
+pass_after_three_failures() {
+ if [ $COUNTER -eq 3 ]; then
+ return 0
+ fi
+ COUNTER=$(( COUNTER + 1))
+ return 255
+}
+
+pass_first() {
+ echo "pass"
+}
+
+pass_never() {
+ return 255
+}
+
+load ../../main/k8s/examples/testlib.sh
+
+@test "Test retry with passing function" {
+ retry pass_first
+}
+
+@test "Test retry with 3 failures" {
+ export RETRY_SLEEP=0
+ retry pass_after_three_failures
+}
+
+@test "Test retry always failure" {
+ export RETRY_SLEEP=0
+ run retry pass_never
+ [ "$status" -eq 255 ]
+}
+
+
+
+
+
diff --git a/hadoop-ozone/dist/src/test/shell/test1/test.sh b/hadoop-ozone/dist/src/test/shell/test1/test.sh
new file mode 100644
index 000000000000..b13ca90d239f
--- /dev/null
+++ b/hadoop-ozone/dist/src/test/shell/test1/test.sh
@@ -0,0 +1,15 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/test/shell/test2/test.sh b/hadoop-ozone/dist/src/test/shell/test2/test.sh
new file mode 100644
index 000000000000..8dbf5b29721c
--- /dev/null
+++ b/hadoop-ozone/dist/src/test/shell/test2/test.sh
@@ -0,0 +1,17 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+#suite:misc
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/test/shell/test3/subtest1/test.sh b/hadoop-ozone/dist/src/test/shell/test3/subtest1/test.sh
new file mode 100644
index 000000000000..8dbf5b29721c
--- /dev/null
+++ b/hadoop-ozone/dist/src/test/shell/test3/subtest1/test.sh
@@ -0,0 +1,17 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+#suite:misc
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/test/shell/test4/test.sh b/hadoop-ozone/dist/src/test/shell/test4/test.sh
new file mode 100644
index 000000000000..accc445711e5
--- /dev/null
+++ b/hadoop-ozone/dist/src/test/shell/test4/test.sh
@@ -0,0 +1,17 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+#suite:one
\ No newline at end of file
diff --git a/hadoop-ozone/fault-injection-test/mini-chaos-tests/pom.xml b/hadoop-ozone/fault-injection-test/mini-chaos-tests/pom.xml
index 941a9cf354c4..5523150b8585 100644
--- a/hadoop-ozone/fault-injection-test/mini-chaos-tests/pom.xml
+++ b/hadoop-ozone/fault-injection-test/mini-chaos-tests/pom.xml
@@ -20,9 +20,9 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
hadoop-ozone-fault-injection-testorg.apache.hadoop
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOT
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOTApache Hadoop Ozone Mini Ozone Chaos TestsApache Hadoop Ozone Mini Ozone Chaos Tests
diff --git a/hadoop-ozone/fault-injection-test/mini-chaos-tests/src/test/java/org/apache/hadoop/ozone/MiniOzoneChaosCluster.java b/hadoop-ozone/fault-injection-test/mini-chaos-tests/src/test/java/org/apache/hadoop/ozone/MiniOzoneChaosCluster.java
index 2a5cf24603e3..3267976f7670 100644
--- a/hadoop-ozone/fault-injection-test/mini-chaos-tests/src/test/java/org/apache/hadoop/ozone/MiniOzoneChaosCluster.java
+++ b/hadoop-ozone/fault-injection-test/mini-chaos-tests/src/test/java/org/apache/hadoop/ozone/MiniOzoneChaosCluster.java
@@ -225,7 +225,8 @@ protected void initializeConfiguration() throws IOException {
conf.setTimeDuration(HddsConfigKeys.HDDS_HEARTBEAT_INTERVAL, 1,
TimeUnit.SECONDS);
conf.setInt(
- OzoneConfigKeys.DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_KEY,
+ OzoneConfigKeys
+ .DFS_CONTAINER_RATIS_NUM_WRITE_CHUNK_THREADS_PER_VOLUME_KEY,
4);
conf.setInt(
OzoneConfigKeys.DFS_CONTAINER_RATIS_NUM_CONTAINER_OP_EXECUTORS_KEY,
diff --git a/hadoop-ozone/fault-injection-test/mini-chaos-tests/src/test/resources/META-INF/services/org.apache.hadoop.fs.FileSystem b/hadoop-ozone/fault-injection-test/mini-chaos-tests/src/test/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
index 03680027d539..e444f66e7ce1 100644
--- a/hadoop-ozone/fault-injection-test/mini-chaos-tests/src/test/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
+++ b/hadoop-ozone/fault-injection-test/mini-chaos-tests/src/test/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
@@ -14,3 +14,4 @@
# limitations under the License.
org.apache.hadoop.fs.ozone.OzoneFileSystem
+org.apache.hadoop.fs.ozone.RootedOzoneFileSystem
diff --git a/hadoop-ozone/fault-injection-test/network-tests/pom.xml b/hadoop-ozone/fault-injection-test/network-tests/pom.xml
index 49c509853cdf..7fe3790b08fc 100644
--- a/hadoop-ozone/fault-injection-test/network-tests/pom.xml
+++ b/hadoop-ozone/fault-injection-test/network-tests/pom.xml
@@ -20,7 +20,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
org.apache.hadoophadoop-ozone-fault-injection-test
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOThadoop-ozone-network-testsApache Hadoop Ozone Network Tests
diff --git a/hadoop-ozone/fault-injection-test/pom.xml b/hadoop-ozone/fault-injection-test/pom.xml
index 99b396784a1b..bf933a3b355f 100644
--- a/hadoop-ozone/fault-injection-test/pom.xml
+++ b/hadoop-ozone/fault-injection-test/pom.xml
@@ -20,10 +20,10 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
org.apache.hadoophadoop-ozone
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOThadoop-ozone-fault-injection-test
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOTApache Hadoop Ozone Fault Injection TestsApache Hadoop Ozone Fault Injection Testspom
diff --git a/hadoop-ozone/insight/pom.xml b/hadoop-ozone/insight/pom.xml
index f40175099922..090aecad341c 100644
--- a/hadoop-ozone/insight/pom.xml
+++ b/hadoop-ozone/insight/pom.xml
@@ -20,10 +20,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
org.apache.hadoophadoop-ozone
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOThadoop-ozone-insight
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOTApache Hadoop Ozone Insight ToolApache Hadoop Ozone Insight Tooljar
diff --git a/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/Insight.java b/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/Insight.java
index 690783ee411b..b4080796be2a 100644
--- a/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/Insight.java
+++ b/hadoop-ozone/insight/src/main/java/org/apache/hadoop/ozone/insight/Insight.java
@@ -34,6 +34,10 @@
mixinStandardHelpOptions = true)
public class Insight extends GenericCli {
+ public Insight() {
+ super(Insight.class);
+ }
+
public static void main(String[] args) throws Exception {
new Insight().run(args);
}
diff --git a/hadoop-ozone/integration-test/pom.xml b/hadoop-ozone/integration-test/pom.xml
index 622a4077da0c..ebfe1c0057c8 100644
--- a/hadoop-ozone/integration-test/pom.xml
+++ b/hadoop-ozone/integration-test/pom.xml
@@ -20,10 +20,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
org.apache.hadoophadoop-ozone
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOThadoop-ozone-integration-test
- 0.6.0-SNAPSHOT
+ 1.1.0-SNAPSHOTApache Hadoop Ozone Integration TestsApache Hadoop Ozone Integration Testsjar
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSWithObjectStoreCreate.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSWithObjectStoreCreate.java
new file mode 100644
index 000000000000..c4e543554a6a
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSWithObjectStoreCreate.java
@@ -0,0 +1,231 @@
+/**
+ * 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
+ *