HDFS-13424. Ozone: Refactor MiniOzoneClassicCluster. Contributed by Nanda Kumar.

This commit is contained in:
Mukul Kumar Singh 2018-04-16 20:18:27 +05:30 committed by Owen O'Malley
parent 025058f251
commit ae8ac7f082
53 changed files with 1140 additions and 1145 deletions

View File

@ -51,18 +51,42 @@ public class HddsDatanodeService implements ServicePlugin {
HddsDatanodeService.class);
private Configuration conf;
private OzoneConfiguration conf;
private DatanodeDetails datanodeDetails;
private DatanodeStateMachine datanodeStateMachine;
private List<ServicePlugin> plugins;
/**
* Default constructor.
*/
public HddsDatanodeService() {
this(null);
}
/**
* Constructs {@link HddsDatanodeService} using the provided {@code conf}
* value.
*
* @param conf OzoneConfiguration
*/
public HddsDatanodeService(Configuration conf) {
if (conf == null) {
this.conf = new OzoneConfiguration();
} else {
this.conf = new OzoneConfiguration(conf);
}
}
/**
* Starts HddsDatanode services.
*
* @param service The service instance invoking this method
*/
@Override
public void start(Object service) {
OzoneConfiguration.activate();
if (service instanceof Configurable) {
conf = new OzoneConfiguration(((Configurable) service).getConf());
} else {
conf = new OzoneConfiguration();
}
if (HddsUtils.isHddsEnabled(conf)) {
try {
@ -109,6 +133,11 @@ public class HddsDatanodeService implements ServicePlugin {
return DatanodeDetails.newBuilder().setUuid(datanodeUuid).build();
}
}
/**
* Starts all the service plugins which are configured using
* OzoneConfigKeys.HDDS_DATANODE_PLUGINS_KEY.
*/
private void startPlugins() {
try {
plugins = conf.getInstances(HDDS_DATANODE_PLUGINS_KEY,
@ -130,7 +159,12 @@ public class HddsDatanodeService implements ServicePlugin {
}
}
public Configuration getConf() {
/**
* Returns the OzoneConfiguration used by this HddsDatanodeService.
*
* @return OzoneConfiguration
*/
public OzoneConfiguration getConf() {
return conf;
}
/**
@ -149,8 +183,13 @@ public class HddsDatanodeService implements ServicePlugin {
return datanodeStateMachine;
}
public void join() throws InterruptedException {
datanodeStateMachine.join();
public void join() {
try {
datanodeStateMachine.join();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
LOG.info("Interrupted during StorageContainerManager join.");
}
}
@Override
@ -172,20 +211,31 @@ public class HddsDatanodeService implements ServicePlugin {
@Override
public void close() throws IOException {
if (plugins != null) {
for (ServicePlugin plugin : plugins) {
try {
plugin.close();
} catch (Throwable t) {
LOG.warn("ServicePlugin {} could not be closed", plugin, t);
}
}
}
}
public static HddsDatanodeService createHddsDatanodeService(String args[]) {
StringUtils.startupShutdownMessage(HddsDatanodeService.class, args, LOG);
return new HddsDatanodeService();
public static HddsDatanodeService createHddsDatanodeService(
Configuration conf) {
return new HddsDatanodeService(conf);
}
public static void main(String args[]) {
try {
HddsDatanodeService hddsDatanodeService = createHddsDatanodeService(args);
StringUtils.startupShutdownMessage(HddsDatanodeService.class, args, LOG);
HddsDatanodeService hddsDatanodeService =
createHddsDatanodeService(new OzoneConfiguration());
hddsDatanodeService.start(null);
hddsDatanodeService.join();
} catch (Throwable e) {
LOG.error("Exception in while starting HddsDatanodeService.", e);
LOG.error("Exception in HddsDatanodeService.", e);
terminate(1, e);
}
}

View File

@ -61,7 +61,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
<scope>test</scope>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.openjdk.jmh</groupId>
<artifactId>jmh-core</artifactId>

View File

@ -18,15 +18,10 @@ package org.apache.hadoop.hdds.scm.container;
import org.apache.commons.lang.RandomStringUtils;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.container.ContainerMapping;
import org.apache.hadoop.hdds.scm.container.ContainerStateManager;
import org.apache.hadoop.hdds.scm.container.Mapping;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.StorageContainerManager;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.junit.After;
@ -54,10 +49,10 @@ public class TestContainerStateManager {
@Before
public void setup() throws IOException {
public void setup() throws Exception {
conf = new OzoneConfiguration();
cluster = new MiniOzoneClassicCluster.Builder(conf).numDataNodes(1)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
cluster.waitForClusterToBeReady();
xceiverClientManager = new XceiverClientManager(conf);
scm = cluster.getStorageContainerManager();
scmContainerMapping = scm.getScmContainerManager();
@ -68,7 +63,6 @@ public class TestContainerStateManager {
public void cleanUp() {
if (cluster != null) {
cluster.shutdown();
cluster.close();
}
}

View File

@ -1,616 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone;
import java.io.File;
import java.util.Optional;
import com.google.common.base.Preconditions;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.ipc.Client;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.ozone.client.rest.OzoneException;
import org.apache.hadoop.ozone.container.common
.statemachine.DatanodeStateMachine;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
import org.apache.hadoop.ozone.ksm.KeySpaceManager;
import org.apache.hadoop.hdds.scm.SCMStorage;
import org.apache.hadoop.ozone.ksm.KSMStorage;
import org.apache.hadoop.ozone.web.client.OzoneRestClient;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.protocolPB
.StorageContainerLocationProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
import org.apache.hadoop.hdds.scm.StorageContainerManager;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.test.GenericTestUtils;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_PLUGINS_KEY;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.slf4j.event.Level;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.URISyntaxException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.Random;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import static org.apache.hadoop.ozone.MiniOzoneTestHelper.*;
import static org.apache.hadoop.ozone.OzoneConfigKeys
.DFS_CONTAINER_IPC_PORT;
import static org.apache.hadoop.ozone.OzoneConfigKeys
.DFS_CONTAINER_IPC_RANDOM_PORT;
import static org.apache.hadoop.ozone.OzoneConfigKeys
.DFS_CONTAINER_RATIS_IPC_PORT;
import static org.apache.hadoop.ozone.OzoneConfigKeys
.DFS_CONTAINER_RATIS_IPC_RANDOM_PORT;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState
.HEALTHY;
import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_PLUGINS_KEY;
import static org.junit.Assert.assertFalse;
/**
* MiniOzoneCluster creates a complete in-process Ozone cluster suitable for
* running tests. The cluster consists of a StorageContainerManager, Namenode
* and multiple DataNodes. This class subclasses {@link MiniDFSCluster} for
* convenient reuse of logic for starting DataNodes.
*/
@InterfaceAudience.Private
public final class MiniOzoneClassicCluster extends MiniDFSCluster
implements MiniOzoneCluster {
private static final Logger LOG =
LoggerFactory.getLogger(MiniOzoneClassicCluster.class);
private static final String USER_AUTH = "hdfs";
private final OzoneConfiguration conf;
private final StorageContainerManager scm;
private KeySpaceManager ksm;
private final Path tempPath;
/**
* Creates a new MiniOzoneCluster.
*
* @param builder cluster builder
* @param scm StorageContainerManager, already running
* @throws IOException if there is an I/O error
*/
private MiniOzoneClassicCluster(Builder builder, StorageContainerManager scm,
KeySpaceManager ksm)
throws IOException {
super(builder);
this.conf = builder.conf;
this.scm = scm;
this.ksm = ksm;
tempPath = Paths.get(builder.getPath(), builder.getRunID());
}
@Override
protected void setupDatanodeAddress(
int i, Configuration dnConf, boolean setupHostsFile,
boolean checkDnAddrConf) throws IOException {
super.setupDatanodeAddress(i, dnConf, setupHostsFile, checkDnAddrConf);
String path = GenericTestUtils.getTempPath(
MiniOzoneClassicCluster.class.getSimpleName() + "datanode");
dnConf.setStrings(ScmConfigKeys.OZONE_SCM_DATANODE_ID,
path + "/" + i + "-datanode.id");
setConf(i, dnConf, OzoneConfigKeys.DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR,
getInstanceStorageDir(i, -1).getCanonicalPath());
String containerMetaDirs = dnConf.get(
OzoneConfigKeys.OZONE_METADATA_DIRS) + "-dn-" + i;
Path containerMetaDirPath = Paths.get(containerMetaDirs);
setConf(i, dnConf, OzoneConfigKeys.OZONE_METADATA_DIRS,
containerMetaDirs);
Path containerRootPath =
containerMetaDirPath.resolve(OzoneConsts.CONTAINER_ROOT_PREFIX);
Files.createDirectories(containerRootPath);
}
static void setConf(int i, Configuration conf, String key, String value) {
conf.set(key, value);
LOG.info("dn{}: set {} = {}", i, key, value);
}
@Override
public void close() {
shutdown();
try {
FileUtils.deleteDirectory(tempPath.toFile());
} catch (IOException e) {
String errorMessage = "Cleaning up metadata directories failed." + e;
assertFalse(errorMessage, true);
}
try {
final String localStorage =
conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
FileUtils.deleteDirectory(new File(localStorage));
} catch (IOException e) {
LOG.error("Cleaning up local storage failed", e);
}
}
@Override
public boolean restartDataNode(int i) throws IOException {
return restartDataNode(i, true);
}
/*
* Restart a particular datanode, wait for it to become active
*/
@Override
public boolean restartDataNode(int i, boolean keepPort) throws IOException {
LOG.info("restarting datanode:{} keepPort:{}", i, keepPort);
if (keepPort) {
DataNodeProperties dnProp = dataNodes.get(i);
OzoneContainer container = getOzoneContainer(dnProp
.getDatanode());
Configuration config = dnProp.getConf();
int currentPort = container.getContainerServerPort();
config.setInt(DFS_CONTAINER_IPC_PORT, currentPort);
config.setBoolean(DFS_CONTAINER_IPC_RANDOM_PORT, false);
int ratisPort = container.getRatisContainerServerPort();
config.setInt(DFS_CONTAINER_RATIS_IPC_PORT, ratisPort);
config.setBoolean(DFS_CONTAINER_RATIS_IPC_RANDOM_PORT, false);
}
boolean status = super.restartDataNode(i, keepPort);
try {
this.waitActive();
this.waitFirstBRCompleted(0, 3000);
waitDatanodeOzoneReady(i);
} catch (TimeoutException | InterruptedException e) {
Thread.interrupted();
}
return status;
}
@Override
public void shutdown() {
super.shutdown();
LOG.info("Shutting down the Mini Ozone Cluster");
if (ksm != null) {
LOG.info("Shutting down the keySpaceManager");
ksm.stop();
ksm.join();
}
if (scm != null) {
LOG.info("Shutting down the StorageContainerManager");
scm.stop();
scm.join();
}
}
@Override
public StorageContainerManager getStorageContainerManager() {
return this.scm;
}
public OzoneConfiguration getConf() {
return conf;
}
@Override
public KeySpaceManager getKeySpaceManager() {
return this.ksm;
}
/**
* Creates an {@link OzoneRestClient} connected to this cluster's REST
* service. Callers take ownership of the client and must close it when done.
*
* @return OzoneRestClient connected to this cluster's REST service
* @throws OzoneException if Ozone encounters an error creating the client
*/
@Override
public OzoneRestClient createOzoneRestClient() throws OzoneException {
Preconditions.checkState(!getDataNodes().isEmpty(),
"Cannot create OzoneRestClient if the cluster has no DataNodes.");
// An Ozone request may originate at any DataNode, so pick one at random.
int dnIndex = new Random().nextInt(getDataNodes().size());
String uri = String.format("http://127.0.0.1:%d",
MiniOzoneTestHelper.getOzoneRestPort(getDataNodes().get(dnIndex)));
LOG.info("Creating Ozone client to DataNode {} with URI {} and user {}",
dnIndex, uri, USER_AUTH);
try {
return new OzoneRestClient(uri, USER_AUTH);
} catch (URISyntaxException e) {
// We control the REST service URI, so it should never be invalid.
throw new IllegalStateException("Unexpected URISyntaxException", e);
}
}
/**
* Creates an RPC proxy connected to this cluster's StorageContainerManager
* for accessing container location information. Callers take ownership of
* the proxy and must close it when done.
*
* @return RPC proxy for accessing container location information
* @throws IOException if there is an I/O error
*/
@Override
public StorageContainerLocationProtocolClientSideTranslatorPB
createStorageContainerLocationClient() throws IOException {
long version = RPC.getProtocolVersion(
StorageContainerLocationProtocolPB.class);
InetSocketAddress address = scm.getClientRpcAddress();
LOG.info(
"Creating StorageContainerLocationProtocol RPC client with address {}",
address);
return new StorageContainerLocationProtocolClientSideTranslatorPB(
RPC.getProxy(StorageContainerLocationProtocolPB.class, version,
address, UserGroupInformation.getCurrentUser(), conf,
NetUtils.getDefaultSocketFactory(conf),
Client.getRpcTimeout(conf)));
}
/**
* Waits for the Ozone cluster to be ready for processing requests.
*/
@Override
public void waitOzoneReady() throws TimeoutException, InterruptedException {
GenericTestUtils.waitFor(() -> {
final int healthy = scm.getNodeCount(HEALTHY);
final boolean isReady = healthy >= numDataNodes;
LOG.info("{}. Got {} of {} DN Heartbeats.",
isReady? "Cluster is ready" : "Waiting for cluster to be ready",
healthy, numDataNodes);
return isReady;
}, 1000, 60 * 1000); //wait for 1 min.
}
/**
* Waits for a particular Datanode to be ready for processing ozone requests.
*/
@Override
public void waitDatanodeOzoneReady(int dnIndex)
throws TimeoutException, InterruptedException {
GenericTestUtils.waitFor(() -> {
DatanodeStateMachine.DatanodeStates state =
MiniOzoneTestHelper.getStateMachine(dataNodes.get(dnIndex)
.getDatanode()).getContext().getState();
final boolean rebootComplete =
(state == DatanodeStateMachine.DatanodeStates.RUNNING);
LOG.info("{} Current state:{}", rebootComplete, state);
return rebootComplete;
}, 1000, 60 * 1000); //wait for 1 min.
}
/**
* Waits for SCM to be out of Chill Mode. Many tests can be run iff we are out
* of Chill mode.
*
* @throws TimeoutException
* @throws InterruptedException
*/
@Override
public void waitTobeOutOfChillMode() throws TimeoutException,
InterruptedException {
GenericTestUtils.waitFor(() -> {
if (scm.getScmNodeManager().isOutOfChillMode()) {
return true;
}
LOG.info("Waiting for cluster to be ready. No datanodes found");
return false;
}, 100, 45000);
}
@Override
public void waitForHeartbeatProcessed() throws TimeoutException,
InterruptedException {
GenericTestUtils.waitFor(() ->
scm.getScmNodeManager().waitForHeartbeatProcessed(), 100,
4 * 1000);
GenericTestUtils.waitFor(() ->
scm.getScmNodeManager().getStats().getCapacity().get() > 0, 100,
4 * 1000);
}
/**
* Builder for configuring the MiniOzoneCluster to run.
*/
public static class Builder
extends MiniDFSCluster.Builder {
private final OzoneConfiguration conf;
private static final int DEFAULT_HB_SECONDS = 1;
private static final int DEFAULT_PROCESSOR_MS = 100;
private final String path;
private final UUID runID;
private Optional<String> ozoneHandlerType = java.util.Optional.empty();
private Optional<Boolean> enableTrace = Optional.of(false);
private Optional<Integer> hbSeconds = Optional.empty();
private Optional<Integer> hbProcessorInterval = Optional.empty();
private Optional<String> scmMetadataDir = Optional.empty();
private Optional<String> clusterId = Optional.empty();
private Optional<String> scmId = Optional.empty();
private Optional<String> ksmId = Optional.empty();
private Boolean ozoneEnabled = true;
private Boolean waitForChillModeFinish = true;
private Boolean randomContainerPort = true;
// Use relative smaller number of handlers for testing
private int numOfKsmHandlers = 20;
private int numOfScmHandlers = 20;
/**
* Creates a new Builder.
*
* @param conf configuration
*/
public Builder(OzoneConfiguration conf) {
super(conf);
// Mini Ozone cluster will not come up if the port is not true, since
// Ratis will exit if the server port cannot be bound. We can remove this
// hard coding once we fix the Ratis default behaviour.
conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_RANDOM_PORT,
true);
this.conf = conf;
path = GenericTestUtils.getTempPath(
MiniOzoneClassicCluster.class.getSimpleName() +
UUID.randomUUID().toString());
runID = UUID.randomUUID();
}
public Builder setRandomContainerPort(boolean randomPort) {
this.randomContainerPort = randomPort;
return this;
}
@Override
public Builder numDataNodes(int val) {
super.numDataNodes(val);
return this;
}
@Override
public Builder storageCapacities(long[] capacities) {
super.storageCapacities(capacities);
return this;
}
public Builder setHandlerType(String handler) {
ozoneHandlerType = Optional.of(handler);
return this;
}
public Builder setTrace(Boolean trace) {
enableTrace = Optional.of(trace);
return this;
}
public Builder setSCMHBInterval(int seconds) {
hbSeconds = Optional.of(seconds);
return this;
}
public Builder setSCMHeartbeatProcessingInterval(int milliseconds) {
hbProcessorInterval = Optional.of(milliseconds);
return this;
}
public Builder setSCMMetadataDir(String scmMetadataDirPath) {
scmMetadataDir = Optional.of(scmMetadataDirPath);
return this;
}
public Builder disableOzone() {
ozoneEnabled = false;
return this;
}
public Builder doNotwaitTobeOutofChillMode() {
waitForChillModeFinish = false;
return this;
}
public Builder setNumOfKSMHandlers(int numOfHandlers) {
numOfKsmHandlers = numOfHandlers;
return this;
}
public Builder setNumOfSCMHandlers(int numOfHandlers) {
numOfScmHandlers = numOfHandlers;
return this;
}
public Builder setClusterId(String cId) {
clusterId = Optional.of(cId);
return this;
}
public Builder setScmId(String sId) {
scmId = Optional.of(sId);
return this;
}
public Builder setKsmId(String kId) {
ksmId = Optional.of(kId);
return this;
}
public String getPath() {
return path;
}
public String getRunID() {
return runID.toString();
}
@Override
public MiniOzoneClassicCluster build() throws IOException {
configureHandler();
configureTrace();
configureSCMheartbeat();
configScmMetadata();
initializeScm();
initializeKSM();
conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "127.0.0.1:0");
conf.set(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY, "127.0.0.1:0");
conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "127.0.0.1:0");
conf.set(ScmConfigKeys.OZONE_SCM_HTTP_ADDRESS_KEY, "127.0.0.1:0");
conf.set(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY, "127.0.0.1:0");
conf.set(KSMConfigKeys.OZONE_KSM_HTTP_ADDRESS_KEY, "127.0.0.1:0");
conf.set(ScmConfigKeys.HDDS_REST_HTTP_ADDRESS_KEY, "127.0.0.1:0");
conf.set(DFS_DATANODE_PLUGINS_KEY,
"org.apache.hadoop.ozone.HddsDatanodeService");
conf.set(HDDS_DATANODE_PLUGINS_KEY,
"org.apache.hadoop.ozone.web.OzoneHddsDatanodeService");
// Configure KSM and SCM handlers
conf.setInt(ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_KEY, numOfScmHandlers);
conf.setInt(KSMConfigKeys.OZONE_KSM_HANDLER_COUNT_KEY, numOfKsmHandlers);
// Use random ports for ozone containers in mini cluster,
// in order to launch multiple container servers per node.
conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT,
randomContainerPort);
StorageContainerManager scm = StorageContainerManager.createSCM(
null, conf);
scm.start();
KeySpaceManager ksm = KeySpaceManager.createKSM(null, conf);
ksm.start();
String addressString = scm.getDatanodeRpcAddress().getHostString() +
":" + scm.getDatanodeRpcAddress().getPort();
conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, addressString);
MiniOzoneClassicCluster cluster =
new MiniOzoneClassicCluster(this, scm, ksm);
try {
cluster.waitOzoneReady();
if (waitForChillModeFinish) {
cluster.waitTobeOutOfChillMode();
}
cluster.waitForHeartbeatProcessed();
} catch (Exception e) {
// A workaround to propagate MiniOzoneCluster failures without
// changing the method signature (which would require cascading
// changes to hundreds of unrelated HDFS tests).
throw new IOException("Failed to start MiniOzoneCluster", e);
}
return cluster;
}
private void configScmMetadata() throws IOException {
if (scmMetadataDir.isPresent()) {
// if user specifies a path in the test, it is assumed that user takes
// care of creating and cleaning up that directory after the tests.
conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS,
scmMetadataDir.get());
return;
}
// If user has not specified a path, create a UUID for this miniCluster
// and create SCM under that directory.
Path scmPath = Paths.get(path, runID.toString(), "cont-meta");
Files.createDirectories(scmPath);
Path containerPath = scmPath.resolve(OzoneConsts.CONTAINER_ROOT_PREFIX);
Files.createDirectories(containerPath);
conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, scmPath
.toString());
}
private void initializeScm() throws IOException {
SCMStorage scmStore = new SCMStorage(conf);
if (!clusterId.isPresent()) {
clusterId = Optional.of(runID.toString());
}
scmStore.setClusterId(clusterId.get());
if (!scmId.isPresent()) {
scmId = Optional.of(UUID.randomUUID().toString());
}
scmStore.setScmId(scmId.get());
scmStore.initialize();
}
private void initializeKSM() throws IOException {
KSMStorage ksmStore = new KSMStorage(conf);
ksmStore.setClusterId(clusterId.get());
ksmStore.setScmId(scmId.get());
ksmStore.setKsmId(ksmId.orElse(UUID.randomUUID().toString()));
ksmStore.initialize();
}
private void configureHandler() {
conf.setBoolean(OzoneConfigKeys.OZONE_ENABLED, this.ozoneEnabled);
if (!ozoneHandlerType.isPresent()) {
throw new IllegalArgumentException(
"The Ozone handler type must be specified.");
} else {
conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
ozoneHandlerType.get());
}
}
private void configureTrace() {
if (enableTrace.isPresent()) {
conf.setBoolean(OzoneConfigKeys.OZONE_TRACE_ENABLED_KEY,
enableTrace.get());
GenericTestUtils.setRootLogLevel(Level.TRACE);
}
GenericTestUtils.setRootLogLevel(Level.INFO);
}
private void configureSCMheartbeat() {
if (hbSeconds.isPresent()) {
conf.getTimeDuration(ScmConfigKeys.OZONE_SCM_HEARTBEAT_INTERVAL,
hbSeconds.get(), TimeUnit.SECONDS);
} else {
conf.setTimeDuration(ScmConfigKeys.OZONE_SCM_HEARTBEAT_INTERVAL,
DEFAULT_HB_SECONDS,
TimeUnit.SECONDS);
}
if (hbProcessorInterval.isPresent()) {
conf.setTimeDuration(
ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL,
hbProcessorInterval.get(),
TimeUnit.MILLISECONDS);
} else {
conf.setTimeDuration(
ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL,
DEFAULT_PROCESSOR_MS,
TimeUnit.MILLISECONDS);
}
}
}
}

