Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
import java.lang.reflect.Method;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.nio.file.Paths;
import java.util.Calendar;
import java.util.Collection;
import java.util.HashSet;
Expand All @@ -50,7 +49,6 @@
import org.apache.hadoop.net.DNS;
import org.apache.hadoop.net.NetUtils;

import com.google.common.base.Strings;
import com.google.common.net.HostAndPort;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_INTERFACE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_NAMESERVER_KEY;
Expand Down Expand Up @@ -312,28 +310,6 @@ public static boolean isHddsEnabled(Configuration conf) {
}


/**
* Get the path for datanode id file.
*
* @param conf - Configuration
* @return the path of datanode id as string
*/
public static String getDatanodeIdFilePath(Configuration conf) {
String dataNodeIDPath = conf.get(ScmConfigKeys.OZONE_SCM_DATANODE_ID);
if (dataNodeIDPath == null) {
String metaPath = conf.get(HddsConfigKeys.OZONE_METADATA_DIRS);
if (Strings.isNullOrEmpty(metaPath)) {
// this means meta data is not found, in theory should not happen at
// this point because should've failed earlier.
throw new IllegalArgumentException("Unable to locate meta data" +
"directory when getting datanode id path");
}
dataNodeIDPath = Paths.get(metaPath,
ScmConfigKeys.OZONE_SCM_DATANODE_ID_PATH_DEFAULT).toString();
}
return dataNodeIDPath;
}