View File

@ -17,46 +17,303 @@
*/
package org.apache.hadoop.ozone;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.StorageContainerManager;
import org.apache.hadoop.ozone.client.rest.OzoneException;
import org.apache.hadoop.ozone.client.OzoneClient;
import org.apache.hadoop.ozone.ksm.KeySpaceManager;
import org.apache.hadoop.ozone.web.client.OzoneRestClient;
import org.apache.hadoop.hdds.scm.protocolPB
.StorageContainerLocationProtocolClientSideTranslatorPB;
import org.apache.hadoop.test.GenericTestUtils;
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
import java.util.Optional;
import java.util.UUID;
import java.util.concurrent.TimeoutException;
/**
* Interface used for MiniOzoneClusters.
*/
public interface MiniOzoneCluster extends AutoCloseable, Closeable {
void close();
public interface MiniOzoneCluster {
boolean restartDataNode(int i) throws IOException;
/**
* Returns the configuration object associated with the MiniOzoneCluster.
*
* @return Configuration
*/
Configuration getConf();
boolean restartDataNode(int i, boolean keepPort) throws IOException;
/**
* Waits for the cluster to be ready, this call blocks till all the
* configured {@link HddsDatanodeService} registers with
* {@link StorageContainerManager}.
*
* @throws TimeoutException In case of timeout
* @throws InterruptedException In case of interrupt while waiting
*/
void waitForClusterToBeReady() throws TimeoutException, InterruptedException;
void shutdown();
/**
* Waits/blocks till the cluster is out of chill mode.
*
* @throws TimeoutException TimeoutException In case of timeout
* @throws InterruptedException In case of interrupt while waiting
*/
void waitTobeOutOfChillMode() throws TimeoutException, InterruptedException;
/**
* Returns {@link StorageContainerManager} associated with this
* {@link MiniOzoneCluster} instance.
*
* @return {@link StorageContainerManager} instance
*/
StorageContainerManager getStorageContainerManager();
/**
* Returns {@link KeySpaceManager} associated with this
* {@link MiniOzoneCluster} instance.
*
* @return {@link KeySpaceManager} instance
*/
KeySpaceManager getKeySpaceManager();
OzoneRestClient createOzoneRestClient() throws OzoneException;
/**
* Returns the list of {@link HddsDatanodeService} which are part of this
* {@link MiniOzoneCluster} instance.
*
* @return List of {@link HddsDatanodeService}
*/
List<HddsDatanodeService> getHddsDatanodes();
/**
* Returns an {@link OzoneClient} to access the {@link MiniOzoneCluster}.
*
* @return {@link OzoneClient}
* @throws IOException
*/
OzoneClient getClient() throws IOException;
/**
* Returns an RPC based {@link OzoneClient} to access the
* {@link MiniOzoneCluster}.
*
* @return {@link OzoneClient}
* @throws IOException
*/
OzoneClient getRpcClient() throws IOException;
/**
* Returns an REST based {@link OzoneClient} to access the
* {@link MiniOzoneCluster}.
*
* @return {@link OzoneClient}
* @throws IOException
*/
OzoneClient getRestClient() throws IOException;
/**
* Returns StorageContainerLocationClient to communicate with
* {@link StorageContainerManager} associated with the MiniOzoneCluster.
*
* @return StorageContainerLocation Client
* @throws IOException
*/
StorageContainerLocationProtocolClientSideTranslatorPB
createStorageContainerLocationClient() throws IOException;
getStorageContainerLocationClient() throws IOException;
void waitOzoneReady() throws TimeoutException, InterruptedException;
/**
* Restarts StorageContainerManager instance.
*
* @throws IOException
*/
void restartStorageContainerManager() throws IOException;
void waitDatanodeOzoneReady(int dnIndex)
throws TimeoutException, InterruptedException;
/**
* Restarts KeySpaceManager instance.
*
* @throws IOException
*/
void restartKeySpaceManager() throws IOException;
void waitTobeOutOfChillMode() throws TimeoutException,
InterruptedException;
/**
* Restart a particular HddsDatanode.
*
* @param i index of HddsDatanode in the MiniOzoneCluster
*/
void restartHddsDatanode(int i);
void waitForHeartbeatProcessed() throws TimeoutException,
InterruptedException;
/**
* Shutdown a particular HddsDatanode.
*
* @param i index of HddsDatanode in the MiniOzoneCluster
*/
void shutdownHddsDatanode(int i);
/**
* Shutdown the MiniOzoneCluster.
*/
void shutdown();
/**
* Returns the Builder to construct MiniOzoneCluster.
*
* @param conf OzoneConfiguration
*
* @return MiniOzoneCluster builder
*/
static Builder newBuilder(OzoneConfiguration conf) {
return new MiniOzoneClusterImpl.Builder(conf);
}
abstract class Builder {
protected static final int DEFAULT_HB_INTERVAL_MS = 1000;
protected static final int DEFAULT_HB_PROCESSOR_INTERVAL_MS = 100;
protected final OzoneConfiguration conf;
protected final String path;
protected String clusterId;
protected Optional<Boolean> enableTrace = Optional.of(false);
protected Optional<Integer> hbInterval = Optional.empty();
protected Optional<Integer> hbProcessorInterval = Optional.empty();
protected Optional<String> scmId = Optional.empty();
protected Optional<String> ksmId = Optional.empty();
protected Boolean ozoneEnabled = true;
protected Boolean randomContainerPort = true;
// Use relative smaller number of handlers for testing
protected int numOfKsmHandlers = 20;
protected int numOfScmHandlers = 20;
protected int numOfDatanodes = 1;
protected Builder(OzoneConfiguration conf) {
this.conf = conf;
this.clusterId = UUID.randomUUID().toString();
this.path = GenericTestUtils.getTempPath(
MiniOzoneClusterImpl.class.getSimpleName() + "-" + clusterId);
}
/**
* Sets the cluster Id.
*
* @param id cluster Id
*
* @return MiniOzoneCluster.Builder
*/
public Builder setClusterId(String id) {
clusterId = id;
return this;
}
/**
* Sets the SCM id.
*
* @param id SCM Id
*
* @return MiniOzoneCluster.Builder
*/
public Builder setScmId(String id) {
scmId = Optional.of(id);
return this;
}
/**
* Sets the KSM id.
*
* @param id KSM Id
*
* @return MiniOzoneCluster.Builder
*/
public Builder setKsmId(String id) {
ksmId = Optional.of(id);
return this;
}
/**
* If set to true container service will be started in a random port.
*
* @param randomPort enable random port
*
* @return MiniOzoneCluster.Builder
*/
public Builder setRandomContainerPort(boolean randomPort) {
randomContainerPort = randomPort;
return this;
}
/**
* Sets the number of HddsDatanodes to be started as part of
* MiniOzoneCluster.
*
* @param val number of datanodes
*
* @return MiniOzoneCluster.Builder
*/
public Builder setNumDatanodes(int val) {
numOfDatanodes = val;
return this;
}
/**
* Sets the number of HeartBeat Interval of Datanodes, the value should be
* in MilliSeconds.
*
* @param val HeartBeat interval in milliseconds
*
* @return MiniOzoneCluster.Builder
*/
public Builder setHbInterval(int val) {
hbInterval = Optional.of(val);
return this;
}
/**
* Sets the number of HeartBeat Processor Interval of Datanodes,
* the value should be in MilliSeconds.
*
* @param val HeartBeat Processor interval in milliseconds
*
* @return MiniOzoneCluster.Builder
*/
public Builder setHbProcessorInterval (int val) {
hbProcessorInterval = Optional.of(val);
return this;
}
/**
* When set to true, enables trace level logging.
*
* @param trace true or false
*
* @return MiniOzoneCluster.Builder
*/
public Builder setTrace(Boolean trace) {
enableTrace = Optional.of(trace);
return this;
}
/**
* Modifies the configuration such that Ozone will be disabled.
*
* @return MiniOzoneCluster.Builder
*/
public Builder disableOzone() {
ozoneEnabled = false;
return this;
}
/**
* Constructs and returns MiniOzoneCluster.
*
* @return {@link MiniOzoneCluster}
*
* @throws IOException
*/
public abstract MiniOzoneCluster build() throws IOException;
}
}

View File

@ -0,0 +1,425 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone;
import java.io.File;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ipc.Client;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.ozone.client.OzoneClient;
import org.apache.hadoop.ozone.client.OzoneClientFactory;
import org.apache.hadoop.ozone.client.rest.OzoneException;
import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
import org.apache.hadoop.ozone.ksm.KeySpaceManager;
import org.apache.hadoop.hdds.scm.SCMStorage;
import org.apache.hadoop.ozone.ksm.KSMStorage;
import org.apache.hadoop.ozone.web.client.OzoneRestClient;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.protocolPB
.StorageContainerLocationProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
import org.apache.hadoop.hdds.scm.StorageContainerManager;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.test.GenericTestUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.slf4j.event.Level;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState
.HEALTHY;
import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_PLUGINS_KEY;
/**
* MiniOzoneCluster creates a complete in-process Ozone cluster suitable for
* running tests. The cluster consists of a KeySpaceManager,
* StorageContainerManager and multiple DataNodes.
*/
@InterfaceAudience.Private
public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
private static final Logger LOG =
LoggerFactory.getLogger(MiniOzoneClusterImpl.class);
private final OzoneConfiguration conf;
private final StorageContainerManager scm;
private final KeySpaceManager ksm;
private final List<HddsDatanodeService> hddsDatanodes;
/**
* Creates a new MiniOzoneCluster.
*
* @throws IOException if there is an I/O error
*/
private MiniOzoneClusterImpl(OzoneConfiguration conf,
KeySpaceManager ksm,
StorageContainerManager scm,
List<HddsDatanodeService> hddsDatanodes) {
this.conf = conf;
this.ksm = ksm;
this.scm = scm;
this.hddsDatanodes = hddsDatanodes;
}
public OzoneConfiguration getConf() {
return conf;
}
/**
* Waits for the Ozone cluster to be ready for processing requests.
*/
@Override
public void waitForClusterToBeReady()
throws TimeoutException, InterruptedException {
GenericTestUtils.waitFor(() -> {
final int healthy = scm.getNodeCount(HEALTHY);
final boolean isReady = healthy == hddsDatanodes.size();
LOG.info("{}. Got {} of {} DN Heartbeats.",
isReady? "Cluster is ready" : "Waiting for cluster to be ready",
healthy, hddsDatanodes.size());
return isReady;
}, 1000, 60 * 1000); //wait for 1 min.
}
/**
* Waits for SCM to be out of Chill Mode. Many tests can be run iff we are out
* of Chill mode.
*
* @throws TimeoutException
* @throws InterruptedException
*/
@Override
public void waitTobeOutOfChillMode()
throws TimeoutException, InterruptedException {
GenericTestUtils.waitFor(() -> {
if (scm.getScmNodeManager().isOutOfChillMode()) {
return true;
}
LOG.info("Waiting for cluster to be ready. No datanodes found");
return false;
}, 100, 45000);
}
@Override
public StorageContainerManager getStorageContainerManager() {
return this.scm;
}
@Override
public KeySpaceManager getKeySpaceManager() {
return this.ksm;
}
@Override
public List<HddsDatanodeService> getHddsDatanodes() {
return hddsDatanodes;
}
@Override
public OzoneClient getClient() throws IOException {
return OzoneClientFactory.getClient(conf);
}
@Override
public OzoneClient getRpcClient() throws IOException {
return OzoneClientFactory.getRpcClient(conf);
}
/**
* Creates an {@link OzoneRestClient} connected to this cluster's REST
* service. Callers take ownership of the client and must close it when done.
*
* @return OzoneRestClient connected to this cluster's REST service
* @throws OzoneException if Ozone encounters an error creating the client
*/
@Override
public OzoneClient getRestClient() throws IOException {
return OzoneClientFactory.getRestClient(conf);
}
/**
* Returns an RPC proxy connected to this cluster's StorageContainerManager
* for accessing container location information. Callers take ownership of
* the proxy and must close it when done.
*
* @return RPC proxy for accessing container location information
* @throws IOException if there is an I/O error
*/
@Override
public StorageContainerLocationProtocolClientSideTranslatorPB
getStorageContainerLocationClient() throws IOException {
long version = RPC.getProtocolVersion(
StorageContainerLocationProtocolPB.class);
InetSocketAddress address = scm.getClientRpcAddress();
LOG.info(
"Creating StorageContainerLocationProtocol RPC client with address {}",
address);
return new StorageContainerLocationProtocolClientSideTranslatorPB(
RPC.getProxy(StorageContainerLocationProtocolPB.class, version,
address, UserGroupInformation.getCurrentUser(), conf,
NetUtils.getDefaultSocketFactory(conf),
Client.getRpcTimeout(conf)));
}
@Override
public void restartStorageContainerManager() throws IOException {
scm.stop();
scm.start();
}
@Override
public void restartKeySpaceManager() throws IOException {
ksm.stop();
ksm.start();
}
@Override
public void restartHddsDatanode(int i) {
HddsDatanodeService datanodeService = hddsDatanodes.get(i);
datanodeService.stop();
datanodeService.join();
datanodeService.start(null);
}
@Override
public void shutdownHddsDatanode(int i) {
hddsDatanodes.get(i).stop();
}
@Override
public void shutdown() {
try {
LOG.info("Shutting down the Mini Ozone Cluster");
File baseDir = new File(GenericTestUtils.getTempPath(
MiniOzoneClusterImpl.class.getSimpleName() + "-" +
scm.getScmInfo().getClusterId()));
FileUtils.deleteDirectory(baseDir);
if (ksm != null) {
LOG.info("Shutting down the keySpaceManager");
ksm.stop();
ksm.join();
}
if (scm != null) {
LOG.info("Shutting down the StorageContainerManager");
scm.stop();
scm.join();
}
if (!hddsDatanodes.isEmpty()) {
LOG.info("Shutting down the HddsDatanodes");
for (HddsDatanodeService hddsDatanode : hddsDatanodes) {
hddsDatanode.stop();
hddsDatanode.join();
}
}
} catch (IOException e) {
LOG.error("Exception while shutting down the cluster.", e);
}
}
/**
* Builder for configuring the MiniOzoneCluster to run.
*/
public static class Builder extends MiniOzoneCluster.Builder {
/**
* Creates a new Builder.
*
* @param conf configuration
*/
public Builder(OzoneConfiguration conf) {
super(conf);
}
@Override
public MiniOzoneCluster build() throws IOException {
DefaultMetricsSystem.setMiniClusterMode(true);
initializeConfiguration();
StorageContainerManager scm = createSCM();
scm.start();
KeySpaceManager ksm = createKSM();
ksm.start();
List<HddsDatanodeService> hddsDatanodes = createHddsDatanodes(scm);
hddsDatanodes.forEach((datanode) -> datanode.start(null));
return new MiniOzoneClusterImpl(conf, ksm, scm, hddsDatanodes);
}
/**
* Initializes the configureation required for starting MiniOzoneCluster.
*
* @throws IOException
*/
private void initializeConfiguration() throws IOException {
conf.setBoolean(OzoneConfigKeys.OZONE_ENABLED, ozoneEnabled);
Path metaDir = Paths.get(path, "ozone-meta");
Files.createDirectories(metaDir);
conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, metaDir.toString());
configureTrace();
}
/**
* Creates a new StorageContainerManager instance.
*
* @return {@link StorageContainerManager}
*
* @throws IOException
*/
private StorageContainerManager createSCM() throws IOException {
configureSCM();
SCMStorage scmStore = new SCMStorage(conf);
scmStore.setClusterId(clusterId);
if (!scmId.isPresent()) {
scmId = Optional.of(UUID.randomUUID().toString());
}
scmStore.setScmId(scmId.get());
scmStore.initialize();
return StorageContainerManager.createSCM(null, conf);
}
/**
* Creates a new KeySpaceManager instance.
*
* @return {@link KeySpaceManager}
*
* @throws IOException
*/
private KeySpaceManager createKSM() throws IOException {
configureKSM();
KSMStorage ksmStore = new KSMStorage(conf);
ksmStore.setClusterId(clusterId);
ksmStore.setScmId(scmId.get());
ksmStore.setKsmId(ksmId.orElse(UUID.randomUUID().toString()));
ksmStore.initialize();
return KeySpaceManager.createKSM(null, conf);
}
/**
* Creates HddsDatanodeService(s) instance.
*
* @return List of HddsDatanodeService
*
* @throws IOException
*/
private List<HddsDatanodeService> createHddsDatanodes(
StorageContainerManager scm) throws IOException {
configureHddsDatanodes();
String scmAddress = scm.getDatanodeRpcAddress().getHostString() +
":" + scm.getDatanodeRpcAddress().getPort();
conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, scmAddress);
List<HddsDatanodeService> hddsDatanodes = new ArrayList<>();
for (int i = 0; i < numOfDatanodes; i++) {
Configuration dnConf = new OzoneConfiguration(conf);
String datanodeBaseDir = path + "/datanode-" + Integer.toString(i);
Path metaDir = Paths.get(datanodeBaseDir, "meta");
Path dataDir = Paths.get(datanodeBaseDir, "data", "containers");
Path ratisDir = Paths.get(datanodeBaseDir, "data", "ratis");
Files.createDirectories(metaDir);
Files.createDirectories(dataDir);
Files.createDirectories(ratisDir);
dnConf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, metaDir.toString());
dnConf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dataDir.toString());
dnConf.set(OzoneConfigKeys.DFS_CONTAINER_RATIS_DATANODE_STORAGE_DIR,
ratisDir.toString());
hddsDatanodes.add(
HddsDatanodeService.createHddsDatanodeService(dnConf));
}
return hddsDatanodes;
}
private void configureSCM() {
conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "127.0.0.1:0");
conf.set(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY, "127.0.0.1:0");
conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "127.0.0.1:0");
conf.set(ScmConfigKeys.OZONE_SCM_HTTP_ADDRESS_KEY, "127.0.0.1:0");
conf.setInt(ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_KEY, numOfScmHandlers);
configureSCMheartbeat();
}
private void configureSCMheartbeat() {
if (hbInterval.isPresent()) {
conf.getTimeDuration(ScmConfigKeys.OZONE_SCM_HEARTBEAT_INTERVAL,
hbInterval.get(), TimeUnit.MILLISECONDS);
} else {
conf.setTimeDuration(ScmConfigKeys.OZONE_SCM_HEARTBEAT_INTERVAL,
DEFAULT_HB_INTERVAL_MS,
TimeUnit.MILLISECONDS);
}
if (hbProcessorInterval.isPresent()) {
conf.setTimeDuration(
ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL,
hbProcessorInterval.get(),
TimeUnit.MILLISECONDS);
} else {
conf.setTimeDuration(
ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL,
DEFAULT_HB_PROCESSOR_INTERVAL_MS,
TimeUnit.MILLISECONDS);
}
}
private void configureKSM() {
conf.set(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY, "127.0.0.1:0");
conf.set(KSMConfigKeys.OZONE_KSM_HTTP_ADDRESS_KEY, "127.0.0.1:0");
conf.setInt(KSMConfigKeys.OZONE_KSM_HANDLER_COUNT_KEY, numOfKsmHandlers);
}
private void configureHddsDatanodes() {
conf.set(ScmConfigKeys.HDDS_REST_HTTP_ADDRESS_KEY, "0.0.0.0:0");
conf.set(HDDS_DATANODE_PLUGINS_KEY,
"org.apache.hadoop.ozone.web.OzoneHddsDatanodeService");
conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT,
randomContainerPort);
conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_RANDOM_PORT,
randomContainerPort);
}
private void configureTrace() {
if (enableTrace.isPresent()) {
conf.setBoolean(OzoneConfigKeys.OZONE_TRACE_ENABLED_KEY,
enableTrace.get());
GenericTestUtils.setRootLogLevel(Level.TRACE);
}
GenericTestUtils.setRootLogLevel(Level.INFO);
}
}
}

View File

@ -1,81 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
import org.apache.hadoop.ozone.container.common.statemachine
.DatanodeStateMachine;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
import org.apache.hadoop.util.ServicePlugin;
import java.lang.reflect.Field;
import java.util.List;
/**
* Stateless helper functions for MiniOzone based tests.
*/
public class MiniOzoneTestHelper {
private MiniOzoneTestHelper() {
}
public static DatanodeDetails getDatanodeDetails(DataNode dataNode) {
return findHddsPlugin(dataNode).getDatanodeDetails();
}
public static int getOzoneRestPort(DataNode dataNode) {
return MiniOzoneTestHelper.getDatanodeDetails(dataNode).getOzoneRestPort();
}
public static OzoneContainer getOzoneContainer(DataNode dataNode) {
return findHddsPlugin(dataNode).getDatanodeStateMachine()
.getContainer();
}
public static ContainerManager getOzoneContainerManager(DataNode dataNode) {
return findHddsPlugin(dataNode).getDatanodeStateMachine()
.getContainer().getContainerManager();
}
public static DatanodeStateMachine getStateMachine(DataNode dataNode) {
return findHddsPlugin(dataNode).getDatanodeStateMachine();
}
private static HddsDatanodeService findHddsPlugin(DataNode dataNode) {
try {
Field pluginsField = DataNode.class.getDeclaredField("plugins");
pluginsField.setAccessible(true);
List<ServicePlugin> plugins =
(List<ServicePlugin>) pluginsField.get(dataNode);
for (ServicePlugin plugin : plugins) {
if (plugin instanceof HddsDatanodeService) {
return (HddsDatanodeService) plugin;
}
}
} catch (NoSuchFieldException | IllegalAccessException e) {
e.printStackTrace();
}
throw new IllegalStateException("Can't find the Hdds server plugin in the"
+ " plugin collection of datanode");
}
}