/**
* Returns the hostname for this datanode. If the hostname is not
* explicitly configured in the given config, then it is determined
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -372,4 +372,26 @@ public static String getDefaultRatisDirectory(Configuration conf) {
File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
return (new File(metaDirPath, "ratis")).getPath();
}

/**
* Get the path for datanode id file.
*
* @param conf - Configuration
* @return the path of datanode id as string
*/
public static String getDatanodeIdFilePath(Configuration conf) {
String dataNodeIDPath = conf.get(ScmConfigKeys.OZONE_SCM_DATANODE_ID);
if (dataNodeIDPath == null) {
File metaDirPath = ServerUtils.getOzoneMetaDirPath(conf);
if (metaDirPath == null) {
// this means meta data is not found, in theory should not happen at
// this point because should've failed earlier.
throw new IllegalArgumentException("Unable to locate meta data" +
"directory when getting datanode id path");
}
dataNodeIDPath = new File(metaDirPath,
ScmConfigKeys.OZONE_SCM_DATANODE_ID_PATH_DEFAULT).toString();
}
return dataNodeIDPath;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.SCMSecurityProtocol;
import org.apache.hadoop.hdds.scm.HddsServerUtil;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
Expand Down Expand Up @@ -308,7 +309,7 @@ public PKCS10CertificationRequest getCSR(Configuration config)
*/
private DatanodeDetails initializeDatanodeDetails()
throws IOException {
String idFilePath = HddsUtils.getDatanodeIdFilePath(conf);
String idFilePath = HddsServerUtil.getDatanodeIdFilePath(conf);
if (idFilePath == null || idFilePath.isEmpty()) {
LOG.error("A valid file path is needed for config setting {}",
ScmConfigKeys.OZONE_SCM_DATANODE_ID);
Expand Down Expand Up @@ -338,7 +339,7 @@ private DatanodeDetails initializeDatanodeDetails()
*/
private void persistDatanodeDetails(DatanodeDetails dnDetails)
throws IOException {
String idFilePath = HddsUtils.getDatanodeIdFilePath(conf);
String idFilePath = HddsServerUtil.getDatanodeIdFilePath(conf);
if (idFilePath == null || idFilePath.isEmpty()) {
LOG.error("A valid file path is needed for config setting {}",
ScmConfigKeys.OZONE_SCM_DATANODE_ID);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,8 @@

import com.google.common.base.Strings;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.HddsUtils;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.HddsServerUtil;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.common.statemachine
Expand Down Expand Up @@ -117,7 +117,7 @@ public DatanodeStateMachine.DatanodeStates call() throws Exception {
* Persist DatanodeDetails to datanode.id file.
*/
private void persistContainerDatanodeDetails() {
String dataNodeIDPath = HddsUtils.getDatanodeIdFilePath(conf);
String dataNodeIDPath = HddsServerUtil.getDatanodeIdFilePath(conf);
if (Strings.isNullOrEmpty(dataNodeIDPath)) {
LOG.error("A valid file path is needed for config setting {}",
ScmConfigKeys.OZONE_SCM_DATANODE_ID);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -125,8 +125,8 @@ public static void releaseConnection(HttpRequestBase request) {
* @return
*/
public static File getScmDbDir(Configuration conf) {
File metadataDir = getDirectoryFromConfig(conf, ScmConfigKeys
.OZONE_SCM_DB_DIRS, "SCM");
File metadataDir = getDirectoryFromConfig(conf,
ScmConfigKeys.OZONE_SCM_DB_DIRS, "SCM");
if (metadataDir != null) {
return metadataDir;
}
Expand All @@ -146,8 +146,8 @@ public static File getScmDbDir(Configuration conf) {
* @return File created from the value of the key in conf.
*/
public static File getDirectoryFromConfig(Configuration conf,
String key,
String componentName) {
String key,
String componentName) {
final Collection<String> metadirs = conf.getTrimmedStringCollection(key);

if (metadirs.size() > 1) {
Expand All @@ -162,33 +162,28 @@ public static File getDirectoryFromConfig(Configuration conf,
if (!dbDirPath.exists() && !dbDirPath.mkdirs()) {
throw new IllegalArgumentException("Unable to create directory " +
dbDirPath + " specified in configuration setting " +
componentName);
key);
}
return dbDirPath;
}

return null;
}

/**
* Checks and creates Ozone Metadir Path if it does not exist.
*
* @param conf - Configuration
*
* @return File MetaDir
* @throws IllegalArgumentException if the configuration setting is not set
*/
public static File getOzoneMetaDirPath(Configuration conf) {
String metaDirPath = conf.getTrimmed(HddsConfigKeys.OZONE_METADATA_DIRS);

if (metaDirPath == null || metaDirPath.isEmpty()) {
File dirPath = getDirectoryFromConfig(conf,
HddsConfigKeys.OZONE_METADATA_DIRS, "Ozone");
if (dirPath == null) {
throw new IllegalArgumentException(
HddsConfigKeys.OZONE_METADATA_DIRS + " must be defined.");
}

File dirPath = new File(metaDirPath);
if (!dirPath.exists() && !dirPath.mkdirs()) {
throw new IllegalArgumentException("Unable to create paths. Path: " +
dirPath);
}
return dirPath;
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,123 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdds.server;

import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.test.PathUtils;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;

import java.io.File;

import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;

/**
* Unit tests for {@link ServerUtils}.
*/
public class TestServerUtils {

@Rule
public ExpectedException thrown = ExpectedException.none();

/**
* Test {@link ServerUtils#getScmDbDir}.
*/
@Test
public void testGetScmDbDir() {
final File testDir = PathUtils.getTestDir(TestServerUtils.class);
final File dbDir = new File(testDir, "scmDbDir");
final File metaDir = new File(testDir, "metaDir");
final Configuration conf = new OzoneConfiguration();
conf.set(ScmConfigKeys.OZONE_SCM_DB_DIRS, dbDir.getPath());
conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, metaDir.getPath());

try {
assertFalse(metaDir.exists());
assertFalse(dbDir.exists());
assertEquals(dbDir, ServerUtils.getScmDbDir(conf));
assertTrue(dbDir.exists());
assertFalse(metaDir.exists());
} finally {
FileUtils.deleteQuietly(dbDir);
}
}

/**
* Test {@link ServerUtils#getScmDbDir} with fallback to OZONE_METADATA_DIRS
* when OZONE_SCM_DB_DIRS is undefined.
*/
@Test
public void testGetScmDbDirWithFallback() {
final File testDir = PathUtils.getTestDir(TestServerUtils.class);
final File metaDir = new File(testDir, "metaDir");
final Configuration conf = new OzoneConfiguration();
conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, metaDir.getPath());
try {
assertFalse(metaDir.exists());
assertEquals(metaDir, ServerUtils.getScmDbDir(conf));
assertTrue(metaDir.exists());
} finally {
FileUtils.deleteQuietly(metaDir);
}
}

@Test
public void testNoScmDbDirConfigured() {
thrown.expect(IllegalArgumentException.class);
ServerUtils.getScmDbDir(new OzoneConfiguration());
}

@Test
public void ozoneMetadataDirIsMandatory() {
thrown.expect(IllegalArgumentException.class);
ServerUtils.getOzoneMetaDirPath(new OzoneConfiguration());
}

@Test
public void ozoneMetadataDirAcceptsSingleItem() {
final File testDir = PathUtils.getTestDir(TestServerUtils.class);
final File metaDir = new File(testDir, "metaDir");
final Configuration conf = new OzoneConfiguration();
conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, metaDir.getPath());

try {
assertFalse(metaDir.exists());
assertEquals(metaDir, ServerUtils.getOzoneMetaDirPath(conf));
assertTrue(metaDir.exists());
} finally {
FileUtils.deleteQuietly(metaDir);
}
}

@Test
public void ozoneMetadataDirRejectsList() {
final Configuration conf = new OzoneConfiguration();
conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, "/data/meta1,/data/meta2");
thrown.expect(IllegalArgumentException.class);

ServerUtils.getOzoneMetaDirPath(conf);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@
import org.slf4j.LoggerFactory;

import java.io.File;
import java.util.Collection;

/**
* SCM utility class.
Expand All @@ -55,22 +54,9 @@ public static void preCheck(ScmOps operation, Precheck... preChecks)
}

public static File getDBPath(Configuration conf, String dbDirectory) {
final Collection<String> dbDirs =
conf.getTrimmedStringCollection(dbDirectory);

if (dbDirs.size() > 1) {
throw new IllegalArgumentException(
"Bad configuration setting " + dbDirectory
+ ". OM does not support multiple metadata dirs currently.");
}

if (dbDirs.size() == 1) {
final File dbDirPath = new File(dbDirs.iterator().next());
if (!dbDirPath.exists() && !dbDirPath.mkdirs()) {
throw new IllegalArgumentException(
"Unable to create directory " + dbDirPath
+ " specified in configuration setting " + dbDirectory);
}
final File dbDirPath =
ServerUtils.getDirectoryFromConfig(conf, dbDirectory, "OM");
if (dbDirPath != null) {
return dbDirPath;
}

Expand Down
Loading