View File

@ -44,10 +44,10 @@ public interface RatisTestHelper {
static final int NUM_DATANODES = 3;
private final OzoneConfiguration conf;
private final MiniOzoneClassicCluster cluster;
private final MiniOzoneCluster cluster;
/**
* Create a {@link MiniOzoneClassicCluster} for testing by setting
* Create a {@link MiniOzoneCluster} for testing by setting
* OZONE_ENABLED = true,
* RATIS_ENABLED = true, and
* OZONE_HANDLER_TYPE_KEY = "distributed".
@ -61,14 +61,10 @@ public interface RatisTestHelper {
return conf;
}
public MiniOzoneClassicCluster getCluster() {
public MiniOzoneCluster getCluster() {
return cluster;
}
public int getDatanodeInfoPort() {
return cluster.getDataNodes().get(0).getInfoPort();
}
public OzoneRestClient newOzoneRestClient()
throws OzoneException, URISyntaxException {
return RatisTestHelper.newOzoneRestClient(getDatanodeOzoneRestPort());
@ -76,12 +72,12 @@ public interface RatisTestHelper {
@Override
public void close() {
cluster.close();
cluster.shutdown();
}
public int getDatanodeOzoneRestPort() {
return MiniOzoneTestHelper.getOzoneRestPort(
cluster.getDataNodes().get(0));
return cluster.getHddsDatanodes().get(0).getDatanodeDetails()
.getOzoneRestPort();
}
}
@ -100,12 +96,10 @@ public interface RatisTestHelper {
+ " = " + rpc.name());
}
static MiniOzoneClassicCluster newMiniOzoneCluster(
static MiniOzoneCluster newMiniOzoneCluster(
int numDatanodes, OzoneConfiguration conf) throws IOException {
final MiniOzoneClassicCluster cluster =
new MiniOzoneClassicCluster.Builder(conf)
.numDataNodes(numDatanodes)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
final MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(numDatanodes).build();
return cluster;
}

View File

@ -49,22 +49,19 @@ public class TestContainerOperations {
@BeforeClass
public static void setup() throws Exception {
int containerSizeGB = 5;
long datanodeCapacities = 3 * OzoneConsts.TB;
ContainerOperationClient.setContainerSizeB(
containerSizeGB * OzoneConsts.GB);
ozoneConf = new OzoneConfiguration();
ozoneConf.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY,
SCMContainerPlacementCapacity.class, ContainerPlacementPolicy.class);
cluster = new MiniOzoneClassicCluster.Builder(ozoneConf).numDataNodes(1)
.storageCapacities(new long[] {datanodeCapacities, datanodeCapacities})
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
cluster = MiniOzoneCluster.newBuilder(ozoneConf).setNumDatanodes(1).build();
StorageContainerLocationProtocolClientSideTranslatorPB client =
cluster.createStorageContainerLocationClient();
cluster.getStorageContainerLocationClient();
RPC.setProtocolEngine(ozoneConf, StorageContainerLocationProtocolPB.class,
ProtobufRpcEngine.class);
storageClient = new ContainerOperationClient(
client, new XceiverClientManager(ozoneConf));
cluster.waitForHeartbeatProcessed();
cluster.waitForClusterToBeReady();
}
@AfterClass

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.ozone;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
@ -34,7 +33,6 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineChannel;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.test.PathUtils;
import org.apache.hadoop.test.TestGenericTestUtils;
import org.apache.hadoop.util.ServicePlugin;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
@ -56,7 +54,7 @@ import static org.junit.Assert.*;
*/
public class TestMiniOzoneCluster {
private static MiniOzoneClassicCluster cluster;
private static MiniOzoneCluster cluster;
private static OzoneConfiguration conf;
private final static File TEST_ROOT = TestGenericTestUtils.getTestDir();
@ -79,24 +77,22 @@ public class TestMiniOzoneCluster {
public static void cleanup() {
if (cluster != null) {
cluster.shutdown();
cluster.close();
}
}
@Test(timeout = 30000)
public void testStartMultipleDatanodes() throws Exception {
final int numberOfNodes = 3;
cluster = new MiniOzoneClassicCluster.Builder(conf)
.numDataNodes(numberOfNodes)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED)
cluster = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(numberOfNodes)
.build();
List<DataNode> datanodes = cluster.getDataNodes();
cluster.waitForClusterToBeReady();
List<HddsDatanodeService> datanodes = cluster.getHddsDatanodes();
assertEquals(numberOfNodes, datanodes.size());
for(DataNode dn : datanodes) {
for(HddsDatanodeService dn : datanodes) {
// Create a single member pipe line
String containerName = OzoneUtils.getRequestID();
DatanodeDetails datanodeDetails =
MiniOzoneTestHelper.getDatanodeDetails(dn);
DatanodeDetails datanodeDetails = dn.getDatanodeDetails();
final PipelineChannel pipelineChannel =
new PipelineChannel(datanodeDetails.getUuidString(),
HddsProtos.LifeCycleState.OPEN,
@ -133,15 +129,6 @@ public class TestMiniOzoneCluster {
assertEquals(id1, validId);
assertEquals(id1.getProtoBufMessage(), validId.getProtoBufMessage());
// Write should fail if unable to create file or directory
File invalidPath = new File(WRITE_TMP, "an/invalid/path");
try {
ContainerUtils.writeDatanodeDetailsTo(id1, invalidPath);
Assert.fail();
} catch (Exception e) {
assertTrue(e instanceof IOException);
}
// Read should return an empty value if file doesn't exist
File nonExistFile = new File(READ_TMP, "non_exist.id");
nonExistFile.delete();

View File

@ -37,7 +37,6 @@ import org.apache.hadoop.hdds.scm.StorageContainerManager.StartupOption;
import org.apache.hadoop.hdds.scm.block.DeletedBlockLog;
import org.apache.hadoop.hdds.scm.block.SCMBlockDeletingService;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.ScmInfo;
@ -62,7 +61,6 @@ import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.io.IOUtils;
import org.junit.rules.Timeout;
import org.mockito.Mockito;
import org.apache.hadoop.test.GenericTestUtils;
@ -87,7 +85,7 @@ public class TestStorageContainerManager {
public ExpectedException exception = ExpectedException.none();
@Test
public void testRpcPermission() throws IOException {
public void testRpcPermission() throws Exception {
// Test with default configuration
OzoneConfiguration defaultConf = new OzoneConfiguration();
testRpcPermissionWithConf(defaultConf, "unknownUser", true);
@ -104,11 +102,9 @@ public class TestStorageContainerManager {
private void testRpcPermissionWithConf(
OzoneConfiguration ozoneConf, String fakeRemoteUsername,
boolean expectPermissionDenied) throws IOException {
MiniOzoneCluster cluster =
new MiniOzoneClassicCluster.Builder(ozoneConf).numDataNodes(1)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
boolean expectPermissionDenied) throws Exception {
MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(ozoneConf).build();
cluster.waitForClusterToBeReady();
try {
String fakeUser = fakeRemoteUsername;
StorageContainerManager mockScm = Mockito.spy(
@ -172,7 +168,7 @@ public class TestStorageContainerManager {
}
}
} finally {
IOUtils.cleanupWithLogger(null, cluster);
cluster.shutdown();
}
}
@ -201,9 +197,8 @@ public class TestStorageContainerManager {
conf.setInt(ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE,
numKeys);
MiniOzoneClassicCluster cluster =
new MiniOzoneClassicCluster.Builder(conf).numDataNodes(1)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf).build();
cluster.waitForClusterToBeReady();
try {
DeletedBlockLog delLog = cluster.getStorageContainerManager()
@ -269,19 +264,17 @@ public class TestStorageContainerManager {
public void testBlockDeletingThrottling() throws Exception {
int numKeys = 15;
OzoneConfiguration conf = new OzoneConfiguration();
conf.setTimeDuration(ScmConfigKeys.OZONE_SCM_HEARTBEAT_INTERVAL, 5,
TimeUnit.SECONDS);
conf.setTimeDuration(ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL,
3000, TimeUnit.MILLISECONDS);
conf.setInt(ScmConfigKeys.OZONE_SCM_BLOCK_DELETION_MAX_RETRY, 5);
conf.setTimeDuration(OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL,
1000, TimeUnit.MILLISECONDS);
conf.setInt(ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE,
numKeys);
MiniOzoneClassicCluster cluster = new MiniOzoneClassicCluster.Builder(conf)
.numDataNodes(1).setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED)
MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf)
.setHbInterval(5000)
.setHbProcessorInterval(3000)
.build();
cluster.waitForClusterToBeReady();
DeletedBlockLog delLog = cluster.getStorageContainerManager()
.getScmBlockManager().getDeletedBlockLog();
@ -402,14 +395,15 @@ public class TestStorageContainerManager {
conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, scmPath.toString());
//This will set the cluster id in the version file
MiniOzoneCluster cluster =
new MiniOzoneClassicCluster.Builder(conf).numDataNodes(1)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
cluster.waitForClusterToBeReady();
StartupOption.INIT.setClusterId("testClusterId");
// This will initialize SCM
StorageContainerManager.scmInit(conf);
SCMStorage scmStore = new SCMStorage(conf);
Assert.assertEquals(NodeType.SCM, scmStore.getNodeType());
Assert.assertNotEquals("testClusterId", scmStore.getClusterID());
cluster.shutdown();
}
@Test

View File

@ -23,7 +23,6 @@ import org.apache.commons.lang.RandomStringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
@ -55,11 +54,11 @@ import java.util.Set;
*/
public class TestStorageContainerManagerHelper {
private final MiniOzoneClassicCluster cluster;
private final MiniOzoneCluster cluster;
private final Configuration conf;
private final StorageHandler storageHandler;
public TestStorageContainerManagerHelper(MiniOzoneClassicCluster cluster,
public TestStorageContainerManagerHelper(MiniOzoneCluster cluster,
Configuration conf) throws IOException {
this.cluster = cluster;
this.conf = conf;
@ -169,10 +168,9 @@ public class TestStorageContainerManagerHelper {
private OzoneContainer getContainerServerByDatanodeUuid(String dnUUID)
throws IOException {
for (DataNode dn : cluster.getDataNodes()) {
if (MiniOzoneTestHelper.getDatanodeDetails(dn).getUuidString()
.equals(dnUUID)) {
return MiniOzoneTestHelper.getOzoneContainer(dn);
for (HddsDatanodeService dn : cluster.getHddsDatanodes()) {
if (dn.getDatanodeDetails().getUuidString().equals(dnUUID)) {
return dn.getDatanodeStateMachine().getContainer();
}
}
throw new IOException("Unable to get the ozone container "

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.ozone.client.rest;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
@ -58,7 +58,7 @@ public class TestOzoneRestClient {
@Rule
public ExpectedException thrown = ExpectedException.none();
private static MiniOzoneClassicCluster cluster = null;
private static MiniOzoneCluster cluster = null;
private static OzoneClient ozClient = null;
private static ObjectStore store = null;
@ -75,9 +75,8 @@ public class TestOzoneRestClient {
OzoneConfiguration conf = new OzoneConfiguration();
conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
cluster = new MiniOzoneClassicCluster.Builder(conf).numDataNodes(1)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
cluster.waitForClusterToBeReady();
InetSocketAddress ksmHttpAddress = cluster.getKeySpaceManager()
.getHttpServer().getHttpAddress();
ozClient = OzoneClientFactory.getRestClient(ksmHttpAddress.getHostName(),

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.ozone.client.rpc;
import org.apache.commons.lang.RandomStringUtils;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@ -70,7 +70,7 @@ public class TestOzoneRpcClient {
@Rule
public ExpectedException thrown = ExpectedException.none();
private static MiniOzoneClassicCluster cluster = null;
private static MiniOzoneCluster cluster = null;
private static OzoneClient ozClient = null;
private static ObjectStore store = null;
private static KeySpaceManager keySpaceManager;
@ -91,12 +91,12 @@ public class TestOzoneRpcClient {
conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
conf.setInt(ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE, 1);
cluster = new MiniOzoneClassicCluster.Builder(conf).numDataNodes(10)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(10).build();
cluster.waitForClusterToBeReady();
ozClient = OzoneClientFactory.getRpcClient(conf);
store = ozClient.getObjectStore();
storageContainerLocationClient =
cluster.createStorageContainerLocationClient();
cluster.getStorageContainerLocationClient();
keySpaceManager = cluster.getKeySpaceManager();
}

View File

@ -19,9 +19,7 @@ package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneTestHelper;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.client.ObjectStore;
import org.apache.hadoop.ozone.client.OzoneClient;
import org.apache.hadoop.ozone.client.OzoneClientFactory;
@ -55,10 +53,9 @@ public class TestCloseContainerHandler {
//setup a cluster (1G free space is enough for a unit test)
OzoneConfiguration conf = new OzoneConfiguration();
conf.set(OZONE_SCM_CONTAINER_SIZE_GB, "1");
MiniOzoneClassicCluster cluster =
new MiniOzoneClassicCluster.Builder(conf).numDataNodes(1)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
cluster.waitOzoneReady();
MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(1).build();
cluster.waitForClusterToBeReady();
//the easiest way to create an open container is creating a key
OzoneClient client = OzoneClientFactory.getClient(conf);
@ -86,8 +83,8 @@ public class TestCloseContainerHandler {
Assert.assertFalse(isContainerClosed(cluster, containerName));
DatanodeDetails datanodeDetails = MiniOzoneTestHelper
.getDatanodeDetails(cluster.getDataNodes().get(0));
DatanodeDetails datanodeDetails = cluster.getHddsDatanodes().get(0)
.getDatanodeDetails();
//send the order to close the container
cluster.getStorageContainerManager().getScmNodeManager()
.addDatanodeCommand(datanodeDetails.getUuid(),
@ -101,12 +98,13 @@ public class TestCloseContainerHandler {
Assert.assertTrue(isContainerClosed(cluster, containerName));
}
private Boolean isContainerClosed(MiniOzoneClassicCluster cluster,
private Boolean isContainerClosed(MiniOzoneCluster cluster,
String containerName) {
ContainerData containerData;
try {
containerData = MiniOzoneTestHelper.getOzoneContainerManager(cluster
.getDataNodes().get(0)).readContainer(containerName);
containerData = cluster.getHddsDatanodes().get(0)
.getDatanodeStateMachine().getContainer().getContainerManager()
.readContainer(containerName);
return !containerData.isOpen();
} catch (StorageContainerException e) {
throw new AssertionError(e);

View File

@ -19,11 +19,9 @@
package org.apache.hadoop.ozone.container.ozoneimpl;
import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.ContainerTestHelper;
import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
@ -59,8 +57,8 @@ public class TestOzoneContainer {
OzoneContainer container = null;
MiniOzoneCluster cluster = null;
try {
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
cluster = MiniOzoneCluster.newBuilder(conf).build();
cluster.waitForClusterToBeReady();
// We don't start Ozone Container via data node, we will do it
// independently in our test path.
Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline(
@ -105,9 +103,10 @@ public class TestOzoneContainer {
conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
pipeline.getLeader().getContainerPort());
cluster = new MiniOzoneClassicCluster.Builder(conf)
cluster = MiniOzoneCluster.newBuilder(conf)
.setRandomContainerPort(false)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
.build();
cluster.waitForClusterToBeReady();
// This client talks to ozone container via datanode.
XceiverClient client = new XceiverClient(pipeline, conf);
@ -208,9 +207,10 @@ public class TestOzoneContainer {
OzoneConfiguration conf = newOzoneConfiguration();
client = createClientForTesting(conf);
cluster = new MiniOzoneClassicCluster.Builder(conf)
cluster = MiniOzoneCluster.newBuilder(conf)
.setRandomContainerPort(false)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
.build();
cluster.waitForClusterToBeReady();
String containerName = client.getPipeline().getContainerName();
runTestBothGetandPutSmallFile(containerName, client);
@ -266,9 +266,10 @@ public class TestOzoneContainer {
OzoneConfiguration conf = newOzoneConfiguration();
client = createClientForTesting(conf);
cluster = new MiniOzoneClassicCluster.Builder(conf)
cluster = MiniOzoneCluster.newBuilder(conf)
.setRandomContainerPort(false)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
.build();
cluster.waitForClusterToBeReady();
client.connect();
String containerName = client.getPipeline().getContainerName();
@ -356,9 +357,10 @@ public class TestOzoneContainer {
OzoneConfiguration conf = newOzoneConfiguration();
client = createClientForTesting(conf);
cluster = new MiniOzoneClassicCluster.Builder(conf)
cluster = MiniOzoneCluster.newBuilder(conf)
.setRandomContainerPort(false)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
.build();
cluster.waitForClusterToBeReady();
client.connect();
String containerName = client.getPipeline().getContainerName();
@ -471,9 +473,10 @@ public class TestOzoneContainer {
OzoneConfiguration conf = newOzoneConfiguration();
client = createClientForTesting(conf);
cluster = new MiniOzoneClassicCluster.Builder(conf)
cluster = MiniOzoneCluster.newBuilder(conf)
.setRandomContainerPort(false)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
.build();
cluster.waitForClusterToBeReady();
String containerName = client.getPipeline().getContainerName();
runAsyncTests(containerName, client);
} finally {
@ -492,9 +495,10 @@ public class TestOzoneContainer {
OzoneConfiguration conf = newOzoneConfiguration();
client = createClientForTesting(conf);
cluster = new MiniOzoneClassicCluster.Builder(conf)
cluster = MiniOzoneCluster.newBuilder(conf)
.setRandomContainerPort(false)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
.build();
cluster.waitForClusterToBeReady();
client.connect();
// Send a request without traceId.

View File

@ -18,11 +18,9 @@
package org.apache.hadoop.ozone.container.ozoneimpl;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.HddsDatanodeService;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.MiniOzoneTestHelper;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.RatisTestHelper;
import org.apache.hadoop.ozone.container.ContainerTestHelper;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
@ -76,20 +74,19 @@ public class TestOzoneContainerRatis {
// create Ozone clusters
final OzoneConfiguration conf = newOzoneConfiguration();
RatisTestHelper.initRatisConf(rpc, conf);
final MiniOzoneClassicCluster cluster =
new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_LOCAL)
.numDataNodes(numNodes)
final MiniOzoneCluster cluster =
MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(numNodes)
.build();
try {
cluster.waitOzoneReady();
cluster.waitForClusterToBeReady();
final String containerName = OzoneUtils.getRequestID();
final List<DataNode> datanodes = cluster.getDataNodes();
final List<HddsDatanodeService> datanodes = cluster.getHddsDatanodes();
final Pipeline pipeline = ContainerTestHelper.createPipeline(
containerName,
CollectionUtils.as(datanodes,
MiniOzoneTestHelper::getDatanodeDetails));
HddsDatanodeService::getDatanodeDetails));
LOG.info("pipeline=" + pipeline);
// Create Ratis cluster

View File

@ -18,12 +18,10 @@
package org.apache.hadoop.ozone.container.ozoneimpl;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.HddsDatanodeService;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.MiniOzoneTestHelper;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.RatisTestHelper;
import org.apache.hadoop.ozone.container.ContainerTestHelper;
import org.apache.ratis.rpc.RpcType;
@ -75,17 +73,15 @@ public class TestRatisManager {
// create Ozone clusters
final OzoneConfiguration conf = newOzoneConfiguration();
RatisTestHelper.initRatisConf(rpc, conf);
final MiniOzoneClassicCluster cluster =
new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_LOCAL)
.numDataNodes(5)
final MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(5)
.build();
try {
cluster.waitOzoneReady();
cluster.waitForClusterToBeReady();
final List<DataNode> datanodes = cluster.getDataNodes();
final List<HddsDatanodeService> datanodes = cluster.getHddsDatanodes();
final List<DatanodeDetails> datanodeDetailsSet = datanodes.stream()
.map(MiniOzoneTestHelper::getDatanodeDetails).collect(
.map(HddsDatanodeService::getDatanodeDetails).collect(
Collectors.toList());
//final RatisManager manager = RatisManager.newRatisManager(conf);

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.ozone.freon;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
@ -56,9 +55,9 @@ public class TestDataValidate {
conf = new OzoneConfiguration();
conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED)
.numDataNodes(5).build();
cluster = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(5).build();
cluster.waitForClusterToBeReady();
}
/**

View File

@ -18,11 +18,8 @@
package org.apache.hadoop.ozone.freon;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.util.ToolRunner;
import org.junit.AfterClass;
import org.junit.Assert;
@ -52,11 +49,8 @@ public class TestFreon {
@BeforeClass
public static void init() throws Exception {
conf = new OzoneConfiguration();
conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED)
.numDataNodes(5).build();
cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(5).build();
cluster.waitForClusterToBeReady();
}
/**

View File

@ -22,8 +22,7 @@ import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneTestHelper;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.client.*;
@ -51,7 +50,7 @@ import java.io.IOException;
public class TestContainerReportWithKeys {
private static final Logger LOG = LoggerFactory.getLogger(
TestContainerReportWithKeys.class);
private static MiniOzoneClassicCluster cluster = null;
private static MiniOzoneCluster cluster = null;
private static OzoneConfiguration conf;
private static StorageContainerManager scm;
@ -71,8 +70,8 @@ public class TestContainerReportWithKeys {
conf = new OzoneConfiguration();
conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
cluster = MiniOzoneCluster.newBuilder(conf).build();
cluster.waitForClusterToBeReady();
scm = cluster.getStorageContainerManager();
}
@ -117,7 +116,7 @@ public class TestContainerReportWithKeys {
cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions()
.get(0).getBlocksLatestVersionOnly().get(0);
ContainerData cd = getContainerData(cluster, keyInfo.getContainerName());
ContainerData cd = getContainerData(keyInfo.getContainerName());
LOG.info("DN Container Data: keyCount: {} used: {} ",
cd.getKeyCount(), cd.getBytesUsed());
@ -129,12 +128,11 @@ public class TestContainerReportWithKeys {
}
private static ContainerData getContainerData(MiniOzoneClassicCluster clus,
String containerName) {
ContainerData containerData = null;
private static ContainerData getContainerData(String containerName) {
ContainerData containerData;
try {
ContainerManager containerManager = MiniOzoneTestHelper
.getOzoneContainerManager(clus.getDataNodes().get(0));
ContainerManager containerManager = cluster.getHddsDatanodes().get(0)
.getDatanodeStateMachine().getContainer().getContainerManager();
containerData = containerManager.readContainer(containerName);
} catch (StorageContainerException e) {
throw new AssertionError(e);

View File

@ -22,7 +22,6 @@ import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
import java.io.IOException;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@ -55,8 +54,8 @@ public class TestKSMMetrcis {
OzoneConfiguration conf = new OzoneConfiguration();
conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
cluster = MiniOzoneCluster.newBuilder(conf).build();
cluster.waitForClusterToBeReady();
ksmManager = cluster.getKeySpaceManager();
}

View File

@ -17,7 +17,7 @@
package org.apache.hadoop.ozone.ksm;
import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
@ -28,6 +28,8 @@ import org.apache.hadoop.ozone.web.handlers.UserArgs;
import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
@ -47,6 +49,7 @@ import java.util.Collection;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.UUID;
import static org.apache.hadoop.ozone.OzoneConsts.KSM_DB_NAME;
import static org.junit.Assert.assertEquals;
@ -59,7 +62,7 @@ import static org.junit.Assert.assertTrue;
*/
@RunWith(Parameterized.class)
public class TestKSMSQLCli {
private MiniOzoneClassicCluster cluster = null;
private MiniOzoneCluster cluster = null;
private StorageHandler storageHandler;
private UserArgs userArgs;
private OzoneConfiguration conf;
@ -104,12 +107,12 @@ public class TestKSMSQLCli {
conf = new OzoneConfiguration();
conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
cluster = MiniOzoneCluster.newBuilder(conf).build();
cluster.waitForClusterToBeReady();
storageHandler = new ObjectStoreHandler(conf).getStorageHandler();
userArgs = new UserArgs(null, OzoneUtils.getRequestID(),
null, null, null, null);
cluster.waitForHeartbeatProcessed();
cluster.waitForClusterToBeReady();
VolumeArgs createVolumeArgs0 = new VolumeArgs(volumeName0, userArgs);
createVolumeArgs0.setUserName(userName);
@ -149,15 +152,23 @@ public class TestKSMSQLCli {
stream = storageHandler.newKeyWriter(keyArgs3);
stream.close();
cluster.shutdown();
cluster.getKeySpaceManager().stop();
cluster.getStorageContainerManager().stop();
conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, metaStoreType);
cli = new SQLCLI(conf);
}
@After
public void shutdown() {
if (cluster != null) {
cluster.shutdown();
}
}
@Test
public void testKSMDB() throws Exception {
String dbOutPath = cluster.getDataDirectory() + "/out_sql.db";
String dbOutPath = GenericTestUtils.getTempPath(
UUID.randomUUID() + "/out_sql.db");
String dbRootPath = conf.get(OzoneConfigKeys.OZONE_METADATA_DIRS);
String dbPath = dbRootPath + "/" + KSM_DB_NAME;

View File

@ -22,7 +22,6 @@ import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@ -123,12 +122,12 @@ public class TestKeySpaceManager {
conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
conf.setInt(OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS, 2);
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED)
cluster = MiniOzoneCluster.newBuilder(conf)
.setClusterId(clusterId)
.setScmId(scmId)
.setKsmId(ksmId)
.build();
cluster.waitForClusterToBeReady();
storageHandler = new ObjectStoreHandler(conf).getStorageHandler();
userArgs = new UserArgs(null, OzoneUtils.getRequestID(),
null, null, null, null);

View File

@ -21,12 +21,9 @@ package org.apache.hadoop.ozone.ksm;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.core.type.TypeReference;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.MiniOzoneTestHelper;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.ServicePort;
@ -45,7 +42,6 @@ import java.net.InetSocketAddress;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
import static org.apache.hadoop.ozone.KsmUtils.getKsmAddressForClients;
@ -61,17 +57,14 @@ public class TestKeySpaceManagerRestInterface {
@BeforeClass
public static void setUp() throws Exception {
conf = new OzoneConfiguration();
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED)
.setClusterId(UUID.randomUUID().toString())
.setScmId(UUID.randomUUID().toString())
.build();
cluster = MiniOzoneCluster.newBuilder(conf).build();
cluster.waitForClusterToBeReady();
}
@AfterClass
public static void tearDown() throws Exception {
if (cluster != null) {
cluster.close();
cluster.shutdown();
}
}
@ -115,9 +108,9 @@ public class TestKeySpaceManagerRestInterface {
scmInfo.getPort(ServicePort.Type.RPC));
ServiceInfo datanodeInfo = serviceMap.get(HddsProtos.NodeType.DATANODE);
DataNode datanode = ((MiniOzoneClassicCluster) cluster)
.getDataNodes().get(0);
Assert.assertEquals(datanode.getDatanodeHostname(),
DatanodeDetails datanodeDetails = cluster.getHddsDatanodes().get(0)
.getDatanodeDetails();
Assert.assertEquals(datanodeDetails.getHostName(),
datanodeInfo.getHostname());
Map<ServicePort.Type, Integer> ports = datanodeInfo.getPorts();
@ -125,7 +118,7 @@ public class TestKeySpaceManagerRestInterface {
switch (type) {
case HTTP:
case HTTPS:
Assert.assertEquals(MiniOzoneTestHelper.getOzoneRestPort(datanode),
Assert.assertEquals(datanodeDetails.getOzoneRestPort(),
(int) ports.get(type));
break;
default:

View File

@ -21,7 +21,6 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
@ -77,8 +76,8 @@ public class TestKsmBlockVersioning {
conf = new OzoneConfiguration();
conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
cluster = MiniOzoneCluster.newBuilder(conf).build();
cluster.waitForClusterToBeReady();
storageHandler = new ObjectStoreHandler(conf).getStorageHandler();
userArgs = new UserArgs(null, OzoneUtils.getRequestID(),
null, null, null, null);

View File

@ -20,7 +20,6 @@ import org.apache.commons.lang.RandomStringUtils;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@ -76,8 +75,8 @@ public class TestMultipleContainerReadWrite {
conf.setInt(ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE, 5);
conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
cluster = MiniOzoneCluster.newBuilder(conf).build();
cluster.waitForClusterToBeReady();
storageHandler = new ObjectStoreHandler(conf).getStorageHandler();
userArgs = new UserArgs(null, OzoneUtils.getRequestID(),
null, null, null, null);

View File

@ -26,9 +26,7 @@ import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.PrintStream;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
@ -38,9 +36,7 @@ import java.util.UUID;
import org.apache.commons.lang.RandomStringUtils;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneTestHelper;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.OzoneAcl.OzoneACLRights;
import org.apache.hadoop.ozone.OzoneAcl.OzoneACLType;
@ -82,7 +78,7 @@ public class TestOzoneShell {
private static String url;
private static File baseDir;
private static OzoneConfiguration conf = null;
private static MiniOzoneClassicCluster cluster = null;
private static MiniOzoneCluster cluster = null;
private static OzoneRestClient client = null;
private static Shell shell = null;
@ -95,11 +91,10 @@ public class TestOzoneShell {
* Create a MiniDFSCluster for testing with using distributed Ozone
* handler type.
*
* @throws IOException
* @throws Exception
*/
@BeforeClass
public static void init()
throws IOException, URISyntaxException, OzoneException {
public static void init() throws Exception {
conf = new OzoneConfiguration();
String path = GenericTestUtils.getTempPath(
@ -115,10 +110,10 @@ public class TestOzoneShell {
shell = new Shell();
shell.setConf(conf);
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
DataNode dataNode = cluster.getDataNodes().get(0);
final int port = MiniOzoneTestHelper.getOzoneRestPort(dataNode);
cluster = MiniOzoneCluster.newBuilder(conf).build();
cluster.waitForClusterToBeReady();
final int port = cluster.getHddsDatanodes().get(0).getDatanodeDetails()
.getOzoneRestPort();
url = String.format("http://localhost:%d", port);
client = new OzoneRestClient(String.format("http://localhost:%d", port));
client.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);

View File

@ -19,10 +19,8 @@ package org.apache.hadoop.ozone.scm;
import org.apache.commons.lang.RandomStringUtils;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
@ -51,15 +49,12 @@ public class TestAllocateContainer {
@BeforeClass
public static void init() throws Exception {
long datanodeCapacities = 3 * OzoneConsts.TB;
conf = new OzoneConfiguration();
cluster = new MiniOzoneClassicCluster.Builder(conf).numDataNodes(3)
.storageCapacities(new long[] {datanodeCapacities, datanodeCapacities})
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(3).build();
cluster.waitForClusterToBeReady();
storageContainerLocationClient =
cluster.createStorageContainerLocationClient();
cluster.getStorageContainerLocationClient();
xceiverClientManager = new XceiverClientManager(conf);
cluster.waitForHeartbeatProcessed();
}
@AfterClass
@ -67,7 +62,7 @@ public class TestAllocateContainer {
if(cluster != null) {
cluster.shutdown();
}
IOUtils.cleanupWithLogger(null, storageContainerLocationClient, cluster);
IOUtils.cleanupWithLogger(null, storageContainerLocationClient);
}
@Test

View File

@ -19,22 +19,19 @@ package org.apache.hadoop.ozone.scm;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.block.BlockManagerImpl;
import org.apache.hadoop.hdds.scm.container.ContainerMapping;
import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementPolicy;
import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementCapacity;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
import org.apache.hadoop.ozone.scm.cli.SQLCLI;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@ -52,12 +49,12 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.UUID;
import static org.apache.hadoop.ozone.OzoneConsts.BLOCK_DB;
import static org.apache.hadoop.ozone.OzoneConsts.SCM_CONTAINER_DB;
import static org.apache.hadoop.ozone.OzoneConsts.KB;
import static org.apache.hadoop.ozone.OzoneConsts.NODEPOOL_DB;
//import static org.apache.hadoop.ozone.OzoneConsts.OPEN_CONTAINERS_DB;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
@ -83,10 +80,9 @@ public class TestContainerSQLCli {
private static SQLCLI cli;
private MiniOzoneClassicCluster cluster;
private MiniOzoneCluster cluster;
private OzoneConfiguration conf;
private StorageContainerLocationProtocolClientSideTranslatorPB
storageContainerLocationClient;
private String datanodeIpAddress;
private ContainerMapping mapping;
private NodeManager nodeManager;
@ -105,7 +101,6 @@ public class TestContainerSQLCli {
@Before
public void setup() throws Exception {
long datanodeCapacities = 3 * OzoneConsts.TB;
blockContainerMap = new HashMap<>();
conf = new OzoneConfiguration();
@ -120,13 +115,12 @@ public class TestContainerSQLCli {
factor = HddsProtos.ReplicationFactor.ONE;
type = HddsProtos.ReplicationType.STAND_ALONE;
}
cluster = new MiniOzoneClassicCluster.Builder(conf).numDataNodes(2)
.storageCapacities(new long[] {datanodeCapacities, datanodeCapacities})
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
storageContainerLocationClient =
cluster.createStorageContainerLocationClient();
cluster.waitForHeartbeatProcessed();
cluster.shutdown();
cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(2).build();
cluster.waitForClusterToBeReady();
datanodeIpAddress = cluster.getHddsDatanodes().get(0)
.getDatanodeDetails().getIpAddress();
cluster.getKeySpaceManager().stop();
cluster.getStorageContainerManager().stop();
nodeManager = cluster.getStorageContainerManager().getScmNodeManager();
mapping = new ContainerMapping(conf, nodeManager, 128);
@ -179,12 +173,15 @@ public class TestContainerSQLCli {
@After
public void shutdown() throws InterruptedException {
IOUtils.cleanupWithLogger(null, storageContainerLocationClient, cluster);
if (cluster != null) {
cluster.shutdown();
}
}
@Test
public void testConvertBlockDB() throws Exception {
String dbOutPath = cluster.getDataDirectory() + "/out_sql.db";
String dbOutPath = GenericTestUtils.getTempPath(
UUID.randomUUID() + "/out_sql.db");
String dbRootPath = conf.get(OzoneConfigKeys.OZONE_METADATA_DIRS);
String dbPath = dbRootPath + "/" + BLOCK_DB;
String[] args = {"-p", dbPath, "-o", dbOutPath};
@ -206,7 +203,8 @@ public class TestContainerSQLCli {
@Test
public void testConvertNodepoolDB() throws Exception {
String dbOutPath = cluster.getDataDirectory() + "/out_sql.db";
String dbOutPath = GenericTestUtils.getTempPath(
UUID.randomUUID() + "/out_sql.db");
String dbRootPath = conf.get(OzoneConfigKeys.OZONE_METADATA_DIRS);
String dbPath = dbRootPath + "/" + NODEPOOL_DB;
String[] args = {"-p", dbPath, "-o", dbOutPath};
@ -233,7 +231,8 @@ public class TestContainerSQLCli {
@Test
public void testConvertContainerDB() throws Exception {
String dbOutPath = cluster.getDataDirectory() + "/out_sql.db";
String dbOutPath = GenericTestUtils.getTempPath(
UUID.randomUUID() + "/out_sql.db");
// TODO : the following will fail due to empty Datanode list, need to fix.
//String dnUUID = cluster.getDataNodes().get(0).getUuid();
String dbRootPath = conf.get(OzoneConfigKeys.OZONE_METADATA_DIRS);
@ -275,7 +274,7 @@ public class TestContainerSQLCli {
rs = executeQuery(conn, sql);
int count = 0;
while (rs.next()) {
assertEquals("127.0.0.1", rs.getString("ipAddr"));
assertEquals(datanodeIpAddress, rs.getString("ipAddress"));
//assertEquals(dnUUID, rs.getString("datanodeUUID"));
count += 1;
}

View File

@ -19,10 +19,8 @@ package org.apache.hadoop.ozone.scm;
import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementPolicy;
import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementCapacity;
@ -59,17 +57,15 @@ public class TestContainerSmallFile {
@BeforeClass
public static void init() throws Exception {
long datanodeCapacities = 3 * OzoneConsts.TB;
ozoneConfig = new OzoneConfiguration();
ozoneConfig.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY,
SCMContainerPlacementCapacity.class, ContainerPlacementPolicy.class);
cluster = new MiniOzoneClassicCluster.Builder(ozoneConfig).numDataNodes(1)
.storageCapacities(new long[] {datanodeCapacities, datanodeCapacities})
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
cluster = MiniOzoneCluster.newBuilder(ozoneConfig).setNumDatanodes(1)
.build();
cluster.waitForClusterToBeReady();
storageContainerLocationClient = cluster
.createStorageContainerLocationClient();
.getStorageContainerLocationClient();
xceiverClientManager = new XceiverClientManager(ozoneConfig);
cluster.waitForHeartbeatProcessed();
}
@AfterClass
@ -77,7 +73,7 @@ public class TestContainerSmallFile {
if (cluster != null) {
cluster.shutdown();
}
IOUtils.cleanupWithLogger(null, storageContainerLocationClient, cluster);
IOUtils.cleanupWithLogger(null, storageContainerLocationClient);
}
@Test

View File

@ -20,10 +20,8 @@ package org.apache.hadoop.ozone.scm;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.StorageContainerManager;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.MiniOzoneTestHelper;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@ -61,7 +59,7 @@ import static org.junit.Assert.assertFalse;
public class TestSCMCli {
private static SCMCLI cli;
private static MiniOzoneClassicCluster cluster;
private static MiniOzoneCluster cluster;
private static OzoneConfiguration conf;
private static StorageContainerLocationProtocolClientSideTranslatorPB
storageContainerLocationClient;
@ -82,11 +80,11 @@ public class TestSCMCli {
@BeforeClass
public static void setup() throws Exception {
conf = new OzoneConfiguration();
cluster = new MiniOzoneClassicCluster.Builder(conf).numDataNodes(3)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(3).build();
cluster.waitForClusterToBeReady();
xceiverClientManager = new XceiverClientManager(conf);
storageContainerLocationClient =
cluster.createStorageContainerLocationClient();
cluster.getStorageContainerLocationClient();
containerOperationClient = new ContainerOperationClient(
storageContainerLocationClient, new XceiverClientManager(conf));
outContent = new ByteArrayOutputStream();
@ -116,7 +114,10 @@ public class TestSCMCli {
@AfterClass
public static void shutdown() throws InterruptedException {
IOUtils.cleanupWithLogger(null, storageContainerLocationClient, cluster);
if (cluster != null) {
cluster.shutdown();
}
IOUtils.cleanupWithLogger(null, storageContainerLocationClient);
}
@Test
@ -235,8 +236,8 @@ public class TestSCMCli {
@Test
public void testInfoContainer() throws Exception {
// The cluster has one Datanode server.
DatanodeDetails datanodeDetails = MiniOzoneTestHelper
.getDatanodeDetails(cluster.getDataNodes().get(0));
DatanodeDetails datanodeDetails = cluster.getHddsDatanodes().get(0)
.getDatanodeDetails();
String formatStr =
"Container Name: %s\n" +
"Container State: %s\n" +

View File

@ -21,11 +21,8 @@ package org.apache.hadoop.ozone.scm;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdds.scm.StorageContainerManager;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.hdds.scm.container.placement.metrics.ContainerStat;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.junit.BeforeClass;
@ -65,18 +62,19 @@ public class TestSCMMXBean {
public static void init() throws IOException, TimeoutException,
InterruptedException {
conf = new OzoneConfiguration();
cluster = new MiniOzoneClassicCluster.Builder(conf)
.numDataNodes(numOfDatanodes)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED)
cluster = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(numOfDatanodes)
.build();
cluster.waitOzoneReady();
cluster.waitForClusterToBeReady();
scm = cluster.getStorageContainerManager();
mbs = ManagementFactory.getPlatformMBeanServer();
}
@AfterClass
public static void shutdown() {
IOUtils.cleanupWithLogger(null, cluster);
if (cluster != null) {
cluster.shutdown();
}
}
@Test

View File

@ -28,11 +28,9 @@ import org.apache.commons.codec.digest.DigestUtils;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.StorageContainerManager;
import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneTestHelper;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.helpers.ContainerReport;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
@ -55,7 +53,7 @@ public class TestSCMMetrics {
@Rule
public Timeout testTimeout = new Timeout(90000);
private static MiniOzoneClassicCluster cluster = null;
private static MiniOzoneCluster cluster = null;
@Test
public void testContainerMetrics() throws Exception {
@ -71,9 +69,9 @@ public class TestSCMMetrics {
OzoneConfiguration conf = new OzoneConfiguration();
try {
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED)
.numDataNodes(nodeCount).build();
cluster = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(nodeCount).build();
cluster.waitForClusterToBeReady();
ContainerStat stat = new ContainerStat(size, used, keyCount, readBytes,
writeBytes, readCount, writeCount);
@ -165,17 +163,16 @@ public class TestSCMMetrics {
OzoneConfiguration conf = new OzoneConfiguration();
try {
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED)
.numDataNodes(nodeCount).build();
cluster = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(nodeCount).build();
cluster.waitForClusterToBeReady();
ContainerStat stat = new ContainerStat(size, used, keyCount, readBytes,
writeBytes, readCount, writeCount);
StorageContainerManager scmManager = cluster.getStorageContainerManager();
DataNode dataNode = cluster.getDataNodes().get(0);
String datanodeUuid = MiniOzoneTestHelper.getDatanodeDetails(dataNode)
.getUuidString();
String datanodeUuid = cluster.getHddsDatanodes().get(0)
.getDatanodeDetails().getUuidString();
ContainerReportsRequestProto request = createContainerReport(numReport,
stat, datanodeUuid);
scmManager.sendContainerReport(request);

View File

@ -20,10 +20,8 @@ package org.apache.hadoop.ozone.scm;
import com.google.common.cache.Cache;
import org.apache.commons.lang.RandomStringUtils;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
@ -57,18 +55,22 @@ public class TestXceiverClientManager {
public ExpectedException exception = ExpectedException.none();
@BeforeClass
public static void init() throws IOException {
public static void init() throws Exception {
config = new OzoneConfiguration();
cluster = new MiniOzoneClassicCluster.Builder(config)
.numDataNodes(3)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
cluster = MiniOzoneCluster.newBuilder(config)
.setNumDatanodes(3)
.build();
cluster.waitForClusterToBeReady();
storageContainerLocationClient = cluster
.createStorageContainerLocationClient();
.getStorageContainerLocationClient();
}
@AfterClass
public static void shutdown() {
IOUtils.cleanupWithLogger(null, cluster, storageContainerLocationClient);
if (cluster != null) {
cluster.shutdown();
}
IOUtils.cleanupWithLogger(null, storageContainerLocationClient);
}
@Test

View File

@ -21,7 +21,6 @@ import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
import static org.apache.hadoop.test.MetricsAsserts.getLongCounter;
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
import java.io.IOException;
import java.util.List;
import java.util.ArrayList;
import java.util.concurrent.CompletableFuture;
@ -32,10 +31,8 @@ import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.ContainerCommandRequestProto;
import org.apache.hadoop.hdds.protocol.proto.ContainerProtos.ContainerCommandResponseProto;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.ContainerTestHelper;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
@ -63,13 +60,12 @@ public class TestXceiverClientMetrics {
private static String containerOwner = "OZONE";
@BeforeClass
public static void init() throws IOException {
public static void init() throws Exception {
config = new OzoneConfiguration();
cluster = new MiniOzoneClassicCluster.Builder(config)
.numDataNodes(1)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
cluster = MiniOzoneCluster.newBuilder(config).build();
cluster.waitForClusterToBeReady();
storageContainerLocationClient = cluster
.createStorageContainerLocationClient();
.getStorageContainerLocationClient();
}
@AfterClass

View File

@ -16,10 +16,9 @@
*/
package org.apache.hadoop.hdds.scm.node;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
import org.apache.hadoop.hdds.scm.client.ContainerOperationClient;
import org.apache.hadoop.test.GenericTestUtils;
@ -34,7 +33,6 @@ import static java.util.concurrent.TimeUnit.SECONDS;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DEAD;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.HEALTHY;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.INVALID;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.STALE;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
@ -52,7 +50,7 @@ import static org.junit.Assert.assertEquals;
*/
public class TestQueryNode {
private static int numOfDatanodes = 5;
private MiniOzoneClassicCluster cluster;
private MiniOzoneCluster cluster;
private ContainerOperationClient scmClient;
@ -67,13 +65,12 @@ public class TestQueryNode {
conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, SECONDS);
conf.setTimeDuration(OZONE_SCM_DEADNODE_INTERVAL, 6, SECONDS);
cluster = new MiniOzoneClassicCluster.Builder(conf)
.numDataNodes(numOfDatanodes)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED)
cluster = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(numOfDatanodes)
.build();
cluster.waitOzoneReady();
cluster.waitForClusterToBeReady();
scmClient = new ContainerOperationClient(cluster
.createStorageContainerLocationClient(),
.getStorageContainerLocationClient(),
new XceiverClientManager(conf));
}
@ -95,8 +92,8 @@ public class TestQueryNode {
@Test(timeout = 10 * 1000L)
public void testStaleNodesCount() throws Exception {
cluster.shutdownDataNode(0);
cluster.shutdownDataNode(1);
cluster.shutdownHddsDatanode(0);
cluster.shutdownHddsDatanode(1);
GenericTestUtils.waitFor(() ->
cluster.getStorageContainerManager().getNodeCount(STALE) == 2,

View File

@ -17,9 +17,7 @@
*/
package org.apache.hadoop.ozone.web;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneTestHelper;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
@ -49,7 +47,7 @@ public class TestDistributedOzoneVolumes extends TestOzoneHelper {
@Rule
public Timeout testTimeout = new Timeout(300000);
private static MiniOzoneClassicCluster cluster = null;
private static MiniOzoneCluster cluster = null;
private static int port = 0;
/**
@ -66,10 +64,10 @@ public class TestDistributedOzoneVolumes extends TestOzoneHelper {
Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
DataNode dataNode = cluster.getDataNodes().get(0);
port = MiniOzoneTestHelper.getOzoneRestPort(dataNode);
cluster = MiniOzoneCluster.newBuilder(conf).build();
cluster.waitForClusterToBeReady();
port = cluster.getHddsDatanodes().get(0)
.getDatanodeDetails().getOzoneRestPort();
}
/**

View File

@ -17,18 +17,16 @@
*/
package org.apache.hadoop.ozone.web;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneTestHelper;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.TestOzoneHelper;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.Timeout;
@ -45,7 +43,7 @@ public class TestLocalOzoneVolumes extends TestOzoneHelper {
@Rule
public Timeout testTimeout = new Timeout(300000);
private static MiniOzoneClassicCluster cluster = null;
private static MiniOzoneCluster cluster = null;
private static int port = 0;
/**
@ -69,10 +67,10 @@ public class TestLocalOzoneVolumes extends TestOzoneHelper {
conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_LOCAL).build();
DataNode dataNode = cluster.getDataNodes().get(0);
port = MiniOzoneTestHelper.getOzoneRestPort(dataNode);
cluster = MiniOzoneCluster.newBuilder(conf).build();
cluster.waitForClusterToBeReady();
port = cluster.getHddsDatanodes().get(0)
.getDatanodeDetails().getOzoneRestPort();
}
/**
@ -175,7 +173,7 @@ public class TestLocalOzoneVolumes extends TestOzoneHelper {
*
* @throws IOException
*/
@Test
@Test @Ignore
public void testGetVolumesOfAnotherUserShouldFail() throws IOException {
super.testGetVolumesOfAnotherUserShouldFail(port);
}

View File

@ -24,7 +24,6 @@ import static org.apache.hadoop.ozone.OzoneConsts.CHUNK_SIZE;
import static org.junit.Assert.*;
import org.apache.commons.lang.RandomStringUtils;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.web.client.OzoneRestClient;
import org.junit.AfterClass;
@ -61,15 +60,21 @@ public class TestOzoneRestWithMiniCluster {
@BeforeClass
public static void init() throws Exception {
conf = new OzoneConfiguration();
cluster = new MiniOzoneClassicCluster.Builder(conf).numDataNodes(1)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
cluster.waitOzoneReady();
ozoneClient = cluster.createOzoneRestClient();
cluster = MiniOzoneCluster.newBuilder(conf).build();
cluster.waitForClusterToBeReady();
int port = cluster.getHddsDatanodes().get(0)
.getDatanodeDetails().getOzoneRestPort();
ozoneClient = new OzoneRestClient(
String.format("http://localhost:%d", port));
ozoneClient.setUserAuth(OzoneConsts.OZONE_SIMPLE_HDFS_USER);
}
@AfterClass
public static void shutdown() throws InterruptedException {
IOUtils.cleanupWithLogger(null, ozoneClient, cluster);
if (cluster != null) {
cluster.shutdown();
}
IOUtils.cleanupWithLogger(null, ozoneClient);
}
@Test

View File

@ -18,9 +18,7 @@
package org.apache.hadoop.ozone.web;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneTestHelper;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
@ -57,7 +55,7 @@ public class TestOzoneWebAccess {
@Rule
public Timeout testTimeout = new Timeout(300000);
private static MiniOzoneClassicCluster cluster;
private static MiniOzoneCluster cluster;
private static int port;
/**
@ -77,10 +75,10 @@ public class TestOzoneWebAccess {
.getTempPath(TestOzoneWebAccess.class.getSimpleName());
conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_LOCAL).build();
DataNode dataNode = cluster.getDataNodes().get(0);
port = MiniOzoneTestHelper.getOzoneRestPort(dataNode);
cluster = MiniOzoneCluster.newBuilder(conf).build();
cluster.waitForClusterToBeReady();
port = cluster.getHddsDatanodes().get(0)
.getDatanodeDetails().getOzoneRestPort();
}
/**

View File

@ -18,12 +18,9 @@
package org.apache.hadoop.ozone.web.client;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneTestHelper;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.client.rest.OzoneException;
import org.apache.hadoop.ozone.web.request.OzoneQuota;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
@ -55,7 +52,7 @@ public class TestBuckets {
@Rule
public Timeout testTimeout = new Timeout(300000);
private static MiniOzoneClassicCluster cluster = null;
private static MiniOzoneCluster cluster = null;
private static OzoneRestClient ozoneRestClient = null;
/**
@ -78,10 +75,9 @@ public class TestBuckets {
OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
DataNode dataNode = cluster.getDataNodes().get(0);
final int port = MiniOzoneTestHelper.getOzoneRestPort(dataNode);
cluster = MiniOzoneCluster.newBuilder(conf).build();
final int port = cluster.getHddsDatanodes().get(0).getDatanodeDetails()
.getOzoneRestPort();
ozoneRestClient = new OzoneRestClient(
String.format("http://localhost:%d", port));
}

View File

@ -26,13 +26,10 @@ import org.apache.commons.lang.math.RandomUtils;
import org.apache.commons.lang3.tuple.ImmutablePair;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneTestHelper;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.ContainerProtos;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.common.helpers.KeyData;
@ -87,7 +84,7 @@ public class TestKeys {
@Rule
public Timeout testTimeout = new Timeout(300000);
private static MiniOzoneClassicCluster ozoneCluster = null;
private static MiniOzoneCluster ozoneCluster = null;
private static String path;
private static OzoneRestClient ozoneRestClient = null;
private static long currentTime;
@ -108,10 +105,10 @@ public class TestKeys {
path = GenericTestUtils.getTempPath(TestKeys.class.getSimpleName());
Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
ozoneCluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
DataNode dataNode = ozoneCluster.getDataNodes().get(0);
final int port = MiniOzoneTestHelper.getOzoneRestPort(dataNode);
ozoneCluster = MiniOzoneCluster.newBuilder(conf).build();
ozoneCluster.waitForClusterToBeReady();
final int port = ozoneCluster.getHddsDatanodes().get(0)
.getDatanodeDetails().getOzoneRestPort();
ozoneRestClient = new OzoneRestClient(
String.format("http://localhost:%d", port));
currentTime = Time.now();
@ -277,12 +274,12 @@ public class TestKeys {
}
private static void restartDatanode(
MiniOzoneClassicCluster cluster, int datanodeIdx, OzoneRestClient client)
throws IOException, OzoneException, URISyntaxException {
cluster.restartDataNode(datanodeIdx);
MiniOzoneCluster cluster, int datanodeIdx, OzoneRestClient client)
throws OzoneException, URISyntaxException {
cluster.restartHddsDatanode(datanodeIdx);
// refresh the datanode endpoint uri after datanode restart
DataNode dataNode = cluster.getDataNodes().get(datanodeIdx);
final int port = MiniOzoneTestHelper.getOzoneRestPort(dataNode);
final int port = ozoneCluster.getHddsDatanodes().get(0)
.getDatanodeDetails().getOzoneRestPort();
client.setEndPoint(String.format("http://localhost:%d", port));
}
@ -297,14 +294,13 @@ public class TestKeys {
}
static void runTestPutAndGetKeyWithDnRestart(
PutHelper helper, MiniOzoneClassicCluster cluster) throws Exception {
PutHelper helper, MiniOzoneCluster cluster) throws Exception {
String keyName = helper.putKey().getKeyName();
assertNotNull(helper.getBucket());
assertNotNull(helper.getFile());
// restart the datanode
restartDatanode(cluster, 0, helper.client);
// verify getKey after the datanode restart
String newFileName = helper.dir + "/"
+ OzoneUtils.getRequestID().toLowerCase();
@ -609,8 +605,8 @@ public class TestKeys {
Assert.assertEquals(20, bucketKeys.totalNumOfKeys());
int numOfCreatedKeys = 0;
OzoneContainer cm = MiniOzoneTestHelper
.getOzoneContainer(ozoneCluster.getDataNodes().get(0));
OzoneContainer cm = ozoneCluster.getHddsDatanodes().get(0)
.getDatanodeStateMachine().getContainer();
// Expected to delete chunk file list.
List<File> expectedChunkFiles = Lists.newArrayList();

View File

@ -18,7 +18,7 @@
package org.apache.hadoop.ozone.web.client;
import org.apache.commons.lang.RandomStringUtils;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.RatisTestHelper;
import org.junit.AfterClass;
@ -50,7 +50,7 @@ public class TestKeysRatis {
@Rule
public Timeout testTimeout = new Timeout(300000);
private static RatisTestHelper.RatisTestSuite suite;
private static MiniOzoneClassicCluster ozoneCluster = null;
private static MiniOzoneCluster ozoneCluster = null;
static private String path;
private static OzoneRestClient ozoneRestClient = null;
@ -59,6 +59,7 @@ public class TestKeysRatis {
suite = new RatisTestHelper.RatisTestSuite(TestBucketsRatis.class);
path = suite.getConf().get(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT);
ozoneCluster = suite.getCluster();
ozoneCluster.waitForClusterToBeReady();
ozoneRestClient = suite.newOzoneRestClient();
}

View File

@ -43,9 +43,7 @@ import io.netty.handler.codec.http.LastHttpContent;
import io.netty.handler.logging.LogLevel;
import io.netty.handler.logging.LoggingHandler;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneTestHelper;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
@ -86,7 +84,7 @@ import static io.netty.util.CharsetUtil.UTF_8;
public class TestOzoneClient {
private static Logger log = Logger.getLogger(TestOzoneClient.class);
private static int testVolumeCount = 5;
private static MiniOzoneClassicCluster cluster = null;
private static MiniOzoneCluster cluster = null;
private static String endpoint = null;
@BeforeClass
@ -95,11 +93,11 @@ public class TestOzoneClient {
OzoneConfiguration conf = new OzoneConfiguration();
conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
DataNode dataNode = cluster.getDataNodes().get(0);
endpoint = String.format("http://localhost:%d",
MiniOzoneTestHelper.getOzoneRestPort(dataNode));
cluster = MiniOzoneCluster.newBuilder(conf).build();
cluster.waitForClusterToBeReady();
int port = cluster.getHddsDatanodes().get(0)
.getDatanodeDetails().getOzoneRestPort();
endpoint = String.format("http://localhost:%d", port);
}
@AfterClass

View File

@ -21,10 +21,8 @@ package org.apache.hadoop.ozone.web.client;
import org.apache.commons.io.FileUtils;
import org.apache.commons.lang.RandomStringUtils;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.MiniOzoneTestHelper;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.Status;
@ -63,7 +61,7 @@ import static org.mockito.Mockito.verify;
* Test Ozone Volumes Lifecycle.
*/
public class TestVolume {
private static MiniOzoneClassicCluster cluster = null;
private static MiniOzoneCluster cluster = null;
private static OzoneRestClient ozoneRestClient = null;
/**
@ -88,10 +86,10 @@ public class TestVolume {
conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
cluster = new MiniOzoneClassicCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
DataNode dataNode = cluster.getDataNodes().get(0);
final int port = MiniOzoneTestHelper.getOzoneRestPort(dataNode);
cluster = MiniOzoneCluster.newBuilder(conf).build();
cluster.waitForClusterToBeReady();
final int port = cluster.getHddsDatanodes().get(0)
.getDatanodeDetails().getOzoneRestPort();
ozoneRestClient = new OzoneRestClient(
String.format("http://localhost:%d", port));

View File

@ -19,11 +19,9 @@
package org.apache.hadoop.ozone.web.client;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.client.rest.OzoneException;
import org.apache.hadoop.test.GenericTestUtils;
@ -41,7 +39,7 @@ public class TestVolumeRatis {
@Rule
public Timeout testTimeout = new Timeout(300000);
private static OzoneRestClient ozoneClient;
private static MiniOzoneClassicCluster cluster;
private static MiniOzoneCluster cluster;
@BeforeClass
public static void init() throws Exception {
@ -60,10 +58,10 @@ public class TestVolumeRatis {
conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
cluster = new MiniOzoneClassicCluster.Builder(conf).numDataNodes(3)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
DataNode dataNode = cluster.getDataNodes().get(0);
final int port = dataNode.getInfoPort();
cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(3).build();
cluster.waitForClusterToBeReady();
final int port = cluster.getHddsDatanodes().get(0)
.getDatanodeDetails().getOzoneRestPort();
ozoneClient = new OzoneRestClient(
String.format("http://localhost:%d", port));

View File

@ -0,0 +1,18 @@
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
# log4j configuration used during build and unit tests
log4j.rootLogger=info,stdout
log4j.threshold=ALL
log4j.appender.stdout=org.apache.log4j.ConsoleAppender
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n

View File

@ -104,8 +104,8 @@ public class SQLCLI extends Configured implements Tool {
"VALUES (\"%s\", \"%s\")";
private static final String INSERT_DATANODE_INFO =
"INSERT INTO datanodeInfo (hostname, datanodeUUid, ipAddress, " +
"containerPort,) " +
"VALUES (\"%s\", \"%s\", \"%s\", %d";
"containerPort) " +
"VALUES (\"%s\", \"%s\", \"%s\", \"%d\")";
private static final String INSERT_CONTAINER_MEMBERS =
"INSERT INTO containerMembers (containerName, datanodeUUID) " +
"VALUES (\"%s\", \"%s\")";
@ -644,7 +644,7 @@ public class SQLCLI extends Configured implements Tool {
String insertDatanodeDetails = String
.format(INSERT_DATANODE_INFO, datanodeDetails.getHostName(),
datanodeDetails.getUuid(), datanodeDetails.getIpAddress(),
datanodeDetails.getUuidString(), datanodeDetails.getIpAddress(),
datanodeDetails.getContainerPort());
executeSQL(conn, insertDatanodeDetails);
}

View File

@ -170,5 +170,30 @@
<artifactId>hadoop-mapreduce-client-jobclient</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdds-server-framework</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdds-server-scm</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdds-client</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdds-container-service</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-ozone-ozone-manager</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
</project>

View File

@ -24,13 +24,12 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.web.handlers.BucketArgs;
import org.apache.hadoop.ozone.web.handlers.UserArgs;
import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
@ -48,7 +47,7 @@ import java.util.Arrays;
* Test OzoneFSInputStream by reading through multiple interfaces.
*/
public class TestOzoneFSInputStream {
private static MiniOzoneClassicCluster cluster = null;
private static MiniOzoneCluster cluster = null;
private static FileSystem fs;
private static StorageHandler storageHandler;
private static Path filePath = null;
@ -66,10 +65,10 @@ public class TestOzoneFSInputStream {
public static void init() throws Exception {
OzoneConfiguration conf = new OzoneConfiguration();
conf.setLong(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB, 10);
cluster = new MiniOzoneClassicCluster.Builder(conf)
.numDataNodes(10)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED)
cluster = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(10)
.build();
cluster.waitForClusterToBeReady();
storageHandler =
new ObjectStoreHandler(conf).getStorageHandler();
@ -88,9 +87,10 @@ public class TestOzoneFSInputStream {
storageHandler.createBucket(bucketArgs);
// Fetch the host and port for File System init
DataNode dataNode = cluster.getDataNodes().get(0);
int port = dataNode.getInfoPort();
String host = dataNode.getDatanodeHostname();
DatanodeDetails datanodeDetails = cluster.getHddsDatanodes().get(0)
.getDatanodeDetails();
int port = datanodeDetails.getOzoneRestPort();
String host = datanodeDetails.getHostName();
// Set the fs.defaultFS and start the filesystem
String uri = String.format("%s://%s.%s/",

View File

@ -23,6 +23,7 @@ import java.net.URI;
import java.util.Arrays;
import java.util.Collection;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
@ -40,8 +41,6 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.web.handlers.BucketArgs;
import org.apache.hadoop.ozone.web.handlers.UserArgs;
import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
@ -82,7 +81,7 @@ public class TestOzoneFileInterfaces {
private boolean useAbsolutePath;
private static MiniOzoneClassicCluster cluster = null;
private static MiniOzoneCluster cluster = null;
private static FileSystem fs;
@ -97,10 +96,10 @@ public class TestOzoneFileInterfaces {
@Before
public void init() throws Exception {
OzoneConfiguration conf = new OzoneConfiguration();
cluster = new MiniOzoneClassicCluster.Builder(conf)
.numDataNodes(3)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED)
cluster = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(3)
.build();
cluster.waitForClusterToBeReady();
storageHandler =
new ObjectStoreHandler(conf).getStorageHandler();
@ -132,9 +131,11 @@ public class TestOzoneFileInterfaces {
@After
public void teardown() throws IOException {
if (cluster != null) {
cluster.shutdown();
}
IOUtils.closeQuietly(fs);
IOUtils.closeQuietly(storageHandler);
IOUtils.closeQuietly(cluster);
}
@Test

View File

@ -26,8 +26,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.AbstractFSContract;
import org.apache.hadoop.fs.ozone.Constants;
import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.client.rest.OzoneException;
import org.apache.hadoop.ozone.web.handlers.BucketArgs;
import org.apache.hadoop.ozone.web.handlers.UserArgs;
@ -45,7 +44,7 @@ import java.io.IOException;
*/
class OzoneContract extends AbstractFSContract {
private static MiniOzoneClassicCluster cluster;
private static MiniOzoneCluster cluster;
private static StorageHandler storageHandler;
private static final String CONTRACT_XML = "contract/ozone.xml";
@ -70,10 +69,12 @@ class OzoneContract extends AbstractFSContract {
OzoneConfiguration conf = new OzoneConfiguration();
conf.addResource(CONTRACT_XML);
cluster =
new MiniOzoneClassicCluster.Builder(conf).numDataNodes(5)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
cluster.waitClusterUp();
cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(5).build();
try {
cluster.waitForClusterToBeReady();
} catch (Exception e) {
throw new IOException(e);
}
storageHandler = new ObjectStoreHandler(conf).getStorageHandler();
}