commit
5c13222110
hadoop-common-project/hadoop-common/src
main
conf
java/org/apache/hadoop
site/markdown
test/java/org/apache/hadoop
fs
test
util
hadoop-hdds
client/src/main/java/org/apache/hadoop/hdds/scm
common/src
main
java/org/apache/hadoop
hdds
HddsConfigKeys.javaHddsUtils.java
cli
conf
scm
security
token
x509
certificate
authority
client
utils
certificates/utils
exceptions
keys
ozone
utils
proto
resources
test/java/org/apache/hadoop
hdds
conf
security/x509
certificate
authority
client
certificates
utils
container-service/src
main
java/org/apache/hadoop/ozone
HddsDatanodeHttpServer.javaHddsDatanodeService.java
container/common
impl
interfaces
transport/server/ratis
resources/webapps/hddsDatanode
test/java/org/apache/hadoop/ozone
docs/content
framework/src/main
server-scm/src
main
java/org/apache/hadoop/hdds/scm
block
chillmode
ChillModeHandler.javaContainerChillModeRule.javaDataNodeChillModeRule.javaOneReplicaPipelineChillModeRule.javaSCMChillModeManager.java
container/replication
events
pipeline
PipelineFactory.javaPipelineManager.javaPipelineStateManager.javaPipelineStateMap.javaRatisPipelineProvider.javaRatisPipelineUtils.javaSCMPipelineManager.java
server
resources/webapps/scm
test/java/org/apache/hadoop/hdds/scm
|
@ -182,25 +182,6 @@ log4j.logger.com.amazonaws.http.AmazonHttpClient=ERROR
|
||||||
#
|
#
|
||||||
log4j.appender.EventCounter=org.apache.hadoop.log.metrics.EventCounter
|
log4j.appender.EventCounter=org.apache.hadoop.log.metrics.EventCounter
|
||||||
|
|
||||||
#
|
|
||||||
# Job Summary Appender
|
|
||||||
#
|
|
||||||
# Use following logger to send summary to separate file defined by
|
|
||||||
# hadoop.mapreduce.jobsummary.log.file :
|
|
||||||
# hadoop.mapreduce.jobsummary.logger=INFO,JSA
|
|
||||||
#
|
|
||||||
hadoop.mapreduce.jobsummary.logger=${hadoop.root.logger}
|
|
||||||
hadoop.mapreduce.jobsummary.log.file=hadoop-mapreduce.jobsummary.log
|
|
||||||
hadoop.mapreduce.jobsummary.log.maxfilesize=256MB
|
|
||||||
hadoop.mapreduce.jobsummary.log.maxbackupindex=20
|
|
||||||
log4j.appender.JSA=org.apache.log4j.RollingFileAppender
|
|
||||||
log4j.appender.JSA.File=${hadoop.log.dir}/${hadoop.mapreduce.jobsummary.log.file}
|
|
||||||
log4j.appender.JSA.MaxFileSize=${hadoop.mapreduce.jobsummary.log.maxfilesize}
|
|
||||||
log4j.appender.JSA.MaxBackupIndex=${hadoop.mapreduce.jobsummary.log.maxbackupindex}
|
|
||||||
log4j.appender.JSA.layout=org.apache.log4j.PatternLayout
|
|
||||||
log4j.appender.JSA.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n
|
|
||||||
log4j.logger.org.apache.hadoop.mapred.JobInProgress$JobSummary=${hadoop.mapreduce.jobsummary.logger}
|
|
||||||
log4j.additivity.org.apache.hadoop.mapred.JobInProgress$JobSummary=false
|
|
||||||
|
|
||||||
#
|
#
|
||||||
# shuffle connection log from shuffleHandler
|
# shuffle connection log from shuffleHandler
|
||||||
|
|
|
@ -740,6 +740,7 @@ public class CryptoInputStream extends FilterInputStream implements
|
||||||
case StreamCapabilities.READAHEAD:
|
case StreamCapabilities.READAHEAD:
|
||||||
case StreamCapabilities.DROPBEHIND:
|
case StreamCapabilities.DROPBEHIND:
|
||||||
case StreamCapabilities.UNBUFFER:
|
case StreamCapabilities.UNBUFFER:
|
||||||
|
case StreamCapabilities.READBYTEBUFFER:
|
||||||
return true;
|
return true;
|
||||||
default:
|
default:
|
||||||
return false;
|
return false;
|
||||||
|
|
|
@ -372,6 +372,12 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
||||||
+ "by ChecksumFileSystem");
|
+ "by ChecksumFileSystem");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void concat(final Path f, final Path[] psrcs) throws IOException {
|
||||||
|
throw new UnsupportedOperationException("Concat is not supported "
|
||||||
|
+ "by ChecksumFileSystem");
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Calculated the length of the checksum file in bytes.
|
* Calculated the length of the checksum file in bytes.
|
||||||
* @param size the length of the data file in bytes
|
* @param size the length of the data file in bytes
|
||||||
|
|
|
@ -59,6 +59,12 @@ public interface StreamCapabilities {
|
||||||
*/
|
*/
|
||||||
String UNBUFFER = "in:unbuffer";
|
String UNBUFFER = "in:unbuffer";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Stream read(ByteBuffer) capability implemented by
|
||||||
|
* {@link ByteBufferReadable#read(java.nio.ByteBuffer)}.
|
||||||
|
*/
|
||||||
|
String READBYTEBUFFER = "in:readbytebuffer";
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Capabilities that a stream can support and be queried for.
|
* Capabilities that a stream can support and be queried for.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -120,6 +120,12 @@ public class Trash extends Configured {
|
||||||
trashPolicy.deleteCheckpoint();
|
trashPolicy.deleteCheckpoint();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Delete all trash immediately. */
|
||||||
|
public void expungeImmediately() throws IOException {
|
||||||
|
trashPolicy.createCheckpoint();
|
||||||
|
trashPolicy.deleteCheckpointsImmediately();
|
||||||
|
}
|
||||||
|
|
||||||
/** get the current working directory */
|
/** get the current working directory */
|
||||||
Path getCurrentTrashDir() throws IOException {
|
Path getCurrentTrashDir() throws IOException {
|
||||||
return trashPolicy.getCurrentTrashDir();
|
return trashPolicy.getCurrentTrashDir();
|
||||||
|
|
|
@ -79,6 +79,11 @@ public abstract class TrashPolicy extends Configured {
|
||||||
*/
|
*/
|
||||||
public abstract void deleteCheckpoint() throws IOException;
|
public abstract void deleteCheckpoint() throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Delete all checkpoints immediately, ie empty trash.
|
||||||
|
*/
|
||||||
|
public abstract void deleteCheckpointsImmediately() throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the current working directory of the Trash Policy
|
* Get the current working directory of the Trash Policy
|
||||||
* This API does not work with files deleted from encryption zone when HDFS
|
* This API does not work with files deleted from encryption zone when HDFS
|
||||||
|
|
|
@ -213,11 +213,20 @@ public class TrashPolicyDefault extends TrashPolicy {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void deleteCheckpoint() throws IOException {
|
public void deleteCheckpoint() throws IOException {
|
||||||
|
deleteCheckpoint(false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void deleteCheckpointsImmediately() throws IOException {
|
||||||
|
deleteCheckpoint(true);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void deleteCheckpoint(boolean deleteImmediately) throws IOException {
|
||||||
Collection<FileStatus> trashRoots = fs.getTrashRoots(false);
|
Collection<FileStatus> trashRoots = fs.getTrashRoots(false);
|
||||||
for (FileStatus trashRoot : trashRoots) {
|
for (FileStatus trashRoot : trashRoots) {
|
||||||
LOG.info("TrashPolicyDefault#deleteCheckpoint for trashRoot: " +
|
LOG.info("TrashPolicyDefault#deleteCheckpoint for trashRoot: " +
|
||||||
trashRoot.getPath());
|
trashRoot.getPath());
|
||||||
deleteCheckpoint(trashRoot.getPath());
|
deleteCheckpoint(trashRoot.getPath(), deleteImmediately);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -283,7 +292,7 @@ public class TrashPolicyDefault extends TrashPolicy {
|
||||||
continue;
|
continue;
|
||||||
try {
|
try {
|
||||||
TrashPolicyDefault trash = new TrashPolicyDefault(fs, conf);
|
TrashPolicyDefault trash = new TrashPolicyDefault(fs, conf);
|
||||||
trash.deleteCheckpoint(trashRoot.getPath());
|
trash.deleteCheckpoint(trashRoot.getPath(), false);
|
||||||
trash.createCheckpoint(trashRoot.getPath(), new Date(now));
|
trash.createCheckpoint(trashRoot.getPath(), new Date(now));
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
LOG.warn("Trash caught: "+e+". Skipping " +
|
LOG.warn("Trash caught: "+e+". Skipping " +
|
||||||
|
@ -341,7 +350,8 @@ public class TrashPolicyDefault extends TrashPolicy {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void deleteCheckpoint(Path trashRoot) throws IOException {
|
private void deleteCheckpoint(Path trashRoot, boolean deleteImmediately)
|
||||||
|
throws IOException {
|
||||||
LOG.info("TrashPolicyDefault#deleteCheckpoint for trashRoot: " + trashRoot);
|
LOG.info("TrashPolicyDefault#deleteCheckpoint for trashRoot: " + trashRoot);
|
||||||
|
|
||||||
FileStatus[] dirs = null;
|
FileStatus[] dirs = null;
|
||||||
|
@ -368,7 +378,7 @@ public class TrashPolicyDefault extends TrashPolicy {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
if ((now - deletionInterval) > time) {
|
if (((now - deletionInterval) > time) || deleteImmediately) {
|
||||||
if (fs.delete(path, true)) {
|
if (fs.delete(path, true)) {
|
||||||
LOG.info("Deleted trash checkpoint: "+dir);
|
LOG.info("Deleted trash checkpoint: "+dir);
|
||||||
} else {
|
} else {
|
||||||
|
|
|
@ -219,16 +219,20 @@ class Delete {
|
||||||
// than the retention threshold.
|
// than the retention threshold.
|
||||||
static class Expunge extends FsCommand {
|
static class Expunge extends FsCommand {
|
||||||
public static final String NAME = "expunge";
|
public static final String NAME = "expunge";
|
||||||
public static final String USAGE = "";
|
public static final String USAGE =
|
||||||
|
"[-immediate]";
|
||||||
public static final String DESCRIPTION =
|
public static final String DESCRIPTION =
|
||||||
"Delete files from the trash that are older " +
|
"Delete files from the trash that are older " +
|
||||||
"than the retention threshold";
|
"than the retention threshold";
|
||||||
|
|
||||||
|
private boolean emptyImmediately = false;
|
||||||
|
|
||||||
// TODO: should probably allow path arguments for the filesystems
|
// TODO: should probably allow path arguments for the filesystems
|
||||||
@Override
|
@Override
|
||||||
protected void processOptions(LinkedList<String> args) throws IOException {
|
protected void processOptions(LinkedList<String> args) throws IOException {
|
||||||
CommandFormat cf = new CommandFormat(0, 0);
|
CommandFormat cf = new CommandFormat(0, 1, "immediate");
|
||||||
cf.parse(args);
|
cf.parse(args);
|
||||||
|
emptyImmediately = cf.getOpt("immediate");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -239,14 +243,23 @@ class Delete {
|
||||||
if (null != childFileSystems) {
|
if (null != childFileSystems) {
|
||||||
for (FileSystem fs : childFileSystems) {
|
for (FileSystem fs : childFileSystems) {
|
||||||
Trash trash = new Trash(fs, getConf());
|
Trash trash = new Trash(fs, getConf());
|
||||||
trash.expunge();
|
if (emptyImmediately) {
|
||||||
trash.checkpoint();
|
trash.expungeImmediately();
|
||||||
|
} else {
|
||||||
|
trash.expunge();
|
||||||
|
trash.checkpoint();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
Trash trash = new Trash(getConf());
|
Trash trash = new Trash(getConf());
|
||||||
trash.expunge();
|
if (emptyImmediately) {
|
||||||
trash.checkpoint();
|
trash.expungeImmediately();
|
||||||
|
} else {
|
||||||
|
trash.expunge();
|
||||||
|
trash.checkpoint();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -288,8 +288,10 @@ public class NetUtils {
|
||||||
if (fqHost == null) {
|
if (fqHost == null) {
|
||||||
try {
|
try {
|
||||||
fqHost = SecurityUtil.getByName(host).getHostName();
|
fqHost = SecurityUtil.getByName(host).getHostName();
|
||||||
// slight race condition, but won't hurt
|
|
||||||
canonicalizedHostCache.putIfAbsent(host, fqHost);
|
canonicalizedHostCache.putIfAbsent(host, fqHost);
|
||||||
|
// ensures that we won't return a canonicalized stale (non-cached)
|
||||||
|
// host name for a given host
|
||||||
|
fqHost = canonicalizedHostCache.get(host);
|
||||||
} catch (UnknownHostException e) {
|
} catch (UnknownHostException e) {
|
||||||
fqHost = host;
|
fqHost = host;
|
||||||
}
|
}
|
||||||
|
|
|
@ -249,7 +249,7 @@ public class NodeHealthScriptRunner extends AbstractService {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Sets if the node is healhty or not considering disks' health also.
|
* Sets if the node is healthy or not considering disks' health also.
|
||||||
*
|
*
|
||||||
* @param isHealthy
|
* @param isHealthy
|
||||||
* if or not node is healthy
|
* if or not node is healthy
|
||||||
|
|
|
@ -264,7 +264,7 @@ Displays a summary of file lengths.
|
||||||
expunge
|
expunge
|
||||||
-------
|
-------
|
||||||
|
|
||||||
Usage: `hadoop fs -expunge`
|
Usage: `hadoop fs -expunge [-immediate]`
|
||||||
|
|
||||||
Permanently delete files in checkpoints older than the retention threshold
|
Permanently delete files in checkpoints older than the retention threshold
|
||||||
from trash directory, and create new checkpoint.
|
from trash directory, and create new checkpoint.
|
||||||
|
@ -279,6 +279,9 @@ users can configure to create and delete checkpoints periodically
|
||||||
by the parameter stored as `fs.trash.checkpoint.interval` (in core-site.xml).
|
by the parameter stored as `fs.trash.checkpoint.interval` (in core-site.xml).
|
||||||
This value should be smaller or equal to `fs.trash.interval`.
|
This value should be smaller or equal to `fs.trash.interval`.
|
||||||
|
|
||||||
|
If the `-immediate` option is passed, all files in the trash for the current
|
||||||
|
user are immediately deleted, ignoring the `fs.trash.interval` setting.
|
||||||
|
|
||||||
Refer to the
|
Refer to the
|
||||||
[HDFS Architecture guide](../hadoop-hdfs/HdfsDesign.html#File_Deletes_and_Undeletes)
|
[HDFS Architecture guide](../hadoop-hdfs/HdfsDesign.html#File_Deletes_and_Undeletes)
|
||||||
for more information about trash feature of HDFS.
|
for more information about trash feature of HDFS.
|
||||||
|
|
|
@ -36,6 +36,8 @@ import java.util.concurrent.atomic.AtomicInteger;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
import static org.junit.Assert.*;
|
import static org.junit.Assert.*;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
@ -486,6 +488,41 @@ public class TestTrash {
|
||||||
trashRootFs.exists(dirToKeep));
|
trashRootFs.exists(dirToKeep));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Verify expunge -immediate removes all checkpoints and current folder
|
||||||
|
{
|
||||||
|
// Setup a recent and old checkpoint and a current folder
|
||||||
|
// to be deleted on the next expunge and one that isn't.
|
||||||
|
long trashInterval = conf.getLong(FS_TRASH_INTERVAL_KEY,
|
||||||
|
FS_TRASH_INTERVAL_DEFAULT);
|
||||||
|
long now = Time.now();
|
||||||
|
DateFormat checkpointFormat = new SimpleDateFormat("yyMMddHHmm");
|
||||||
|
Path oldCheckpoint = new Path(trashRoot.getParent(),
|
||||||
|
checkpointFormat.format(now - (trashInterval * 60 * 1000) - 1));
|
||||||
|
Path recentCheckpoint = new Path(trashRoot.getParent(),
|
||||||
|
checkpointFormat.format(now));
|
||||||
|
Path currentFolder = new Path(trashRoot.getParent(), "Current");
|
||||||
|
mkdir(trashRootFs, oldCheckpoint);
|
||||||
|
mkdir(trashRootFs, recentCheckpoint);
|
||||||
|
mkdir(trashRootFs, currentFolder);
|
||||||
|
|
||||||
|
// Clear out trash
|
||||||
|
int rc = -1;
|
||||||
|
try {
|
||||||
|
rc = shell.run(new String[] {"-expunge", "-immediate"});
|
||||||
|
} catch (Exception e) {
|
||||||
|
fail("Unexpected exception running the trash shell: " +
|
||||||
|
e.getLocalizedMessage());
|
||||||
|
}
|
||||||
|
assertEquals("Expunge immediate should return zero", 0, rc);
|
||||||
|
assertFalse("Old checkpoint should be removed",
|
||||||
|
trashRootFs.exists(oldCheckpoint));
|
||||||
|
assertFalse("Recent checkpoint should be removed",
|
||||||
|
trashRootFs.exists(recentCheckpoint));
|
||||||
|
assertFalse("Current folder should be removed",
|
||||||
|
trashRootFs.exists(currentFolder));
|
||||||
|
assertEquals("Ensure trash folder is empty",
|
||||||
|
trashRootFs.listStatus(trashRoot.getParent()).length, 0);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void trashNonDefaultFS(Configuration conf) throws IOException {
|
public static void trashNonDefaultFS(Configuration conf) throws IOException {
|
||||||
|
@ -1000,6 +1037,10 @@ public class TestTrash {
|
||||||
public void deleteCheckpoint() throws IOException {
|
public void deleteCheckpoint() throws IOException {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void deleteCheckpointsImmediately() throws IOException {
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Path getCurrentTrashDir() {
|
public Path getCurrentTrashDir() {
|
||||||
return null;
|
return null;
|
||||||
|
@ -1059,6 +1100,11 @@ public class TestTrash {
|
||||||
AuditableCheckpoints.delete();
|
AuditableCheckpoints.delete();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void deleteCheckpointsImmediately() throws IOException {
|
||||||
|
AuditableCheckpoints.deleteAll();
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Path getCurrentTrashDir() {
|
public Path getCurrentTrashDir() {
|
||||||
return null;
|
return null;
|
||||||
|
@ -1115,25 +1161,32 @@ public class TestTrash {
|
||||||
*/
|
*/
|
||||||
private static class AuditableCheckpoints {
|
private static class AuditableCheckpoints {
|
||||||
|
|
||||||
|
private static final Logger LOG =
|
||||||
|
LoggerFactory.getLogger(AuditableCheckpoints.class);
|
||||||
|
|
||||||
private static AtomicInteger numOfCheckpoint =
|
private static AtomicInteger numOfCheckpoint =
|
||||||
new AtomicInteger(0);
|
new AtomicInteger(0);
|
||||||
|
|
||||||
private static void add() {
|
private static void add() {
|
||||||
numOfCheckpoint.incrementAndGet();
|
numOfCheckpoint.incrementAndGet();
|
||||||
System.out.println(String
|
LOG.info("Create a checkpoint, current number of checkpoints {}",
|
||||||
.format("Create a checkpoint, current number of checkpoints %d",
|
numOfCheckpoint.get());
|
||||||
numOfCheckpoint.get()));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void delete() {
|
private static void delete() {
|
||||||
if(numOfCheckpoint.get() > 0) {
|
if(numOfCheckpoint.get() > 0) {
|
||||||
numOfCheckpoint.decrementAndGet();
|
numOfCheckpoint.decrementAndGet();
|
||||||
System.out.println(String
|
LOG.info("Delete a checkpoint, current number of checkpoints {}",
|
||||||
.format("Delete a checkpoint, current number of checkpoints %d",
|
numOfCheckpoint.get());
|
||||||
numOfCheckpoint.get()));
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static void deleteAll() {
|
||||||
|
numOfCheckpoint.set(0);
|
||||||
|
LOG.info("Delete all checkpoints, current number of checkpoints {}",
|
||||||
|
numOfCheckpoint.get());
|
||||||
|
}
|
||||||
|
|
||||||
private static int get() {
|
private static int get() {
|
||||||
return numOfCheckpoint.get();
|
return numOfCheckpoint.get();
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,6 +17,8 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.fs.contract.localfs;
|
package org.apache.hadoop.fs.contract.localfs;
|
||||||
|
|
||||||
|
import org.junit.Assume;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.contract.AbstractContractMultipartUploaderTest;
|
import org.apache.hadoop.fs.contract.AbstractContractMultipartUploaderTest;
|
||||||
import org.apache.hadoop.fs.contract.AbstractFSContract;
|
import org.apache.hadoop.fs.contract.AbstractFSContract;
|
||||||
|
@ -27,6 +29,12 @@ import org.apache.hadoop.fs.contract.AbstractFSContract;
|
||||||
public class TestLocalFSContractMultipartUploader
|
public class TestLocalFSContractMultipartUploader
|
||||||
extends AbstractContractMultipartUploaderTest {
|
extends AbstractContractMultipartUploaderTest {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setup() throws Exception {
|
||||||
|
Assume.assumeTrue("Skipping until HDFS-13934", false);
|
||||||
|
super.setup();
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected AbstractFSContract createContract(Configuration conf) {
|
protected AbstractFSContract createContract(Configuration conf) {
|
||||||
return new LocalFSContract(conf);
|
return new LocalFSContract(conf);
|
||||||
|
|
|
@ -34,6 +34,7 @@ import java.util.Arrays;
|
||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.Enumeration;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
@ -191,6 +192,14 @@ public abstract class GenericTestUtils {
|
||||||
setLogLevel(LogManager.getRootLogger(), Level.toLevel(level.toString()));
|
setLogLevel(LogManager.getRootLogger(), Level.toLevel(level.toString()));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static void setCurrentLoggersLogLevel(org.slf4j.event.Level level) {
|
||||||
|
for (Enumeration<?> loggers = LogManager.getCurrentLoggers();
|
||||||
|
loggers.hasMoreElements();) {
|
||||||
|
Logger logger = (Logger) loggers.nextElement();
|
||||||
|
logger.setLevel(Level.toLevel(level.toString()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public static org.slf4j.event.Level toLevel(String level) {
|
public static org.slf4j.event.Level toLevel(String level) {
|
||||||
return toLevel(level, org.slf4j.event.Level.DEBUG);
|
return toLevel(level, org.slf4j.event.Level.DEBUG);
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,138 +19,148 @@
|
||||||
package org.apache.hadoop.util;
|
package org.apache.hadoop.util;
|
||||||
|
|
||||||
import java.io.ByteArrayInputStream;
|
import java.io.ByteArrayInputStream;
|
||||||
|
import java.nio.charset.StandardCharsets;
|
||||||
|
import java.util.Arrays;
|
||||||
|
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.util.LineReader;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import org.junit.Assert;
|
|
||||||
|
|
||||||
public class TestLineReader {
|
public class TestLineReader {
|
||||||
private LineReader lineReader;
|
|
||||||
private String TestData;
|
|
||||||
private String Delimiter;
|
|
||||||
private Text line;
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* TEST_1: The test scenario is the tail of the buffer equals the starting
|
||||||
|
* character/s of delimiter.
|
||||||
|
*
|
||||||
|
* The Test Data is such that,
|
||||||
|
*
|
||||||
|
* 1) we will have "</entity>" as delimiter
|
||||||
|
*
|
||||||
|
* 2) The tail of the current buffer would be "</" which matches with the
|
||||||
|
* starting character sequence of delimiter.
|
||||||
|
*
|
||||||
|
* 3) The Head of the next buffer would be "id>" which does NOT match with
|
||||||
|
* the remaining characters of delimiter.
|
||||||
|
*
|
||||||
|
* 4) Input data would be prefixed by char 'a' about
|
||||||
|
* numberOfCharToFillTheBuffer times. So that, one iteration to buffer the
|
||||||
|
* input data, would end at '</' ie equals starting 2 char of delimiter
|
||||||
|
*
|
||||||
|
* 5) For this we would take BufferSize as 64 * 1024;
|
||||||
|
*
|
||||||
|
* Check Condition In the second key value pair, the value should contain
|
||||||
|
* "</" from currentToken and "id>" from next token
|
||||||
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testCustomDelimiter() throws Exception {
|
public void testCustomDelimiter1() throws Exception {
|
||||||
/* TEST_1
|
|
||||||
* The test scenario is the tail of the buffer
|
|
||||||
* equals the starting character/s of delimiter
|
|
||||||
*
|
|
||||||
* The Test Data is such that,
|
|
||||||
*
|
|
||||||
* 1) we will have "</entity>" as delimiter
|
|
||||||
*
|
|
||||||
* 2) The tail of the current buffer would be "</"
|
|
||||||
* which matches with the starting character sequence of delimiter.
|
|
||||||
*
|
|
||||||
* 3) The Head of the next buffer would be "id>"
|
|
||||||
* which does NOT match with the remaining characters of delimiter.
|
|
||||||
*
|
|
||||||
* 4) Input data would be prefixed by char 'a'
|
|
||||||
* about numberOfCharToFillTheBuffer times.
|
|
||||||
* So that, one iteration to buffer the input data,
|
|
||||||
* would end at '</' ie equals starting 2 char of delimiter
|
|
||||||
*
|
|
||||||
* 5) For this we would take BufferSize as 64 * 1024;
|
|
||||||
*
|
|
||||||
* Check Condition
|
|
||||||
* In the second key value pair, the value should contain
|
|
||||||
* "</" from currentToken and
|
|
||||||
* "id>" from next token
|
|
||||||
*/
|
|
||||||
|
|
||||||
Delimiter="</entity>";
|
|
||||||
|
|
||||||
String CurrentBufferTailToken=
|
|
||||||
"</entity><entity><id>Gelesh</";
|
|
||||||
// Ending part of Input Data Buffer
|
|
||||||
// It contains '</' ie delimiter character
|
|
||||||
|
|
||||||
String NextBufferHeadToken=
|
|
||||||
"id><name>Omathil</name></entity>";
|
|
||||||
// Supposing the start of next buffer is this
|
|
||||||
|
|
||||||
String Expected =
|
|
||||||
(CurrentBufferTailToken+NextBufferHeadToken)
|
|
||||||
.replace(Delimiter, "");
|
|
||||||
// Expected ,must capture from both the buffer, excluding Delimiter
|
|
||||||
|
|
||||||
String TestPartOfInput = CurrentBufferTailToken+NextBufferHeadToken;
|
|
||||||
|
|
||||||
int BufferSize=64 * 1024;
|
|
||||||
int numberOfCharToFillTheBuffer =
|
|
||||||
BufferSize - CurrentBufferTailToken.length();
|
|
||||||
StringBuilder fillerString=new StringBuilder();
|
|
||||||
for (int i=0; i<numberOfCharToFillTheBuffer; i++) {
|
|
||||||
fillerString.append('a'); // char 'a' as a filler for the test string
|
|
||||||
}
|
|
||||||
|
|
||||||
TestData = fillerString + TestPartOfInput;
|
final String delimiter = "</entity>";
|
||||||
lineReader = new LineReader(
|
|
||||||
new ByteArrayInputStream(TestData.getBytes()), Delimiter.getBytes());
|
// Ending part of Input Data Buffer
|
||||||
|
// It contains '</' ie delimiter character
|
||||||
line = new Text();
|
final String currentBufferTailToken = "</entity><entity><id>Gelesh</";
|
||||||
|
|
||||||
|
// Supposing the start of next buffer is this
|
||||||
|
final String nextBufferHeadToken = "id><name>Omathil</name></entity>";
|
||||||
|
|
||||||
|
// Expected must capture from both the buffer, excluding Delimiter
|
||||||
|
final String expected =
|
||||||
|
(currentBufferTailToken + nextBufferHeadToken).replace(delimiter, "");
|
||||||
|
|
||||||
|
final String testPartOfInput = currentBufferTailToken + nextBufferHeadToken;
|
||||||
|
|
||||||
|
final int bufferSize = 64 * 1024;
|
||||||
|
int numberOfCharToFillTheBuffer =
|
||||||
|
bufferSize - currentBufferTailToken.length();
|
||||||
|
|
||||||
|
final char[] fillBuffer = new char[numberOfCharToFillTheBuffer];
|
||||||
|
|
||||||
|
// char 'a' as a filler for the test string
|
||||||
|
Arrays.fill(fillBuffer, 'a');
|
||||||
|
|
||||||
|
final StringBuilder fillerString = new StringBuilder();
|
||||||
|
|
||||||
|
final String testData = fillerString + testPartOfInput;
|
||||||
|
|
||||||
|
final LineReader lineReader = new LineReader(
|
||||||
|
new ByteArrayInputStream(testData.getBytes(StandardCharsets.UTF_8)),
|
||||||
|
delimiter.getBytes(StandardCharsets.UTF_8));
|
||||||
|
|
||||||
|
final Text line = new Text();
|
||||||
lineReader.readLine(line);
|
lineReader.readLine(line);
|
||||||
|
lineReader.close();
|
||||||
|
|
||||||
Assert.assertEquals(fillerString.toString(), line.toString());
|
Assert.assertEquals(fillerString.toString(), line.toString());
|
||||||
|
|
||||||
lineReader.readLine(line);
|
lineReader.readLine(line);
|
||||||
Assert.assertEquals(Expected, line.toString());
|
Assert.assertEquals(expected, line.toString());
|
||||||
|
}
|
||||||
/*TEST_2
|
|
||||||
* The test scenario is such that,
|
/**
|
||||||
* the character/s preceding the delimiter,
|
* TEST_2: The test scenario is such that, the character/s preceding the
|
||||||
* equals the starting character/s of delimiter
|
* delimiter, equals the starting character/s of delimiter.
|
||||||
*/
|
*/
|
||||||
|
@Test
|
||||||
Delimiter = "record";
|
public void testCustomDelimiter2() throws Exception {
|
||||||
StringBuilder TestStringBuilder = new StringBuilder();
|
final String delimiter = "record";
|
||||||
|
final StringBuilder testStringBuilder = new StringBuilder();
|
||||||
TestStringBuilder.append(Delimiter + "Kerala ");
|
|
||||||
TestStringBuilder.append(Delimiter + "Bangalore");
|
testStringBuilder.append(delimiter).append("Kerala ");
|
||||||
TestStringBuilder.append(Delimiter + " North Korea");
|
testStringBuilder.append(delimiter).append("Bangalore");
|
||||||
TestStringBuilder.append(Delimiter + Delimiter+
|
testStringBuilder.append(delimiter).append(" North Korea");
|
||||||
"Guantanamo");
|
testStringBuilder.append(delimiter).append(delimiter).append("Guantanamo");
|
||||||
TestStringBuilder.append(Delimiter + "ecord"
|
|
||||||
+ "recor" + "core"); //~EOF with 're'
|
// ~EOF with 're'
|
||||||
|
testStringBuilder.append(delimiter + "ecord" + "recor" + "core");
|
||||||
TestData=TestStringBuilder.toString();
|
|
||||||
|
final String testData = testStringBuilder.toString();
|
||||||
lineReader = new LineReader(
|
|
||||||
new ByteArrayInputStream(TestData.getBytes()), Delimiter.getBytes());
|
final LineReader lineReader = new LineReader(
|
||||||
|
new ByteArrayInputStream(testData.getBytes(StandardCharsets.UTF_8)),
|
||||||
|
delimiter.getBytes((StandardCharsets.UTF_8)));
|
||||||
|
|
||||||
|
final Text line = new Text();
|
||||||
|
|
||||||
lineReader.readLine(line);
|
lineReader.readLine(line);
|
||||||
Assert.assertEquals("", line.toString());
|
Assert.assertEquals("", line.toString());
|
||||||
lineReader.readLine(line);
|
lineReader.readLine(line);
|
||||||
Assert.assertEquals("Kerala ", line.toString());
|
Assert.assertEquals("Kerala ", line.toString());
|
||||||
|
|
||||||
lineReader.readLine(line);
|
|
||||||
Assert.assertEquals("Bangalore", line.toString());
|
|
||||||
|
|
||||||
lineReader.readLine(line);
|
|
||||||
Assert.assertEquals(" North Korea", line.toString());
|
|
||||||
|
|
||||||
lineReader.readLine(line);
|
|
||||||
Assert.assertEquals("", line.toString());
|
|
||||||
lineReader.readLine(line);
|
|
||||||
Assert.assertEquals("Guantanamo", line.toString());
|
|
||||||
|
|
||||||
lineReader.readLine(line);
|
|
||||||
Assert.assertEquals(("ecord"+"recor"+"core"), line.toString());
|
|
||||||
|
|
||||||
// Test 3
|
lineReader.readLine(line);
|
||||||
// The test scenario is such that,
|
Assert.assertEquals("Bangalore", line.toString());
|
||||||
// aaaabccc split by aaab
|
|
||||||
TestData = "aaaabccc";
|
lineReader.readLine(line);
|
||||||
Delimiter = "aaab";
|
Assert.assertEquals(" North Korea", line.toString());
|
||||||
lineReader = new LineReader(
|
|
||||||
new ByteArrayInputStream(TestData.getBytes()), Delimiter.getBytes());
|
lineReader.readLine(line);
|
||||||
|
Assert.assertEquals("", line.toString());
|
||||||
|
lineReader.readLine(line);
|
||||||
|
Assert.assertEquals("Guantanamo", line.toString());
|
||||||
|
|
||||||
|
lineReader.readLine(line);
|
||||||
|
Assert.assertEquals(("ecord" + "recor" + "core"), line.toString());
|
||||||
|
|
||||||
|
lineReader.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test 3: The test scenario is such that, aaabccc split by aaab.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testCustomDelimiter3() throws Exception {
|
||||||
|
final String testData = "aaaabccc";
|
||||||
|
final String delimiter = "aaab";
|
||||||
|
final LineReader lineReader = new LineReader(
|
||||||
|
new ByteArrayInputStream(testData.getBytes(StandardCharsets.UTF_8)),
|
||||||
|
delimiter.getBytes(StandardCharsets.UTF_8));
|
||||||
|
|
||||||
|
final Text line = new Text();
|
||||||
|
|
||||||
lineReader.readLine(line);
|
lineReader.readLine(line);
|
||||||
Assert.assertEquals("a", line.toString());
|
Assert.assertEquals("a", line.toString());
|
||||||
lineReader.readLine(line);
|
lineReader.readLine(line);
|
||||||
Assert.assertEquals("ccc", line.toString());
|
Assert.assertEquals("ccc", line.toString());
|
||||||
|
|
||||||
|
lineReader.close();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -214,14 +214,14 @@ public class XceiverClientGrpc extends XceiverClientSpi {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public XceiverClientReply sendCommand(
|
public XceiverClientReply sendCommand(
|
||||||
ContainerCommandRequestProto request, List<UUID> excludeDns)
|
ContainerCommandRequestProto request, List<DatanodeDetails> excludeDns)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Preconditions.checkState(HddsUtils.isReadOnly(request));
|
Preconditions.checkState(HddsUtils.isReadOnly(request));
|
||||||
return sendCommandWithRetry(request, excludeDns);
|
return sendCommandWithRetry(request, excludeDns);
|
||||||
}
|
}
|
||||||
|
|
||||||
private XceiverClientReply sendCommandWithRetry(
|
private XceiverClientReply sendCommandWithRetry(
|
||||||
ContainerCommandRequestProto request, List<UUID> excludeDns)
|
ContainerCommandRequestProto request, List<DatanodeDetails> excludeDns)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
ContainerCommandResponseProto responseProto = null;
|
ContainerCommandResponseProto responseProto = null;
|
||||||
|
|
||||||
|
@ -231,24 +231,24 @@ public class XceiverClientGrpc extends XceiverClientSpi {
|
||||||
// TODO: cache the correct leader info in here, so that any subsequent calls
|
// TODO: cache the correct leader info in here, so that any subsequent calls
|
||||||
// should first go to leader
|
// should first go to leader
|
||||||
List<DatanodeDetails> dns = pipeline.getNodes();
|
List<DatanodeDetails> dns = pipeline.getNodes();
|
||||||
DatanodeDetails datanode = null;
|
|
||||||
List<DatanodeDetails> healthyDns =
|
List<DatanodeDetails> healthyDns =
|
||||||
excludeDns != null ? dns.stream().filter(dnId -> {
|
excludeDns != null ? dns.stream().filter(dnId -> {
|
||||||
for (UUID excludeId : excludeDns) {
|
for (DatanodeDetails excludeId : excludeDns) {
|
||||||
if (dnId.getUuid().equals(excludeId)) {
|
if (dnId.equals(excludeId)) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return true;
|
return true;
|
||||||
}).collect(Collectors.toList()) : dns;
|
}).collect(Collectors.toList()) : dns;
|
||||||
|
XceiverClientReply reply = new XceiverClientReply(null);
|
||||||
for (DatanodeDetails dn : healthyDns) {
|
for (DatanodeDetails dn : healthyDns) {
|
||||||
try {
|
try {
|
||||||
LOG.debug("Executing command " + request + " on datanode " + dn);
|
LOG.debug("Executing command " + request + " on datanode " + dn);
|
||||||
// In case the command gets retried on a 2nd datanode,
|
// In case the command gets retried on a 2nd datanode,
|
||||||
// sendCommandAsyncCall will create a new channel and async stub
|
// sendCommandAsyncCall will create a new channel and async stub
|
||||||
// in case these don't exist for the specific datanode.
|
// in case these don't exist for the specific datanode.
|
||||||
|
reply.addDatanode(dn);
|
||||||
responseProto = sendCommandAsync(request, dn).getResponse().get();
|
responseProto = sendCommandAsync(request, dn).getResponse().get();
|
||||||
datanode = dn;
|
|
||||||
if (responseProto.getResult() == ContainerProtos.Result.SUCCESS) {
|
if (responseProto.getResult() == ContainerProtos.Result.SUCCESS) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
@ -264,8 +264,8 @@ public class XceiverClientGrpc extends XceiverClientSpi {
|
||||||
}
|
}
|
||||||
|
|
||||||
if (responseProto != null) {
|
if (responseProto != null) {
|
||||||
return new XceiverClientReply(
|
reply.setResponse(CompletableFuture.completedFuture(responseProto));
|
||||||
CompletableFuture.completedFuture(responseProto), datanode.getUuid());
|
return reply;
|
||||||
} else {
|
} else {
|
||||||
throw new IOException(
|
throw new IOException(
|
||||||
"Failed to execute command " + request + " on the pipeline "
|
"Failed to execute command " + request + " on the pipeline "
|
||||||
|
@ -382,11 +382,11 @@ public class XceiverClientGrpc extends XceiverClientSpi {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long watchForCommit(long index, long timeout)
|
public XceiverClientReply watchForCommit(long index, long timeout)
|
||||||
throws InterruptedException, ExecutionException, TimeoutException,
|
throws InterruptedException, ExecutionException, TimeoutException,
|
||||||
IOException {
|
IOException {
|
||||||
// there is no notion of watch for commit index in standalone pipeline
|
// there is no notion of watch for commit index in standalone pipeline
|
||||||
return 0;
|
return null;
|
||||||
};
|
};
|
||||||
|
|
||||||
public long getReplicatedMinCommitIndex() {
|
public long getReplicatedMinCommitIndex() {
|
||||||
|
|
|
@ -18,8 +18,8 @@
|
||||||
|
|
||||||
package org.apache.hadoop.hdds.scm;
|
package org.apache.hadoop.hdds.scm;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
|
||||||
import org.apache.hadoop.hdds.HddsUtils;
|
import org.apache.hadoop.hdds.HddsUtils;
|
||||||
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
||||||
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
||||||
|
|
||||||
|
@ -59,6 +59,7 @@ import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An abstract implementation of {@link XceiverClientSpi} using Ratis.
|
* An abstract implementation of {@link XceiverClientSpi} using Ratis.
|
||||||
|
@ -91,7 +92,7 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
||||||
private final GrpcTlsConfig tlsConfig;
|
private final GrpcTlsConfig tlsConfig;
|
||||||
|
|
||||||
// Map to track commit index at every server
|
// Map to track commit index at every server
|
||||||
private final ConcurrentHashMap<String, Long> commitInfoMap;
|
private final ConcurrentHashMap<UUID, Long> commitInfoMap;
|
||||||
|
|
||||||
// create a separate RaftClient for watchForCommit API
|
// create a separate RaftClient for watchForCommit API
|
||||||
private RaftClient watchClient;
|
private RaftClient watchClient;
|
||||||
|
@ -118,7 +119,8 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
||||||
// of the servers
|
// of the servers
|
||||||
if (commitInfoMap.isEmpty()) {
|
if (commitInfoMap.isEmpty()) {
|
||||||
commitInfoProtos.forEach(proto -> commitInfoMap
|
commitInfoProtos.forEach(proto -> commitInfoMap
|
||||||
.put(proto.getServer().getAddress(), proto.getCommitIndex()));
|
.put(RatisHelper.toDatanodeId(proto.getServer()),
|
||||||
|
proto.getCommitIndex()));
|
||||||
// In case the commit is happening 2 way, just update the commitIndex
|
// In case the commit is happening 2 way, just update the commitIndex
|
||||||
// for the servers which have been successfully updating the commit
|
// for the servers which have been successfully updating the commit
|
||||||
// indexes. This is important because getReplicatedMinCommitIndex()
|
// indexes. This is important because getReplicatedMinCommitIndex()
|
||||||
|
@ -126,7 +128,7 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
||||||
// been replicating data successfully.
|
// been replicating data successfully.
|
||||||
} else {
|
} else {
|
||||||
commitInfoProtos.forEach(proto -> commitInfoMap
|
commitInfoProtos.forEach(proto -> commitInfoMap
|
||||||
.computeIfPresent(proto.getServer().getAddress(),
|
.computeIfPresent(RatisHelper.toDatanodeId(proto.getServer()),
|
||||||
(address, index) -> {
|
(address, index) -> {
|
||||||
index = proto.getCommitIndex();
|
index = proto.getCommitIndex();
|
||||||
return index;
|
return index;
|
||||||
|
@ -218,15 +220,23 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
||||||
return minIndex.isPresent() ? minIndex.getAsLong() : 0;
|
return minIndex.isPresent() ? minIndex.getAsLong() : 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void addDatanodetoReply(UUID address, XceiverClientReply reply) {
|
||||||
|
DatanodeDetails.Builder builder = DatanodeDetails.newBuilder();
|
||||||
|
builder.setUuid(address.toString());
|
||||||
|
reply.addDatanode(builder.build());
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long watchForCommit(long index, long timeout)
|
public XceiverClientReply watchForCommit(long index, long timeout)
|
||||||
throws InterruptedException, ExecutionException, TimeoutException,
|
throws InterruptedException, ExecutionException, TimeoutException,
|
||||||
IOException {
|
IOException {
|
||||||
long commitIndex = getReplicatedMinCommitIndex();
|
long commitIndex = getReplicatedMinCommitIndex();
|
||||||
|
XceiverClientReply clientReply = new XceiverClientReply(null);
|
||||||
if (commitIndex >= index) {
|
if (commitIndex >= index) {
|
||||||
// return the min commit index till which the log has been replicated to
|
// return the min commit index till which the log has been replicated to
|
||||||
// all servers
|
// all servers
|
||||||
return commitIndex;
|
clientReply.setLogIndex(commitIndex);
|
||||||
|
return clientReply;
|
||||||
}
|
}
|
||||||
LOG.debug("commit index : {} watch timeout : {}", index, timeout);
|
LOG.debug("commit index : {} watch timeout : {}", index, timeout);
|
||||||
// create a new RaftClient instance for watch request
|
// create a new RaftClient instance for watch request
|
||||||
|
@ -250,26 +260,30 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
||||||
// TODO : need to remove the code to create the new RaftClient instance
|
// TODO : need to remove the code to create the new RaftClient instance
|
||||||
// here once the watch request bypassing sliding window in Raft Client
|
// here once the watch request bypassing sliding window in Raft Client
|
||||||
// gets fixed.
|
// gets fixed.
|
||||||
watchClient =
|
watchClient = RatisHelper
|
||||||
RatisHelper.newRaftClient(rpcType, getPipeline(), retryPolicy,
|
.newRaftClient(rpcType, getPipeline(), retryPolicy,
|
||||||
maxOutstandingRequests, tlsConfig);
|
maxOutstandingRequests, tlsConfig);
|
||||||
reply = watchClient
|
reply = watchClient
|
||||||
.sendWatchAsync(index, RaftProtos.ReplicationLevel.MAJORITY_COMMITTED)
|
.sendWatchAsync(index, RaftProtos.ReplicationLevel.MAJORITY_COMMITTED)
|
||||||
.get(timeout, TimeUnit.MILLISECONDS);
|
.get(timeout, TimeUnit.MILLISECONDS);
|
||||||
Optional<RaftProtos.CommitInfoProto>
|
List<RaftProtos.CommitInfoProto> commitInfoProtoList =
|
||||||
proto = reply.getCommitInfos().stream().min(Comparator.comparing(
|
reply.getCommitInfos().stream()
|
||||||
RaftProtos.CommitInfoProto :: getCommitIndex));
|
.filter(i -> i.getCommitIndex() < index)
|
||||||
Preconditions.checkState(proto.isPresent());
|
.collect(Collectors.toList());
|
||||||
String address = proto.get().getServer().getAddress();
|
commitInfoProtoList.parallelStream().forEach(proto -> {
|
||||||
// since 3 way commit has failed, the updated map from now on will
|
UUID address = RatisHelper.toDatanodeId(proto.getServer());
|
||||||
// only store entries for those datanodes which have had successful
|
addDatanodetoReply(address, clientReply);
|
||||||
// replication.
|
// since 3 way commit has failed, the updated map from now on will
|
||||||
commitInfoMap.remove(address);
|
// only store entries for those datanodes which have had successful
|
||||||
LOG.info(
|
// replication.
|
||||||
"Could not commit " + index + " to all the nodes. Server " + address
|
commitInfoMap.remove(address);
|
||||||
+ " has failed." + " Committed by majority.");
|
LOG.info(
|
||||||
|
"Could not commit " + index + " to all the nodes. Server " + address
|
||||||
|
+ " has failed." + " Committed by majority.");
|
||||||
|
});
|
||||||
}
|
}
|
||||||
return index;
|
clientReply.setLogIndex(index);
|
||||||
|
return clientReply;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -296,17 +310,28 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
||||||
RaftRetryFailureException raftRetryFailureException =
|
RaftRetryFailureException raftRetryFailureException =
|
||||||
reply.getRetryFailureException();
|
reply.getRetryFailureException();
|
||||||
if (raftRetryFailureException != null) {
|
if (raftRetryFailureException != null) {
|
||||||
|
// in case of raft retry failure, the raft client is
|
||||||
|
// not able to connect to the leader hence the pipeline
|
||||||
|
// can not be used but this instance of RaftClient will close
|
||||||
|
// and refreshed again. In case the client cannot connect to
|
||||||
|
// leader, getClient call will fail.
|
||||||
|
|
||||||
|
// No need to set the failed Server ID here. Ozone client
|
||||||
|
// will directly exclude this pipeline in next allocate block
|
||||||
|
// to SCM as in this case, it is the raft client which is not
|
||||||
|
// able to connect to leader in the pipeline, though the
|
||||||
|
// pipeline can still be functional.
|
||||||
throw new CompletionException(raftRetryFailureException);
|
throw new CompletionException(raftRetryFailureException);
|
||||||
}
|
}
|
||||||
ContainerCommandResponseProto response =
|
ContainerCommandResponseProto response =
|
||||||
ContainerCommandResponseProto
|
ContainerCommandResponseProto
|
||||||
.parseFrom(reply.getMessage().getContent());
|
.parseFrom(reply.getMessage().getContent());
|
||||||
|
UUID serverId = RatisHelper.toDatanodeId(reply.getReplierId());
|
||||||
if (response.getResult() == ContainerProtos.Result.SUCCESS) {
|
if (response.getResult() == ContainerProtos.Result.SUCCESS) {
|
||||||
updateCommitInfosMap(reply.getCommitInfos());
|
updateCommitInfosMap(reply.getCommitInfos());
|
||||||
asyncReply.setLogIndex(reply.getLogIndex());
|
|
||||||
asyncReply.setDatanode(
|
|
||||||
RatisHelper.toDatanodeId(reply.getReplierId()));
|
|
||||||
}
|
}
|
||||||
|
asyncReply.setLogIndex(reply.getLogIndex());
|
||||||
|
addDatanodetoReply(serverId, asyncReply);
|
||||||
return response;
|
return response;
|
||||||
} catch (InvalidProtocolBufferException e) {
|
} catch (InvalidProtocolBufferException e) {
|
||||||
throw new CompletionException(e);
|
throw new CompletionException(e);
|
||||||
|
|
|
@ -42,7 +42,6 @@ import java.nio.ByteBuffer;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.UUID;
|
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -290,7 +289,7 @@ public class BlockInputStream extends InputStream implements Seekable {
|
||||||
XceiverClientReply reply;
|
XceiverClientReply reply;
|
||||||
ReadChunkResponseProto readChunkResponse = null;
|
ReadChunkResponseProto readChunkResponse = null;
|
||||||
final ChunkInfo chunkInfo = chunks.get(chunkIndex);
|
final ChunkInfo chunkInfo = chunks.get(chunkIndex);
|
||||||
List<UUID> excludeDns = null;
|
List<DatanodeDetails> excludeDns = null;
|
||||||
ByteString byteString;
|
ByteString byteString;
|
||||||
List<DatanodeDetails> dnList = xceiverClient.getPipeline().getNodes();
|
List<DatanodeDetails> dnList = xceiverClient.getPipeline().getNodes();
|
||||||
while (true) {
|
while (true) {
|
||||||
|
@ -334,7 +333,7 @@ public class BlockInputStream extends InputStream implements Seekable {
|
||||||
if (excludeDns == null) {
|
if (excludeDns == null) {
|
||||||
excludeDns = new ArrayList<>();
|
excludeDns = new ArrayList<>();
|
||||||
}
|
}
|
||||||
excludeDns.add(reply.getDatanode());
|
excludeDns.addAll(reply.getDatanodes());
|
||||||
if (excludeDns.size() == dnList.size()) {
|
if (excludeDns.size() == dnList.size()) {
|
||||||
throw ioe;
|
throw ioe;
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,6 +18,7 @@
|
||||||
|
|
||||||
package org.apache.hadoop.hdds.scm.storage;
|
package org.apache.hadoop.hdds.scm.storage;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
||||||
import org.apache.hadoop.hdds.scm.XceiverClientReply;
|
import org.apache.hadoop.hdds.scm.XceiverClientReply;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
|
||||||
|
@ -41,6 +42,7 @@ import java.io.IOException;
|
||||||
import java.io.OutputStream;
|
import java.io.OutputStream;
|
||||||
import java.nio.Buffer;
|
import java.nio.Buffer;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
@ -102,14 +104,17 @@ public class BlockOutputStream extends OutputStream {
|
||||||
// by all servers
|
// by all servers
|
||||||
private long totalAckDataLength;
|
private long totalAckDataLength;
|
||||||
|
|
||||||
// list to hold up all putBlock futures
|
// future Map to hold up all putBlock futures
|
||||||
private List<CompletableFuture<ContainerProtos.ContainerCommandResponseProto>>
|
private ConcurrentHashMap<Long,
|
||||||
futureList;
|
CompletableFuture<ContainerProtos.ContainerCommandResponseProto>>
|
||||||
|
futureMap;
|
||||||
// map containing mapping for putBlock logIndex to to flushedDataLength Map.
|
// map containing mapping for putBlock logIndex to to flushedDataLength Map.
|
||||||
private ConcurrentHashMap<Long, Long> commitIndex2flushedDataMap;
|
private ConcurrentHashMap<Long, Long> commitIndex2flushedDataMap;
|
||||||
|
|
||||||
private int currentBufferIndex;
|
private int currentBufferIndex;
|
||||||
|
|
||||||
|
private List<DatanodeDetails> failedServers;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a new BlockOutputStream.
|
* Creates a new BlockOutputStream.
|
||||||
*
|
*
|
||||||
|
@ -157,10 +162,11 @@ public class BlockOutputStream extends OutputStream {
|
||||||
responseExecutor = Executors.newSingleThreadExecutor();
|
responseExecutor = Executors.newSingleThreadExecutor();
|
||||||
commitIndex2flushedDataMap = new ConcurrentHashMap<>();
|
commitIndex2flushedDataMap = new ConcurrentHashMap<>();
|
||||||
totalAckDataLength = 0;
|
totalAckDataLength = 0;
|
||||||
futureList = new ArrayList<>();
|
futureMap = new ConcurrentHashMap<>();
|
||||||
totalDataFlushedLength = 0;
|
totalDataFlushedLength = 0;
|
||||||
currentBufferIndex = 0;
|
currentBufferIndex = 0;
|
||||||
writtenDataLength = 0;
|
writtenDataLength = 0;
|
||||||
|
failedServers = Collections.emptyList();
|
||||||
}
|
}
|
||||||
|
|
||||||
public BlockID getBlockID() {
|
public BlockID getBlockID() {
|
||||||
|
@ -182,6 +188,9 @@ public class BlockOutputStream extends OutputStream {
|
||||||
return dataLength;
|
return dataLength;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public List<DatanodeDetails> getFailedServers() {
|
||||||
|
return failedServers;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void write(int b) throws IOException {
|
public void write(int b) throws IOException {
|
||||||
|
@ -299,7 +308,7 @@ public class BlockOutputStream extends OutputStream {
|
||||||
Preconditions.checkState(commitIndex2flushedDataMap.containsKey(index));
|
Preconditions.checkState(commitIndex2flushedDataMap.containsKey(index));
|
||||||
totalAckDataLength = commitIndex2flushedDataMap.remove(index);
|
totalAckDataLength = commitIndex2flushedDataMap.remove(index);
|
||||||
LOG.debug("Total data successfully replicated: " + totalAckDataLength);
|
LOG.debug("Total data successfully replicated: " + totalAckDataLength);
|
||||||
futureList.remove(0);
|
futureMap.remove(totalAckDataLength);
|
||||||
// Flush has been committed to required servers successful.
|
// Flush has been committed to required servers successful.
|
||||||
// just swap the bufferList head and tail after clearing.
|
// just swap the bufferList head and tail after clearing.
|
||||||
ByteBuffer currentBuffer = bufferList.remove(0);
|
ByteBuffer currentBuffer = bufferList.remove(0);
|
||||||
|
@ -320,7 +329,7 @@ public class BlockOutputStream extends OutputStream {
|
||||||
private void handleFullBuffer() throws IOException {
|
private void handleFullBuffer() throws IOException {
|
||||||
try {
|
try {
|
||||||
checkOpen();
|
checkOpen();
|
||||||
if (!futureList.isEmpty()) {
|
if (!futureMap.isEmpty()) {
|
||||||
waitOnFlushFutures();
|
waitOnFlushFutures();
|
||||||
}
|
}
|
||||||
} catch (InterruptedException | ExecutionException e) {
|
} catch (InterruptedException | ExecutionException e) {
|
||||||
|
@ -362,9 +371,22 @@ public class BlockOutputStream extends OutputStream {
|
||||||
private void watchForCommit(long commitIndex) throws IOException {
|
private void watchForCommit(long commitIndex) throws IOException {
|
||||||
checkOpen();
|
checkOpen();
|
||||||
Preconditions.checkState(!commitIndex2flushedDataMap.isEmpty());
|
Preconditions.checkState(!commitIndex2flushedDataMap.isEmpty());
|
||||||
|
long index;
|
||||||
try {
|
try {
|
||||||
long index =
|
XceiverClientReply reply =
|
||||||
xceiverClient.watchForCommit(commitIndex, watchTimeout);
|
xceiverClient.watchForCommit(commitIndex, watchTimeout);
|
||||||
|
if (reply == null) {
|
||||||
|
index = 0;
|
||||||
|
} else {
|
||||||
|
List<DatanodeDetails> dnList = reply.getDatanodes();
|
||||||
|
if (!dnList.isEmpty()) {
|
||||||
|
if (failedServers.isEmpty()) {
|
||||||
|
failedServers = new ArrayList<>();
|
||||||
|
}
|
||||||
|
failedServers.addAll(dnList);
|
||||||
|
}
|
||||||
|
index = reply.getLogIndex();
|
||||||
|
}
|
||||||
adjustBuffers(index);
|
adjustBuffers(index);
|
||||||
} catch (TimeoutException | InterruptedException | ExecutionException e) {
|
} catch (TimeoutException | InterruptedException | ExecutionException e) {
|
||||||
LOG.warn("watchForCommit failed for index " + commitIndex, e);
|
LOG.warn("watchForCommit failed for index " + commitIndex, e);
|
||||||
|
@ -392,8 +414,7 @@ public class BlockOutputStream extends OutputStream {
|
||||||
try {
|
try {
|
||||||
validateResponse(e);
|
validateResponse(e);
|
||||||
} catch (IOException sce) {
|
} catch (IOException sce) {
|
||||||
future.completeExceptionally(sce);
|
throw new CompletionException(sce);
|
||||||
return e;
|
|
||||||
}
|
}
|
||||||
// if the ioException is not set, putBlock is successful
|
// if the ioException is not set, putBlock is successful
|
||||||
if (ioException == null) {
|
if (ioException == null) {
|
||||||
|
@ -422,7 +443,7 @@ public class BlockOutputStream extends OutputStream {
|
||||||
throw new IOException(
|
throw new IOException(
|
||||||
"Unexpected Storage Container Exception: " + e.toString(), e);
|
"Unexpected Storage Container Exception: " + e.toString(), e);
|
||||||
}
|
}
|
||||||
futureList.add(flushFuture);
|
futureMap.put(flushPos, flushFuture);
|
||||||
return flushFuture;
|
return flushFuture;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -516,8 +537,8 @@ public class BlockOutputStream extends OutputStream {
|
||||||
|
|
||||||
private void waitOnFlushFutures()
|
private void waitOnFlushFutures()
|
||||||
throws InterruptedException, ExecutionException {
|
throws InterruptedException, ExecutionException {
|
||||||
CompletableFuture<Void> combinedFuture = CompletableFuture
|
CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(
|
||||||
.allOf(futureList.toArray(new CompletableFuture[futureList.size()]));
|
futureMap.values().toArray(new CompletableFuture[futureMap.size()]));
|
||||||
// wait for all the transactions to complete
|
// wait for all the transactions to complete
|
||||||
combinedFuture.get();
|
combinedFuture.get();
|
||||||
}
|
}
|
||||||
|
@ -553,10 +574,10 @@ public class BlockOutputStream extends OutputStream {
|
||||||
}
|
}
|
||||||
xceiverClientManager = null;
|
xceiverClientManager = null;
|
||||||
xceiverClient = null;
|
xceiverClient = null;
|
||||||
if (futureList != null) {
|
if (futureMap != null) {
|
||||||
futureList.clear();
|
futureMap.clear();
|
||||||
}
|
}
|
||||||
futureList = null;
|
futureMap = null;
|
||||||
if (commitIndex2flushedDataMap != null) {
|
if (commitIndex2flushedDataMap != null) {
|
||||||
commitIndex2flushedDataMap.clear();
|
commitIndex2flushedDataMap.clear();
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,6 +17,9 @@
|
||||||
package org.apache.hadoop.hdds;
|
package org.apache.hadoop.hdds;
|
||||||
|
|
||||||
import org.apache.hadoop.utils.db.DBProfile;
|
import org.apache.hadoop.utils.db.DBProfile;
|
||||||
|
import org.apache.ratis.util.TimeDuration;
|
||||||
|
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class contains constants for configuration keys and default values
|
* This class contains constants for configuration keys and default values
|
||||||
|
@ -70,6 +73,14 @@ public final class HddsConfigKeys {
|
||||||
"hdds.scm.chillmode.min.datanode";
|
"hdds.scm.chillmode.min.datanode";
|
||||||
public static final int HDDS_SCM_CHILLMODE_MIN_DATANODE_DEFAULT = 1;
|
public static final int HDDS_SCM_CHILLMODE_MIN_DATANODE_DEFAULT = 1;
|
||||||
|
|
||||||
|
|
||||||
|
public static final String
|
||||||
|
HDDS_SCM_WAIT_TIME_AFTER_CHILL_MODE_EXIT =
|
||||||
|
"hdds.scm.wait.time.after.chillmode.exit";
|
||||||
|
|
||||||
|
public static final String
|
||||||
|
HDDS_SCM_WAIT_TIME_AFTER_CHILL_MODE_EXIT_DEFAULT = "5m";
|
||||||
|
|
||||||
public static final String HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK =
|
public static final String HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK =
|
||||||
"hdds.scm.chillmode.pipeline-availability.check";
|
"hdds.scm.chillmode.pipeline-availability.check";
|
||||||
public static final boolean
|
public static final boolean
|
||||||
|
@ -89,6 +100,11 @@ public final class HddsConfigKeys {
|
||||||
public static final double
|
public static final double
|
||||||
HDDS_SCM_CHILLMODE_HEALTHY_PIPELINE_THRESHOLD_PCT_DEFAULT = 0.10;
|
HDDS_SCM_CHILLMODE_HEALTHY_PIPELINE_THRESHOLD_PCT_DEFAULT = 0.10;
|
||||||
|
|
||||||
|
public static final String HDDS_SCM_CHILLMODE_ONE_NODE_REPORTED_PIPELINE_PCT =
|
||||||
|
"hdds.scm.chillmode.atleast.one.node.reported.pipeline.pct";
|
||||||
|
public static final double
|
||||||
|
HDDS_SCM_CHILLMODE_ONE_NODE_REPORTED_PIPELINE_PCT_DEFAULT = 0.90;
|
||||||
|
|
||||||
public static final String HDDS_LOCK_MAX_CONCURRENCY =
|
public static final String HDDS_LOCK_MAX_CONCURRENCY =
|
||||||
"hdds.lock.max.concurrency";
|
"hdds.lock.max.concurrency";
|
||||||
public static final int HDDS_LOCK_MAX_CONCURRENCY_DEFAULT = 100;
|
public static final int HDDS_LOCK_MAX_CONCURRENCY_DEFAULT = 100;
|
||||||
|
@ -222,4 +238,24 @@ public final class HddsConfigKeys {
|
||||||
public static final String HDDS_SECURITY_CLIENT_SCM_CERTIFICATE_PROTOCOL_ACL =
|
public static final String HDDS_SECURITY_CLIENT_SCM_CERTIFICATE_PROTOCOL_ACL =
|
||||||
"hdds.security.client.scm.certificate.protocol.acl";
|
"hdds.security.client.scm.certificate.protocol.acl";
|
||||||
|
|
||||||
|
public static final String HDDS_DATANODE_HTTP_ENABLED_KEY =
|
||||||
|
"hdds.datanode.http.enabled";
|
||||||
|
public static final String HDDS_DATANODE_HTTP_BIND_HOST_KEY =
|
||||||
|
"hdds.datanode.http-bind-host";
|
||||||
|
public static final String HDDS_DATANODE_HTTPS_BIND_HOST_KEY =
|
||||||
|
"hdds.datanode.https-bind-host";
|
||||||
|
public static final String HDDS_DATANODE_HTTP_ADDRESS_KEY =
|
||||||
|
"hdds.datanode.http-address";
|
||||||
|
public static final String HDDS_DATANODE_HTTPS_ADDRESS_KEY =
|
||||||
|
"hdds.datanode.https-address";
|
||||||
|
|
||||||
|
public static final String HDDS_DATANODE_HTTP_BIND_HOST_DEFAULT = "0.0.0.0";
|
||||||
|
public static final int HDDS_DATANODE_HTTP_BIND_PORT_DEFAULT = 9882;
|
||||||
|
public static final int HDDS_DATANODE_HTTPS_BIND_PORT_DEFAULT = 9883;
|
||||||
|
public static final String
|
||||||
|
HDDS_DATANODE_HTTP_KERBEROS_PRINCIPAL_KEY =
|
||||||
|
"hdds.datanode.http.kerberos.principal";
|
||||||
|
public static final String
|
||||||
|
HDDS_DATANODE_HTTP_KERBEROS_KEYTAB_FILE_KEY =
|
||||||
|
"hdds.datanode.http.kerberos.keytab";
|
||||||
}
|
}
|
|
@ -19,6 +19,7 @@
|
||||||
package org.apache.hadoop.hdds;
|
package org.apache.hadoop.hdds;
|
||||||
|
|
||||||
import javax.management.ObjectName;
|
import javax.management.ObjectName;
|
||||||
|
import java.io.IOException;
|
||||||
import java.lang.reflect.InvocationTargetException;
|
import java.lang.reflect.InvocationTargetException;
|
||||||
import java.lang.reflect.Method;
|
import java.lang.reflect.Method;
|
||||||
import java.net.InetSocketAddress;
|
import java.net.InetSocketAddress;
|
||||||
|
@ -36,7 +37,15 @@ import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
||||||
|
import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
|
||||||
|
import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolPB;
|
||||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||||
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
|
import org.apache.hadoop.hdds.protocol.SCMSecurityProtocol;
|
||||||
|
import org.apache.hadoop.hdds.scm.protocolPB.ScmBlockLocationProtocolPB;
|
||||||
|
import org.apache.hadoop.ipc.Client;
|
||||||
|
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
||||||
|
import org.apache.hadoop.ipc.RPC;
|
||||||
import org.apache.hadoop.metrics2.util.MBeans;
|
import org.apache.hadoop.metrics2.util.MBeans;
|
||||||
import org.apache.hadoop.net.DNS;
|
import org.apache.hadoop.net.DNS;
|
||||||
import org.apache.hadoop.net.NetUtils;
|
import org.apache.hadoop.net.NetUtils;
|
||||||
|
@ -48,6 +57,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_NAMESERVER_K
|
||||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY;
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY;
|
||||||
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED;
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED;
|
||||||
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED_DEFAULT;
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED_DEFAULT;
|
||||||
|
|
||||||
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
@ -161,6 +172,29 @@ public final class HddsUtils {
|
||||||
.orElse(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_PORT_DEFAULT));
|
.orElse(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_PORT_DEFAULT));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a scm security client.
|
||||||
|
* @param conf - Ozone configuration.
|
||||||
|
* @param address - inet socket address of scm.
|
||||||
|
*
|
||||||
|
* @return {@link SCMSecurityProtocol}
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public static SCMSecurityProtocol getScmSecurityClient(
|
||||||
|
OzoneConfiguration conf, InetSocketAddress address) throws IOException {
|
||||||
|
RPC.setProtocolEngine(conf, SCMSecurityProtocolPB.class,
|
||||||
|
ProtobufRpcEngine.class);
|
||||||
|
long scmVersion =
|
||||||
|
RPC.getProtocolVersion(ScmBlockLocationProtocolPB.class);
|
||||||
|
SCMSecurityProtocolClientSideTranslatorPB scmSecurityClient =
|
||||||
|
new SCMSecurityProtocolClientSideTranslatorPB(
|
||||||
|
RPC.getProxy(SCMSecurityProtocolPB.class, scmVersion,
|
||||||
|
address, UserGroupInformation.getCurrentUser(),
|
||||||
|
conf, NetUtils.getDefaultSocketFactory(conf),
|
||||||
|
Client.getRpcTimeout(conf)));
|
||||||
|
return scmSecurityClient;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Retrieve the hostname, trying the supplied config keys in order.
|
* Retrieve the hostname, trying the supplied config keys in order.
|
||||||
* Each config value may be absent, or if present in the format
|
* Each config value may be absent, or if present in the format
|
||||||
|
|
|
@ -85,8 +85,7 @@ public class GenericCli implements Callable<Void>, GenericParentCommand {
|
||||||
OzoneConfiguration ozoneConf = new OzoneConfiguration();
|
OzoneConfiguration ozoneConf = new OzoneConfiguration();
|
||||||
if (configurationOverrides != null) {
|
if (configurationOverrides != null) {
|
||||||
for (Entry<String, String> entry : configurationOverrides.entrySet()) {
|
for (Entry<String, String> entry : configurationOverrides.entrySet()) {
|
||||||
ozoneConf
|
ozoneConf.set(entry.getKey(), entry.getValue());
|
||||||
.set(entry.getKey(), entry.getValue());
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return ozoneConf;
|
return ozoneConf;
|
||||||
|
|
|
@ -30,7 +30,9 @@ import javax.xml.bind.annotation.XmlElement;
|
||||||
import javax.xml.bind.annotation.XmlRootElement;
|
import javax.xml.bind.annotation.XmlRootElement;
|
||||||
import java.net.URL;
|
import java.net.URL;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Enumeration;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Properties;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Configuration for ozone.
|
* Configuration for ozone.
|
||||||
|
@ -161,4 +163,31 @@ public class OzoneConfiguration extends Configuration {
|
||||||
Configuration.addDefaultResource("ozone-default.xml");
|
Configuration.addDefaultResource("ozone-default.xml");
|
||||||
Configuration.addDefaultResource("ozone-site.xml");
|
Configuration.addDefaultResource("ozone-site.xml");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The super class method getAllPropertiesByTag
|
||||||
|
* does not override values of properties
|
||||||
|
* if there is no tag present in the configs of
|
||||||
|
* newly added resources.
|
||||||
|
* @param tag
|
||||||
|
* @return Properties that belong to the tag
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public Properties getAllPropertiesByTag(String tag) {
|
||||||
|
// Call getProps first to load the newly added resources
|
||||||
|
// before calling super.getAllPropertiesByTag
|
||||||
|
Properties updatedProps = getProps();
|
||||||
|
Properties propertiesByTag = super.getAllPropertiesByTag(tag);
|
||||||
|
Properties props = new Properties();
|
||||||
|
Enumeration properties = propertiesByTag.propertyNames();
|
||||||
|
while (properties.hasMoreElements()) {
|
||||||
|
Object propertyName = properties.nextElement();
|
||||||
|
// get the current value of the property
|
||||||
|
Object value = updatedProps.getProperty(propertyName.toString());
|
||||||
|
if (value != null) {
|
||||||
|
props.put(propertyName, value);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return props;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,20 +19,28 @@
|
||||||
package org.apache.hadoop.hdds.scm;
|
package org.apache.hadoop.hdds.scm;
|
||||||
|
|
||||||
|
|
||||||
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||||
.ContainerCommandResponseProto;
|
.ContainerCommandResponseProto;
|
||||||
|
|
||||||
import java.util.UUID;
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class represents the Async reply from XceiverClient.
|
* This class represents the reply from XceiverClient.
|
||||||
*/
|
*/
|
||||||
public class XceiverClientReply {
|
public class XceiverClientReply {
|
||||||
|
|
||||||
private CompletableFuture<ContainerCommandResponseProto> response;
|
private CompletableFuture<ContainerCommandResponseProto> response;
|
||||||
private Long logIndex;
|
private Long logIndex;
|
||||||
private UUID dnId;
|
|
||||||
|
/**
|
||||||
|
* List of datanodes where the command got executed and reply is received.
|
||||||
|
* If there is an exception in the reply, these datanodes will inform
|
||||||
|
* about the servers where there is a failure.
|
||||||
|
*/
|
||||||
|
private List<DatanodeDetails> datanodes;
|
||||||
|
|
||||||
public XceiverClientReply(
|
public XceiverClientReply(
|
||||||
CompletableFuture<ContainerCommandResponseProto> response) {
|
CompletableFuture<ContainerCommandResponseProto> response) {
|
||||||
|
@ -40,10 +48,11 @@ public class XceiverClientReply {
|
||||||
}
|
}
|
||||||
|
|
||||||
public XceiverClientReply(
|
public XceiverClientReply(
|
||||||
CompletableFuture<ContainerCommandResponseProto> response, UUID dnId) {
|
CompletableFuture<ContainerCommandResponseProto> response,
|
||||||
|
List<DatanodeDetails> datanodes) {
|
||||||
this.logIndex = (long) 0;
|
this.logIndex = (long) 0;
|
||||||
this.response = response;
|
this.response = response;
|
||||||
this.dnId = dnId;
|
this.datanodes = datanodes == null ? new ArrayList<>() : datanodes;
|
||||||
}
|
}
|
||||||
|
|
||||||
public CompletableFuture<ContainerCommandResponseProto> getResponse() {
|
public CompletableFuture<ContainerCommandResponseProto> getResponse() {
|
||||||
|
@ -58,12 +67,12 @@ public class XceiverClientReply {
|
||||||
this.logIndex = logIndex;
|
this.logIndex = logIndex;
|
||||||
}
|
}
|
||||||
|
|
||||||
public UUID getDatanode() {
|
public List<DatanodeDetails> getDatanodes() {
|
||||||
return dnId;
|
return datanodes;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setDatanode(UUID datanodeId) {
|
public void addDatanode(DatanodeDetails dn) {
|
||||||
this.dnId = datanodeId;
|
datanodes.add(dn);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setResponse(
|
public void setResponse(
|
||||||
|
|
|
@ -21,11 +21,11 @@ package org.apache.hadoop.hdds.scm;
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.UUID;
|
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
|
@ -123,7 +123,7 @@ public abstract class XceiverClientSpi implements Closeable {
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public XceiverClientReply sendCommand(
|
public XceiverClientReply sendCommand(
|
||||||
ContainerCommandRequestProto request, List<UUID> excludeDns)
|
ContainerCommandRequestProto request, List<DatanodeDetails> excludeDns)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
try {
|
try {
|
||||||
XceiverClientReply reply;
|
XceiverClientReply reply;
|
||||||
|
@ -157,14 +157,14 @@ public abstract class XceiverClientSpi implements Closeable {
|
||||||
* Check if an specfic commitIndex is replicated to majority/all servers.
|
* Check if an specfic commitIndex is replicated to majority/all servers.
|
||||||
* @param index index to watch for
|
* @param index index to watch for
|
||||||
* @param timeout timeout provided for the watch ipeartion to complete
|
* @param timeout timeout provided for the watch ipeartion to complete
|
||||||
* @return the min commit index replicated to all or majority servers
|
* @return reply containing the min commit index replicated to all or majority
|
||||||
* in case of a failure
|
* servers in case of a failure
|
||||||
* @throws InterruptedException
|
* @throws InterruptedException
|
||||||
* @throws ExecutionException
|
* @throws ExecutionException
|
||||||
* @throws TimeoutException
|
* @throws TimeoutException
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public abstract long watchForCommit(long index, long timeout)
|
public abstract XceiverClientReply watchForCommit(long index, long timeout)
|
||||||
throws InterruptedException, ExecutionException, TimeoutException,
|
throws InterruptedException, ExecutionException, TimeoutException,
|
||||||
IOException;
|
IOException;
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,103 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with this
|
||||||
|
* work for additional information regarding copyright ownership. The ASF
|
||||||
|
* licenses this file to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance with the License.
|
||||||
|
* You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||||
|
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||||
|
* License for the specific language governing permissions and limitations under
|
||||||
|
* the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.hdds.scm.container.common.helpers;
|
||||||
|
|
||||||
|
|
||||||
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
|
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
|
||||||
|
|
||||||
|
import java.util.*;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This class contains set of dns and containers which ozone client provides
|
||||||
|
* to be handed over to SCM when block allocation request comes.
|
||||||
|
*/
|
||||||
|
public class ExcludeList {
|
||||||
|
|
||||||
|
private final List<DatanodeDetails> datanodes;
|
||||||
|
private final List<ContainerID> containerIds;
|
||||||
|
private final List<PipelineID> pipelineIds;
|
||||||
|
|
||||||
|
|
||||||
|
public ExcludeList() {
|
||||||
|
datanodes = new ArrayList<>();
|
||||||
|
containerIds = new ArrayList<>();
|
||||||
|
pipelineIds = new ArrayList<>();
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<ContainerID> getContainerIds() {
|
||||||
|
return containerIds;
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<DatanodeDetails> getDatanodes() {
|
||||||
|
return datanodes;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void addDatanodes(Collection<DatanodeDetails> dns) {
|
||||||
|
datanodes.addAll(dns);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void addDatanode(DatanodeDetails dn) {
|
||||||
|
datanodes.add(dn);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void addConatinerId(ContainerID containerId) {
|
||||||
|
containerIds.add(containerId);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void addPipeline(PipelineID pipelineId) {
|
||||||
|
pipelineIds.add(pipelineId);
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<PipelineID> getPipelineIds() {
|
||||||
|
return pipelineIds;
|
||||||
|
}
|
||||||
|
|
||||||
|
public HddsProtos.ExcludeListProto getProtoBuf() {
|
||||||
|
HddsProtos.ExcludeListProto.Builder builder =
|
||||||
|
HddsProtos.ExcludeListProto.newBuilder();
|
||||||
|
containerIds.parallelStream()
|
||||||
|
.forEach(id -> builder.addContainerIds(id.getId()));
|
||||||
|
datanodes.parallelStream().forEach(dn -> {
|
||||||
|
builder.addDatanodes(dn.getUuidString());
|
||||||
|
});
|
||||||
|
pipelineIds.parallelStream().forEach(pipelineID -> {
|
||||||
|
builder.addPipelineIds(pipelineID.getProtobuf());
|
||||||
|
});
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static ExcludeList getFromProtoBuf(
|
||||||
|
HddsProtos.ExcludeListProto excludeListProto) {
|
||||||
|
ExcludeList excludeList = new ExcludeList();
|
||||||
|
excludeListProto.getContainerIdsList().parallelStream().forEach(id -> {
|
||||||
|
excludeList.addConatinerId(ContainerID.valueof(id));
|
||||||
|
});
|
||||||
|
DatanodeDetails.Builder builder = DatanodeDetails.newBuilder();
|
||||||
|
excludeListProto.getDatanodesList().forEach(dn -> {
|
||||||
|
builder.setUuid(dn);
|
||||||
|
excludeList.addDatanode(builder.build());
|
||||||
|
});
|
||||||
|
excludeListProto.getPipelineIdsList().forEach(pipelineID -> {
|
||||||
|
excludeList.addPipeline(PipelineID.getFromProtobuf(pipelineID));
|
||||||
|
});
|
||||||
|
return excludeList;
|
||||||
|
}
|
||||||
|
}
|
|
@ -18,6 +18,7 @@
|
||||||
package org.apache.hadoop.hdds.scm.protocol;
|
package org.apache.hadoop.hdds.scm.protocol;
|
||||||
|
|
||||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
|
||||||
import org.apache.hadoop.security.KerberosInfo;
|
import org.apache.hadoop.security.KerberosInfo;
|
||||||
import org.apache.hadoop.hdds.scm.ScmInfo;
|
import org.apache.hadoop.hdds.scm.ScmInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
||||||
|
@ -47,11 +48,14 @@ public interface ScmBlockLocationProtocol extends Closeable {
|
||||||
* Asks SCM where a block should be allocated. SCM responds with the
|
* Asks SCM where a block should be allocated. SCM responds with the
|
||||||
* set of datanodes that should be used creating this block.
|
* set of datanodes that should be used creating this block.
|
||||||
* @param size - size of the block.
|
* @param size - size of the block.
|
||||||
|
* @param excludeList List of datanodes/containers to exclude during block
|
||||||
|
* allocation.
|
||||||
* @return allocated block accessing info (key, pipeline).
|
* @return allocated block accessing info (key, pipeline).
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
AllocatedBlock allocateBlock(long size, ReplicationType type,
|
AllocatedBlock allocateBlock(long size, ReplicationType type,
|
||||||
ReplicationFactor factor, String owner) throws IOException;
|
ReplicationFactor factor, String owner, ExcludeList excludeList)
|
||||||
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Delete blocks for a set of object keys.
|
* Delete blocks for a set of object keys.
|
||||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.Dele
|
||||||
import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.KeyBlocks;
|
import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.KeyBlocks;
|
||||||
import org.apache.hadoop.hdds.scm.ScmInfo;
|
import org.apache.hadoop.hdds.scm.ScmInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
|
||||||
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
||||||
import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
|
import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
|
||||||
import org.apache.hadoop.hdds.tracing.TracingUtil;
|
import org.apache.hadoop.hdds.tracing.TracingUtil;
|
||||||
|
@ -80,7 +81,7 @@ public final class ScmBlockLocationProtocolClientSideTranslatorPB
|
||||||
@Override
|
@Override
|
||||||
public AllocatedBlock allocateBlock(long size,
|
public AllocatedBlock allocateBlock(long size,
|
||||||
HddsProtos.ReplicationType type, HddsProtos.ReplicationFactor factor,
|
HddsProtos.ReplicationType type, HddsProtos.ReplicationFactor factor,
|
||||||
String owner) throws IOException {
|
String owner, ExcludeList excludeList) throws IOException {
|
||||||
Preconditions.checkArgument(size > 0, "block size must be greater than 0");
|
Preconditions.checkArgument(size > 0, "block size must be greater than 0");
|
||||||
|
|
||||||
AllocateScmBlockRequestProto request =
|
AllocateScmBlockRequestProto request =
|
||||||
|
@ -90,6 +91,7 @@ public final class ScmBlockLocationProtocolClientSideTranslatorPB
|
||||||
.setFactor(factor)
|
.setFactor(factor)
|
||||||
.setOwner(owner)
|
.setOwner(owner)
|
||||||
.setTraceID(TracingUtil.exportCurrentSpan())
|
.setTraceID(TracingUtil.exportCurrentSpan())
|
||||||
|
.setExcludeList(excludeList.getProtoBuf())
|
||||||
.build();
|
.build();
|
||||||
final AllocateScmBlockResponseProto response;
|
final AllocateScmBlockResponseProto response;
|
||||||
try {
|
try {
|
||||||
|
|
|
@ -18,9 +18,11 @@
|
||||||
|
|
||||||
package org.apache.hadoop.hdds.scm.storage;
|
package org.apache.hadoop.hdds.scm.storage;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.scm.XceiverClientReply;
|
import org.apache.hadoop.hdds.scm.XceiverClientReply;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers
|
import org.apache.hadoop.hdds.scm.container.common.helpers
|
||||||
.BlockNotCommittedException;
|
.BlockNotCommittedException;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerNotOpenException;
|
||||||
import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
|
import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
|
||||||
import org.apache.hadoop.hdds.security.token.OzoneBlockTokenSelector;
|
import org.apache.hadoop.hdds.security.token.OzoneBlockTokenSelector;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
|
@ -71,7 +73,6 @@ import org.apache.hadoop.hdds.client.BlockID;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.UUID;
|
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -232,7 +233,8 @@ public final class ContainerProtocolCalls {
|
||||||
* @throws IOException if there is an I/O error while performing the call
|
* @throws IOException if there is an I/O error while performing the call
|
||||||
*/
|
*/
|
||||||
public static XceiverClientReply readChunk(XceiverClientSpi xceiverClient,
|
public static XceiverClientReply readChunk(XceiverClientSpi xceiverClient,
|
||||||
ChunkInfo chunk, BlockID blockID, String traceID, List<UUID> excludeDns)
|
ChunkInfo chunk, BlockID blockID, String traceID,
|
||||||
|
List<DatanodeDetails> excludeDns)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
ReadChunkRequestProto.Builder readChunkRequest = ReadChunkRequestProto
|
ReadChunkRequestProto.Builder readChunkRequest = ReadChunkRequestProto
|
||||||
.newBuilder()
|
.newBuilder()
|
||||||
|
@ -563,6 +565,9 @@ public final class ContainerProtocolCalls {
|
||||||
} else if (response.getResult()
|
} else if (response.getResult()
|
||||||
== ContainerProtos.Result.BLOCK_NOT_COMMITTED) {
|
== ContainerProtos.Result.BLOCK_NOT_COMMITTED) {
|
||||||
throw new BlockNotCommittedException(response.getMessage());
|
throw new BlockNotCommittedException(response.getMessage());
|
||||||
|
} else if (response.getResult()
|
||||||
|
== ContainerProtos.Result.CLOSED_CONTAINER_IO) {
|
||||||
|
throw new ContainerNotOpenException(response.getMessage());
|
||||||
}
|
}
|
||||||
throw new StorageContainerException(
|
throw new StorageContainerException(
|
||||||
response.getMessage(), response.getResult());
|
response.getMessage(), response.getResult());
|
||||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.BlockTokenSecretProto.Bu
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||||
|
import org.apache.hadoop.security.token.Token.TrivialRenewer;
|
||||||
|
|
||||||
import java.io.DataInput;
|
import java.io.DataInput;
|
||||||
import java.io.DataInputStream;
|
import java.io.DataInputStream;
|
||||||
|
@ -195,5 +196,17 @@ public class OzoneBlockTokenIdentifier extends TokenIdentifier {
|
||||||
}
|
}
|
||||||
out.write(builder.build().toByteArray());
|
out.write(builder.build().toByteArray());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Default TrivialRenewer.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public static class Renewer extends TrivialRenewer {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Text getKind() {
|
||||||
|
return KIND_NAME;
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -60,17 +60,22 @@ public interface CertificateApprover {
|
||||||
* @param validFrom - Begin Date
|
* @param validFrom - Begin Date
|
||||||
* @param validTill - End Date
|
* @param validTill - End Date
|
||||||
* @param certificationRequest - Certification Request.
|
* @param certificationRequest - Certification Request.
|
||||||
|
* @param scmId - SCM id.
|
||||||
|
* @param clusterId - Cluster id.
|
||||||
* @return Signed Certificate.
|
* @return Signed Certificate.
|
||||||
* @throws IOException - On Error
|
* @throws IOException - On Error
|
||||||
* @throws OperatorCreationException - on Error.
|
* @throws OperatorCreationException - on Error.
|
||||||
*/
|
*/
|
||||||
|
@SuppressWarnings("ParameterNumber")
|
||||||
X509CertificateHolder sign(
|
X509CertificateHolder sign(
|
||||||
SecurityConfig config,
|
SecurityConfig config,
|
||||||
PrivateKey caPrivate,
|
PrivateKey caPrivate,
|
||||||
X509CertificateHolder caCertificate,
|
X509CertificateHolder caCertificate,
|
||||||
Date validFrom,
|
Date validFrom,
|
||||||
Date validTill,
|
Date validTill,
|
||||||
PKCS10CertificationRequest certificationRequest)
|
PKCS10CertificationRequest certificationRequest,
|
||||||
|
String scmId,
|
||||||
|
String clusterId)
|
||||||
throws IOException, OperatorCreationException;
|
throws IOException, OperatorCreationException;
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -22,7 +22,10 @@ package org.apache.hadoop.hdds.security.x509.certificate.authority;
|
||||||
import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
|
import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
|
||||||
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
||||||
import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.PKIProfile;
|
import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.PKIProfile;
|
||||||
|
import org.apache.hadoop.hdds.security.x509.keys.SecurityUtil;
|
||||||
import org.apache.hadoop.util.Time;
|
import org.apache.hadoop.util.Time;
|
||||||
|
import org.bouncycastle.asn1.x500.X500Name;
|
||||||
|
import org.bouncycastle.asn1.x500.style.BCStyle;
|
||||||
import org.bouncycastle.asn1.x509.AlgorithmIdentifier;
|
import org.bouncycastle.asn1.x509.AlgorithmIdentifier;
|
||||||
import org.bouncycastle.asn1.x509.SubjectPublicKeyInfo;
|
import org.bouncycastle.asn1.x509.SubjectPublicKeyInfo;
|
||||||
import org.bouncycastle.cert.X509CertificateHolder;
|
import org.bouncycastle.cert.X509CertificateHolder;
|
||||||
|
@ -67,18 +70,22 @@ public class DefaultApprover extends BaseApprover {
|
||||||
* @param validFrom - Begin Da te
|
* @param validFrom - Begin Da te
|
||||||
* @param validTill - End Date
|
* @param validTill - End Date
|
||||||
* @param certificationRequest - Certification Request.
|
* @param certificationRequest - Certification Request.
|
||||||
|
* @param scmId - SCM id.
|
||||||
|
* @param clusterId - Cluster id.
|
||||||
* @return Signed Certificate.
|
* @return Signed Certificate.
|
||||||
* @throws IOException - On Error
|
* @throws IOException - On Error
|
||||||
* @throws OperatorCreationException - on Error.
|
* @throws OperatorCreationException - on Error.
|
||||||
*/
|
*/
|
||||||
|
@SuppressWarnings("ParameterNumber")
|
||||||
public X509CertificateHolder sign(
|
public X509CertificateHolder sign(
|
||||||
SecurityConfig config,
|
SecurityConfig config,
|
||||||
PrivateKey caPrivate,
|
PrivateKey caPrivate,
|
||||||
X509CertificateHolder caCertificate,
|
X509CertificateHolder caCertificate,
|
||||||
Date validFrom,
|
Date validFrom,
|
||||||
Date validTill,
|
Date validTill,
|
||||||
PKCS10CertificationRequest certificationRequest)
|
PKCS10CertificationRequest certificationRequest,
|
||||||
throws IOException, OperatorCreationException {
|
String scmId,
|
||||||
|
String clusterId) throws IOException, OperatorCreationException {
|
||||||
|
|
||||||
AlgorithmIdentifier sigAlgId = new
|
AlgorithmIdentifier sigAlgId = new
|
||||||
DefaultSignatureAlgorithmIdentifierFinder().find(
|
DefaultSignatureAlgorithmIdentifierFinder().find(
|
||||||
|
@ -91,6 +98,29 @@ public class DefaultApprover extends BaseApprover {
|
||||||
SubjectPublicKeyInfo keyInfo =
|
SubjectPublicKeyInfo keyInfo =
|
||||||
certificationRequest.getSubjectPublicKeyInfo();
|
certificationRequest.getSubjectPublicKeyInfo();
|
||||||
|
|
||||||
|
// Get scmId and cluster Id from subject name.
|
||||||
|
X500Name x500Name = certificationRequest.getSubject();
|
||||||
|
String csrScmId = x500Name.getRDNs(BCStyle.OU)[0].getFirst().getValue().
|
||||||
|
toASN1Primitive().toString();
|
||||||
|
String csrClusterId = x500Name.getRDNs(BCStyle.O)[0].getFirst().getValue().
|
||||||
|
toASN1Primitive().toString();
|
||||||
|
|
||||||
|
if (!scmId.equals(csrScmId) || !clusterId.equals(csrClusterId)) {
|
||||||
|
if (csrScmId.equalsIgnoreCase("null") &&
|
||||||
|
csrClusterId.equalsIgnoreCase("null")) {
|
||||||
|
// Special case to handle DN certificate generation as DN might not know
|
||||||
|
// scmId and clusterId before registration. In secure mode registration
|
||||||
|
// will succeed only after datanode has a valid certificate.
|
||||||
|
String cn = x500Name.getRDNs(BCStyle.CN)[0].getFirst().getValue()
|
||||||
|
.toASN1Primitive().toString();
|
||||||
|
x500Name = SecurityUtil.getDistinguishedName(cn, scmId, clusterId);
|
||||||
|
} else {
|
||||||
|
// Throw exception if scmId and clusterId doesn't match.
|
||||||
|
throw new SCMSecurityException("ScmId and ClusterId in CSR subject" +
|
||||||
|
" are incorrect.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
RSAKeyParameters rsa =
|
RSAKeyParameters rsa =
|
||||||
(RSAKeyParameters) PublicKeyFactory.createKey(keyInfo);
|
(RSAKeyParameters) PublicKeyFactory.createKey(keyInfo);
|
||||||
if (rsa.getModulus().bitLength() < config.getSize()) {
|
if (rsa.getModulus().bitLength() < config.getSize()) {
|
||||||
|
@ -104,7 +134,7 @@ public class DefaultApprover extends BaseApprover {
|
||||||
BigInteger.valueOf(Time.monotonicNowNanos()),
|
BigInteger.valueOf(Time.monotonicNowNanos()),
|
||||||
validFrom,
|
validFrom,
|
||||||
validTill,
|
validTill,
|
||||||
certificationRequest.getSubject(), keyInfo);
|
x500Name, keyInfo);
|
||||||
|
|
||||||
ContentSigner sigGen = new BcRSAContentSignerBuilder(sigAlgId, digAlgId)
|
ContentSigner sigGen = new BcRSAContentSignerBuilder(sigAlgId, digAlgId)
|
||||||
.build(asymmetricKP);
|
.build(asymmetricKP);
|
||||||
|
|
|
@ -227,7 +227,7 @@ public class DefaultCAServer implements CertificateServer {
|
||||||
X509CertificateHolder xcert = approver.sign(config,
|
X509CertificateHolder xcert = approver.sign(config,
|
||||||
getCAKeys().getPrivate(),
|
getCAKeys().getPrivate(),
|
||||||
getCACertificate(), java.sql.Date.valueOf(beginDate),
|
getCACertificate(), java.sql.Date.valueOf(beginDate),
|
||||||
java.sql.Date.valueOf(endDate), csr);
|
java.sql.Date.valueOf(endDate), csr, scmID, clusterID);
|
||||||
store.storeValidCertificate(xcert.getSerialNumber(),
|
store.storeValidCertificate(xcert.getSerialNumber(),
|
||||||
CertificateCodec.getX509Certificate(xcert));
|
CertificateCodec.getX509Certificate(xcert));
|
||||||
xcertHolder.complete(xcert);
|
xcertHolder.complete(xcert);
|
||||||
|
|
|
@ -20,6 +20,8 @@
|
||||||
package org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles;
|
package org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
import org.apache.commons.codec.DecoderException;
|
||||||
|
import org.apache.commons.codec.binary.Hex;
|
||||||
import org.apache.commons.validator.routines.DomainValidator;
|
import org.apache.commons.validator.routines.DomainValidator;
|
||||||
import org.bouncycastle.asn1.ASN1ObjectIdentifier;
|
import org.bouncycastle.asn1.ASN1ObjectIdentifier;
|
||||||
import org.bouncycastle.asn1.x500.RDN;
|
import org.bouncycastle.asn1.x500.RDN;
|
||||||
|
@ -32,7 +34,6 @@ import org.bouncycastle.asn1.x509.KeyUsage;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import javax.xml.bind.DatatypeConverter;
|
|
||||||
import java.net.InetAddress;
|
import java.net.InetAddress;
|
||||||
import java.net.UnknownHostException;
|
import java.net.UnknownHostException;
|
||||||
import java.util.AbstractMap.SimpleEntry;
|
import java.util.AbstractMap.SimpleEntry;
|
||||||
|
@ -234,10 +235,10 @@ public class DefaultProfile implements PKIProfile {
|
||||||
// TODO: Fail? if we cannot resolve the Hostname?
|
// TODO: Fail? if we cannot resolve the Hostname?
|
||||||
try {
|
try {
|
||||||
final InetAddress byAddress = InetAddress.getByAddress(
|
final InetAddress byAddress = InetAddress.getByAddress(
|
||||||
DatatypeConverter.parseHexBinary(value.substring(1)));
|
Hex.decodeHex(value.substring(1)));
|
||||||
LOG.debug("Host Name/IP Address : {}", byAddress.toString());
|
LOG.debug("Host Name/IP Address : {}", byAddress.toString());
|
||||||
return true;
|
return true;
|
||||||
} catch (UnknownHostException e) {
|
} catch (UnknownHostException | DecoderException e) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
case GeneralName.dNSName:
|
case GeneralName.dNSName:
|
||||||
|
|
|
@ -109,7 +109,7 @@ public interface CertificateClient {
|
||||||
*
|
*
|
||||||
* @return CertificateSignRequest.Builder
|
* @return CertificateSignRequest.Builder
|
||||||
*/
|
*/
|
||||||
CertificateSignRequest.Builder getCSRBuilder();
|
CertificateSignRequest.Builder getCSRBuilder() throws CertificateException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the certificate of well-known entity from SCM.
|
* Get the certificate of well-known entity from SCM.
|
||||||
|
|
|
@ -19,6 +19,8 @@
|
||||||
|
|
||||||
package org.apache.hadoop.hdds.security.x509.certificate.client;
|
package org.apache.hadoop.hdds.security.x509.certificate.client;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
|
||||||
|
import org.apache.hadoop.hdds.security.x509.exceptions.CertificateException;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
@ -30,8 +32,22 @@ public class DNCertificateClient extends DefaultCertificateClient {
|
||||||
|
|
||||||
private static final Logger LOG =
|
private static final Logger LOG =
|
||||||
LoggerFactory.getLogger(DNCertificateClient.class);
|
LoggerFactory.getLogger(DNCertificateClient.class);
|
||||||
DNCertificateClient(SecurityConfig securityConfig, String component) {
|
public DNCertificateClient(SecurityConfig securityConfig) {
|
||||||
super(securityConfig, component, LOG);
|
super(securityConfig, LOG);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a CSR builder that can be used to creates a Certificate signing
|
||||||
|
* request.
|
||||||
|
*
|
||||||
|
* @return CertificateSignRequest.Builder
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public CertificateSignRequest.Builder getCSRBuilder()
|
||||||
|
throws CertificateException {
|
||||||
|
return super.getCSRBuilder()
|
||||||
|
.setDigitalEncryption(false)
|
||||||
|
.setDigitalSignature(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Logger getLogger() {
|
public Logger getLogger() {
|
||||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.hdds.security.x509.certificate.client;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.commons.lang3.RandomStringUtils;
|
import org.apache.commons.lang3.RandomStringUtils;
|
||||||
|
import org.apache.commons.validator.routines.DomainValidator;
|
||||||
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
||||||
import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
|
import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
|
||||||
import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
|
import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
|
||||||
|
@ -66,20 +67,16 @@ public abstract class DefaultCertificateClient implements CertificateClient {
|
||||||
|
|
||||||
private final Logger logger;
|
private final Logger logger;
|
||||||
private final SecurityConfig securityConfig;
|
private final SecurityConfig securityConfig;
|
||||||
private final String component;
|
|
||||||
private final KeyCodec keyCodec;
|
private final KeyCodec keyCodec;
|
||||||
private PrivateKey privateKey;
|
private PrivateKey privateKey;
|
||||||
private PublicKey publicKey;
|
private PublicKey publicKey;
|
||||||
private X509Certificate x509Certificate;
|
private X509Certificate x509Certificate;
|
||||||
|
|
||||||
|
|
||||||
DefaultCertificateClient(SecurityConfig securityConfig, String component,
|
DefaultCertificateClient(SecurityConfig securityConfig, Logger log) {
|
||||||
Logger log) {
|
|
||||||
Objects.requireNonNull(securityConfig);
|
Objects.requireNonNull(securityConfig);
|
||||||
Objects.requireNonNull(component);
|
|
||||||
this.component = component;
|
|
||||||
this.securityConfig = securityConfig;
|
this.securityConfig = securityConfig;
|
||||||
keyCodec = new KeyCodec(securityConfig, component);
|
keyCodec = new KeyCodec(securityConfig);
|
||||||
this.logger = log;
|
this.logger = log;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -95,15 +92,14 @@ public abstract class DefaultCertificateClient implements CertificateClient {
|
||||||
return privateKey;
|
return privateKey;
|
||||||
}
|
}
|
||||||
|
|
||||||
Path keyPath = securityConfig.getKeyLocation(component);
|
Path keyPath = securityConfig.getKeyLocation();
|
||||||
if (OzoneSecurityUtil.checkIfFileExist(keyPath,
|
if (OzoneSecurityUtil.checkIfFileExist(keyPath,
|
||||||
securityConfig.getPrivateKeyFileName())) {
|
securityConfig.getPrivateKeyFileName())) {
|
||||||
try {
|
try {
|
||||||
privateKey = keyCodec.readPrivateKey();
|
privateKey = keyCodec.readPrivateKey();
|
||||||
} catch (InvalidKeySpecException | NoSuchAlgorithmException
|
} catch (InvalidKeySpecException | NoSuchAlgorithmException
|
||||||
| IOException e) {
|
| IOException e) {
|
||||||
getLogger().error("Error while getting private key for {}",
|
getLogger().error("Error while getting private key.", e);
|
||||||
component, e);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return privateKey;
|
return privateKey;
|
||||||
|
@ -121,15 +117,14 @@ public abstract class DefaultCertificateClient implements CertificateClient {
|
||||||
return publicKey;
|
return publicKey;
|
||||||
}
|
}
|
||||||
|
|
||||||
Path keyPath = securityConfig.getKeyLocation(component);
|
Path keyPath = securityConfig.getKeyLocation();
|
||||||
if (OzoneSecurityUtil.checkIfFileExist(keyPath,
|
if (OzoneSecurityUtil.checkIfFileExist(keyPath,
|
||||||
securityConfig.getPublicKeyFileName())) {
|
securityConfig.getPublicKeyFileName())) {
|
||||||
try {
|
try {
|
||||||
publicKey = keyCodec.readPublicKey();
|
publicKey = keyCodec.readPublicKey();
|
||||||
} catch (InvalidKeySpecException | NoSuchAlgorithmException
|
} catch (InvalidKeySpecException | NoSuchAlgorithmException
|
||||||
| IOException e) {
|
| IOException e) {
|
||||||
getLogger().error("Error while getting private key for {}",
|
getLogger().error("Error while getting public key.", e);
|
||||||
component, e);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return publicKey;
|
return publicKey;
|
||||||
|
@ -147,18 +142,18 @@ public abstract class DefaultCertificateClient implements CertificateClient {
|
||||||
return x509Certificate;
|
return x509Certificate;
|
||||||
}
|
}
|
||||||
|
|
||||||
Path certPath = securityConfig.getCertificateLocation(component);
|
Path certPath = securityConfig.getCertificateLocation();
|
||||||
if (OzoneSecurityUtil.checkIfFileExist(certPath,
|
if (OzoneSecurityUtil.checkIfFileExist(certPath,
|
||||||
securityConfig.getCertificateFileName())) {
|
securityConfig.getCertificateFileName())) {
|
||||||
CertificateCodec certificateCodec =
|
CertificateCodec certificateCodec =
|
||||||
new CertificateCodec(securityConfig, component);
|
new CertificateCodec(securityConfig);
|
||||||
try {
|
try {
|
||||||
X509CertificateHolder x509CertificateHolder =
|
X509CertificateHolder x509CertificateHolder =
|
||||||
certificateCodec.readCertificate();
|
certificateCodec.readCertificate();
|
||||||
x509Certificate =
|
x509Certificate =
|
||||||
CertificateCodec.getX509Certificate(x509CertificateHolder);
|
CertificateCodec.getX509Certificate(x509CertificateHolder);
|
||||||
} catch (java.security.cert.CertificateException | IOException e) {
|
} catch (java.security.cert.CertificateException | IOException e) {
|
||||||
getLogger().error("Error reading certificate for {}", component, e);
|
getLogger().error("Error reading certificate.", e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return x509Certificate;
|
return x509Certificate;
|
||||||
|
@ -318,8 +313,26 @@ public abstract class DefaultCertificateClient implements CertificateClient {
|
||||||
* @return CertificateSignRequest.Builder
|
* @return CertificateSignRequest.Builder
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public CertificateSignRequest.Builder getCSRBuilder() {
|
public CertificateSignRequest.Builder getCSRBuilder()
|
||||||
return new CertificateSignRequest.Builder();
|
throws CertificateException {
|
||||||
|
CertificateSignRequest.Builder builder =
|
||||||
|
new CertificateSignRequest.Builder()
|
||||||
|
.setConfiguration(securityConfig.getConfiguration());
|
||||||
|
try {
|
||||||
|
DomainValidator validator = DomainValidator.getInstance();
|
||||||
|
// Add all valid ips.
|
||||||
|
OzoneSecurityUtil.getValidInetsForCurrentHost().forEach(
|
||||||
|
ip -> {
|
||||||
|
builder.addIpAddress(ip.getHostAddress());
|
||||||
|
if(validator.isValid(ip.getCanonicalHostName())) {
|
||||||
|
builder.addDnsName(ip.getCanonicalHostName());
|
||||||
|
}
|
||||||
|
});
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new CertificateException("Error while adding ip to CSR builder",
|
||||||
|
e, CSR_ERROR);
|
||||||
|
}
|
||||||
|
return builder;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -345,8 +358,7 @@ public abstract class DefaultCertificateClient implements CertificateClient {
|
||||||
@Override
|
@Override
|
||||||
public void storeCertificate(X509Certificate certificate)
|
public void storeCertificate(X509Certificate certificate)
|
||||||
throws CertificateException {
|
throws CertificateException {
|
||||||
CertificateCodec certificateCodec = new CertificateCodec(securityConfig,
|
CertificateCodec certificateCodec = new CertificateCodec(securityConfig);
|
||||||
component);
|
|
||||||
try {
|
try {
|
||||||
certificateCodec.writeCertificate(
|
certificateCodec.writeCertificate(
|
||||||
new X509CertificateHolder(certificate.getEncoded()));
|
new X509CertificateHolder(certificate.getEncoded()));
|
||||||
|
@ -595,7 +607,7 @@ public abstract class DefaultCertificateClient implements CertificateClient {
|
||||||
* location.
|
* location.
|
||||||
* */
|
* */
|
||||||
protected void bootstrapClientKeys() throws CertificateException {
|
protected void bootstrapClientKeys() throws CertificateException {
|
||||||
Path keyPath = securityConfig.getKeyLocation(component);
|
Path keyPath = securityConfig.getKeyLocation();
|
||||||
if (Files.notExists(keyPath)) {
|
if (Files.notExists(keyPath)) {
|
||||||
try {
|
try {
|
||||||
Files.createDirectories(keyPath);
|
Files.createDirectories(keyPath);
|
||||||
|
@ -618,10 +630,9 @@ public abstract class DefaultCertificateClient implements CertificateClient {
|
||||||
keyCodec.writePrivateKey(keyPair.getPrivate());
|
keyCodec.writePrivateKey(keyPair.getPrivate());
|
||||||
} catch (NoSuchProviderException | NoSuchAlgorithmException
|
} catch (NoSuchProviderException | NoSuchAlgorithmException
|
||||||
| IOException e) {
|
| IOException e) {
|
||||||
getLogger().error("Error while bootstrapping certificate client for {}",
|
getLogger().error("Error while bootstrapping certificate client.", e);
|
||||||
component, e);
|
throw new CertificateException("Error while bootstrapping certificate.",
|
||||||
throw new CertificateException("Error while bootstrapping certificate " +
|
BOOTSTRAP_ERROR);
|
||||||
"client for" + component, BOOTSTRAP_ERROR);
|
|
||||||
}
|
}
|
||||||
return keyPair;
|
return keyPair;
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package org.apache.hadoop.hdds.security.x509.certificate.client;
|
package org.apache.hadoop.hdds.security.x509.certificate.client;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
@ -38,8 +39,8 @@ public class OMCertificateClient extends DefaultCertificateClient {
|
||||||
private static final Logger LOG =
|
private static final Logger LOG =
|
||||||
LoggerFactory.getLogger(OMCertificateClient.class);
|
LoggerFactory.getLogger(OMCertificateClient.class);
|
||||||
|
|
||||||
public OMCertificateClient(SecurityConfig securityConfig, String component) {
|
public OMCertificateClient(SecurityConfig securityConfig) {
|
||||||
super(securityConfig, component, LOG);
|
super(securityConfig, LOG);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected InitResponse handleCase(InitCase init) throws
|
protected InitResponse handleCase(InitCase init) throws
|
||||||
|
@ -96,6 +97,21 @@ public class OMCertificateClient extends DefaultCertificateClient {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a CSR builder that can be used to creates a Certificate signing
|
||||||
|
* request.
|
||||||
|
*
|
||||||
|
* @return CertificateSignRequest.Builder
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public CertificateSignRequest.Builder getCSRBuilder()
|
||||||
|
throws CertificateException {
|
||||||
|
return super.getCSRBuilder()
|
||||||
|
.setDigitalEncryption(true)
|
||||||
|
.setDigitalSignature(true);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
public Logger getLogger() {
|
public Logger getLogger() {
|
||||||
return LOG;
|
return LOG;
|
||||||
}
|
}
|
||||||
|
|
|
@ -80,6 +80,16 @@ public class CertificateCodec {
|
||||||
this.location = securityConfig.getCertificateLocation(component);
|
this.location = securityConfig.getCertificateLocation(component);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates an CertificateCodec.
|
||||||
|
*
|
||||||
|
* @param config - Security Config.
|
||||||
|
*/
|
||||||
|
public CertificateCodec(SecurityConfig config) {
|
||||||
|
this.securityConfig = config;
|
||||||
|
this.location = securityConfig.getCertificateLocation();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates an CertificateCodec.
|
* Creates an CertificateCodec.
|
||||||
*
|
*
|
||||||
|
@ -167,6 +177,22 @@ public class CertificateCodec {
|
||||||
return location;
|
return location;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets the X.509 Certificate from PEM encoded String.
|
||||||
|
*
|
||||||
|
* @param pemEncodedString - PEM encoded String.
|
||||||
|
* @return X509Certificate - Certificate.
|
||||||
|
* @throws CertificateException - Thrown on Failure.
|
||||||
|
* @throws IOException - Thrown on Failure.
|
||||||
|
*/
|
||||||
|
public static X509Certificate getX509Cert(String pemEncodedString)
|
||||||
|
throws CertificateException, IOException {
|
||||||
|
CertificateFactory fact = CertificateFactory.getInstance("X.509");
|
||||||
|
try (InputStream input = IOUtils.toInputStream(pemEncodedString, UTF_8)) {
|
||||||
|
return (X509Certificate) fact.generateCertificate(input);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Write the Certificate pointed to the location by the configs.
|
* Write the Certificate pointed to the location by the configs.
|
||||||
*
|
*
|
||||||
|
|
|
@ -144,6 +144,8 @@ public final class CertificateSignRequest {
|
||||||
private SecurityConfig config;
|
private SecurityConfig config;
|
||||||
private List<GeneralName> altNames;
|
private List<GeneralName> altNames;
|
||||||
private Boolean ca = false;
|
private Boolean ca = false;
|
||||||
|
private boolean digitalSignature;
|
||||||
|
private boolean digitalEncryption;
|
||||||
|
|
||||||
public CertificateSignRequest.Builder setConfiguration(
|
public CertificateSignRequest.Builder setConfiguration(
|
||||||
Configuration configuration) {
|
Configuration configuration) {
|
||||||
|
@ -171,6 +173,16 @@ public final class CertificateSignRequest {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Builder setDigitalSignature(boolean dSign) {
|
||||||
|
this.digitalSignature = dSign;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder setDigitalEncryption(boolean dEncryption) {
|
||||||
|
this.digitalEncryption = dEncryption;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
// Support SAN extenion with DNS and RFC822 Name
|
// Support SAN extenion with DNS and RFC822 Name
|
||||||
// other name type will be added as needed.
|
// other name type will be added as needed.
|
||||||
public CertificateSignRequest.Builder addDnsName(String dnsName) {
|
public CertificateSignRequest.Builder addDnsName(String dnsName) {
|
||||||
|
@ -200,8 +212,13 @@ public final class CertificateSignRequest {
|
||||||
}
|
}
|
||||||
|
|
||||||
private Extension getKeyUsageExtension() throws IOException {
|
private Extension getKeyUsageExtension() throws IOException {
|
||||||
int keyUsageFlag = KeyUsage.digitalSignature | KeyUsage.keyEncipherment
|
int keyUsageFlag = KeyUsage.keyAgreement;
|
||||||
| KeyUsage.dataEncipherment | KeyUsage.keyAgreement;
|
if(digitalEncryption){
|
||||||
|
keyUsageFlag |= KeyUsage.keyEncipherment | KeyUsage.dataEncipherment;
|
||||||
|
}
|
||||||
|
if(digitalSignature) {
|
||||||
|
keyUsageFlag |= KeyUsage.digitalSignature;
|
||||||
|
}
|
||||||
|
|
||||||
if (ca) {
|
if (ca) {
|
||||||
keyUsageFlag |= KeyUsage.keyCertSign | KeyUsage.cRLSign;
|
keyUsageFlag |= KeyUsage.keyCertSign | KeyUsage.cRLSign;
|
||||||
|
@ -252,10 +269,6 @@ public final class CertificateSignRequest {
|
||||||
Preconditions.checkNotNull(key, "KeyPair cannot be null");
|
Preconditions.checkNotNull(key, "KeyPair cannot be null");
|
||||||
Preconditions.checkArgument(Strings.isNotBlank(subject), "Subject " +
|
Preconditions.checkArgument(Strings.isNotBlank(subject), "Subject " +
|
||||||
"cannot be blank");
|
"cannot be blank");
|
||||||
Preconditions.checkArgument(Strings.isNotBlank(clusterID), "Cluster ID " +
|
|
||||||
"cannot be blank");
|
|
||||||
Preconditions.checkArgument(Strings.isNotBlank(scmID), "SCM ID cannot " +
|
|
||||||
"be blank");
|
|
||||||
|
|
||||||
try {
|
try {
|
||||||
CertificateSignRequest csr = new CertificateSignRequest(subject, scmID,
|
CertificateSignRequest csr = new CertificateSignRequest(subject, scmID,
|
||||||
|
|
|
@ -82,6 +82,7 @@ public class CertificateException extends SCMSecurityException {
|
||||||
CRYPTO_SIGN_ERROR,
|
CRYPTO_SIGN_ERROR,
|
||||||
CERTIFICATE_ERROR,
|
CERTIFICATE_ERROR,
|
||||||
BOOTSTRAP_ERROR,
|
BOOTSTRAP_ERROR,
|
||||||
|
CSR_ERROR,
|
||||||
CRYPTO_SIGNATURE_VERIFICATION_ERROR
|
CRYPTO_SIGNATURE_VERIFICATION_ERROR
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -87,6 +87,17 @@ public class KeyCodec {
|
||||||
this.location = securityConfig.getKeyLocation(component);
|
this.location = securityConfig.getKeyLocation(component);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates an KeyCodec.
|
||||||
|
*
|
||||||
|
* @param config - Security Config.
|
||||||
|
*/
|
||||||
|
public KeyCodec(SecurityConfig config) {
|
||||||
|
this.securityConfig = config;
|
||||||
|
isPosixFileSystem = KeyCodec::isPosix;
|
||||||
|
this.location = securityConfig.getKeyLocation();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates an HDDS Key Writer.
|
* Creates an HDDS Key Writer.
|
||||||
*
|
*
|
||||||
|
|
|
@ -379,6 +379,23 @@ public final class OzoneConfigKeys {
|
||||||
public static final String OZONE_FS_ISOLATED_CLASSLOADER =
|
public static final String OZONE_FS_ISOLATED_CLASSLOADER =
|
||||||
"ozone.fs.isolated-classloader";
|
"ozone.fs.isolated-classloader";
|
||||||
|
|
||||||
|
// Ozone Client Retry and Failover configurations
|
||||||
|
public static final String OZONE_CLIENT_RETRY_MAX_ATTEMPTS_KEY =
|
||||||
|
"ozone.client.retry.max.attempts";
|
||||||
|
public static final int OZONE_CLIENT_RETRY_MAX_ATTEMPTS_DEFAULT =
|
||||||
|
10;
|
||||||
|
public static final String OZONE_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY =
|
||||||
|
"ozone.client.failover.max.attempts";
|
||||||
|
public static final int OZONE_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT =
|
||||||
|
15;
|
||||||
|
public static final String OZONE_CLIENT_FAILOVER_SLEEP_BASE_MILLIS_KEY =
|
||||||
|
"ozone.client.failover.sleep.base.millis";
|
||||||
|
public static final int OZONE_CLIENT_FAILOVER_SLEEP_BASE_MILLIS_DEFAULT =
|
||||||
|
500;
|
||||||
|
public static final String OZONE_CLIENT_FAILOVER_SLEEP_MAX_MILLIS_KEY =
|
||||||
|
"ozone.client.failover.sleep.max.millis";
|
||||||
|
public static final int OZONE_CLIENT_FAILOVER_SLEEP_MAX_MILLIS_DEFAULT =
|
||||||
|
15000;
|
||||||
|
|
||||||
public static final String OZONE_FREON_HTTP_ENABLED_KEY =
|
public static final String OZONE_FREON_HTTP_ENABLED_KEY =
|
||||||
"ozone.freon.http.enabled";
|
"ozone.freon.http.enabled";
|
||||||
|
|
|
@ -272,6 +272,11 @@ public final class OzoneConsts {
|
||||||
public static final Metadata.Key<String> USER_METADATA_KEY =
|
public static final Metadata.Key<String> USER_METADATA_KEY =
|
||||||
Metadata.Key.of(OZONE_USER, ASCII_STRING_MARSHALLER);
|
Metadata.Key.of(OZONE_USER, ASCII_STRING_MARSHALLER);
|
||||||
|
|
||||||
|
public static final String RPC_PORT = "RPC";
|
||||||
|
|
||||||
// Default OMServiceID for OM Ratis servers to use as RaftGroupId
|
// Default OMServiceID for OM Ratis servers to use as RaftGroupId
|
||||||
public static final String OM_SERVICE_ID_DEFAULT = "omServiceIdDefault";
|
public static final String OM_SERVICE_ID_DEFAULT = "omServiceIdDefault";
|
||||||
|
|
||||||
|
// Dummy OMNodeID for OM Clients to use for a non-HA OM setup
|
||||||
|
public static final String OM_NODE_ID_DUMMY = "omNodeIdDummy";
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,13 +21,25 @@ package org.apache.hadoop.ozone;
|
||||||
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SECURITY_ENABLED_DEFAULT;
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SECURITY_ENABLED_DEFAULT;
|
||||||
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SECURITY_ENABLED_KEY;
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SECURITY_ENABLED_KEY;
|
||||||
|
|
||||||
|
import org.apache.commons.validator.routines.InetAddressValidator;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.net.InetAddress;
|
||||||
|
import java.net.NetworkInterface;
|
||||||
import java.nio.file.Files;
|
import java.nio.file.Files;
|
||||||
import java.nio.file.Path;
|
import java.nio.file.Path;
|
||||||
import java.nio.file.Paths;
|
import java.nio.file.Paths;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Enumeration;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Ozone security Util class.
|
* Ozone security Util class.
|
||||||
|
@ -36,6 +48,12 @@ import java.nio.file.Paths;
|
||||||
@InterfaceStability.Evolving
|
@InterfaceStability.Evolving
|
||||||
public final class OzoneSecurityUtil {
|
public final class OzoneSecurityUtil {
|
||||||
|
|
||||||
|
private final static Logger LOG =
|
||||||
|
LoggerFactory.getLogger(OzoneSecurityUtil.class);
|
||||||
|
// List of ip's not recommended to be added to CSR.
|
||||||
|
private final static Set<String> INVALID_IPS = new HashSet<>(Arrays.asList(
|
||||||
|
"0.0.0.0", "127.0.0.1"));
|
||||||
|
|
||||||
private OzoneSecurityUtil() {
|
private OzoneSecurityUtil() {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -57,4 +75,44 @@ public final class OzoneSecurityUtil {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Iterates through network interfaces and return all valid ip's not
|
||||||
|
* listed in CertificateSignRequest#INVALID_IPS.
|
||||||
|
*
|
||||||
|
* @return List<InetAddress>
|
||||||
|
* @throws IOException if no network interface are found or if an error
|
||||||
|
* occurs.
|
||||||
|
*/
|
||||||
|
public static List<InetAddress> getValidInetsForCurrentHost()
|
||||||
|
throws IOException {
|
||||||
|
List<InetAddress> hostIps = new ArrayList<>();
|
||||||
|
InetAddressValidator ipValidator = InetAddressValidator.getInstance();
|
||||||
|
|
||||||
|
Enumeration<NetworkInterface> enumNI =
|
||||||
|
NetworkInterface.getNetworkInterfaces();
|
||||||
|
if (enumNI != null) {
|
||||||
|
while (enumNI.hasMoreElements()) {
|
||||||
|
NetworkInterface ifc = enumNI.nextElement();
|
||||||
|
if (ifc.isUp()) {
|
||||||
|
Enumeration<InetAddress> enumAdds = ifc.getInetAddresses();
|
||||||
|
while (enumAdds.hasMoreElements()) {
|
||||||
|
InetAddress addr = enumAdds.nextElement();
|
||||||
|
|
||||||
|
if (ipValidator.isValid(addr.getHostAddress())
|
||||||
|
&& !INVALID_IPS.contains(addr.getHostAddress())) {
|
||||||
|
LOG.info("Adding ip:{},host:{}", addr.getHostAddress(),
|
||||||
|
addr.getHostName());
|
||||||
|
hostIps.add(addr);
|
||||||
|
} else {
|
||||||
|
LOG.info("ip:{},host:{} not returned.", addr.getHostAddress(),
|
||||||
|
addr.getHostName());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return hostIps;
|
||||||
|
} else {
|
||||||
|
throw new IOException("Unable to get network interfaces.");
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,6 +24,7 @@ import io.opentracing.Scope;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hdds.scm.ScmInfo;
|
import org.apache.hadoop.hdds.scm.ScmInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
|
||||||
import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
|
import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
|
||||||
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
|
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
|
||||||
import org.apache.hadoop.hdds.scm.protocolPB.ScmBlockLocationProtocolPB;
|
import org.apache.hadoop.hdds.scm.protocolPB.ScmBlockLocationProtocolPB;
|
||||||
|
@ -77,7 +78,8 @@ public final class ScmBlockLocationProtocolServerSideTranslatorPB
|
||||||
request.getTraceID())) {
|
request.getTraceID())) {
|
||||||
AllocatedBlock allocatedBlock =
|
AllocatedBlock allocatedBlock =
|
||||||
impl.allocateBlock(request.getSize(), request.getType(),
|
impl.allocateBlock(request.getSize(), request.getType(),
|
||||||
request.getFactor(), request.getOwner());
|
request.getFactor(), request.getOwner(),
|
||||||
|
ExcludeList.getFromProtoBuf(request.getExcludeList()));
|
||||||
if (allocatedBlock != null) {
|
if (allocatedBlock != null) {
|
||||||
return
|
return
|
||||||
AllocateScmBlockResponseProto.newBuilder()
|
AllocateScmBlockResponseProto.newBuilder()
|
||||||
|
|
|
@ -61,4 +61,9 @@ public class LevelDBStoreIterator implements MetaStoreIterator<KeyValue> {
|
||||||
public void seekToLast() {
|
public void seekToLast() {
|
||||||
levelDBIterator.seekToLast();
|
levelDBIterator.seekToLast();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void prefixSeek(byte[] prefix) {
|
||||||
|
levelDBIterator.seek(prefix);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -36,4 +36,9 @@ public interface MetaStoreIterator<T> extends Iterator<T> {
|
||||||
*/
|
*/
|
||||||
void seekToLast();
|
void seekToLast();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* seek with prefix.
|
||||||
|
*/
|
||||||
|
void prefixSeek(byte[] prefix);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -63,4 +63,9 @@ public class RocksDBStoreIterator implements MetaStoreIterator<KeyValue> {
|
||||||
rocksDBIterator.seekToLast();
|
rocksDBIterator.seekToLast();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void prefixSeek(byte[] prefix) {
|
||||||
|
rocksDBIterator.seek(prefix);
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -42,6 +42,11 @@ public interface DBCheckpoint {
|
||||||
*/
|
*/
|
||||||
long getLatestSequenceNumber();
|
long getLatestSequenceNumber();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Time taken in milliseconds for the checkpoint to be created.
|
||||||
|
*/
|
||||||
|
long checkpointCreationTimeTaken();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Destroy the contents of the specified checkpoint to ensure
|
* Destroy the contents of the specified checkpoint to ensure
|
||||||
* proper cleanup of the footprint on disk.
|
* proper cleanup of the footprint on disk.
|
||||||
|
|
|
@ -22,6 +22,8 @@ package org.apache.hadoop.utils.db;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.file.Path;
|
import java.nio.file.Path;
|
||||||
import java.nio.file.Paths;
|
import java.nio.file.Paths;
|
||||||
|
import java.time.Duration;
|
||||||
|
import java.time.Instant;
|
||||||
|
|
||||||
import org.apache.commons.io.FileUtils;
|
import org.apache.commons.io.FileUtils;
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
|
@ -41,7 +43,6 @@ public class RDBCheckpointManager {
|
||||||
public static final String RDB_CHECKPOINT_DIR_PREFIX = "rdb_checkpoint_";
|
public static final String RDB_CHECKPOINT_DIR_PREFIX = "rdb_checkpoint_";
|
||||||
private static final Logger LOG =
|
private static final Logger LOG =
|
||||||
LoggerFactory.getLogger(RDBCheckpointManager.class);
|
LoggerFactory.getLogger(RDBCheckpointManager.class);
|
||||||
public static final String JAVA_TMP_DIR = "java.io.tmpdir";
|
|
||||||
private String checkpointNamePrefix = "";
|
private String checkpointNamePrefix = "";
|
||||||
|
|
||||||
public RDBCheckpointManager(RocksDB rocksDB) {
|
public RDBCheckpointManager(RocksDB rocksDB) {
|
||||||
|
@ -79,12 +80,19 @@ public class RDBCheckpointManager {
|
||||||
checkpointDir += "_" + RDB_CHECKPOINT_DIR_PREFIX + currentTime;
|
checkpointDir += "_" + RDB_CHECKPOINT_DIR_PREFIX + currentTime;
|
||||||
|
|
||||||
Path checkpointPath = Paths.get(parentDir, checkpointDir);
|
Path checkpointPath = Paths.get(parentDir, checkpointDir);
|
||||||
|
Instant start = Instant.now();
|
||||||
checkpoint.createCheckpoint(checkpointPath.toString());
|
checkpoint.createCheckpoint(checkpointPath.toString());
|
||||||
|
Instant end = Instant.now();
|
||||||
|
|
||||||
|
long duration = Duration.between(start, end).toMillis();
|
||||||
|
LOG.debug("Created checkpoint at " + checkpointPath.toString() + " in "
|
||||||
|
+ duration + " milliseconds");
|
||||||
|
|
||||||
return new RocksDBCheckpoint(
|
return new RocksDBCheckpoint(
|
||||||
checkpointPath,
|
checkpointPath,
|
||||||
currentTime,
|
currentTime,
|
||||||
db.getLatestSequenceNumber()); //Best guesstimate here. Not accurate.
|
db.getLatestSequenceNumber(), //Best guesstimate here. Not accurate.
|
||||||
|
duration);
|
||||||
|
|
||||||
} catch (RocksDBException e) {
|
} catch (RocksDBException e) {
|
||||||
LOG.error("Unable to create RocksDB Snapshot.", e);
|
LOG.error("Unable to create RocksDB Snapshot.", e);
|
||||||
|
@ -97,13 +105,16 @@ public class RDBCheckpointManager {
|
||||||
private Path checkpointLocation;
|
private Path checkpointLocation;
|
||||||
private long checkpointTimestamp;
|
private long checkpointTimestamp;
|
||||||
private long latestSequenceNumber;
|
private long latestSequenceNumber;
|
||||||
|
private long checkpointCreationTimeTaken;
|
||||||
|
|
||||||
RocksDBCheckpoint(Path checkpointLocation,
|
RocksDBCheckpoint(Path checkpointLocation,
|
||||||
long snapshotTimestamp,
|
long snapshotTimestamp,
|
||||||
long latestSequenceNumber) {
|
long latestSequenceNumber,
|
||||||
|
long checkpointCreationTimeTaken) {
|
||||||
this.checkpointLocation = checkpointLocation;
|
this.checkpointLocation = checkpointLocation;
|
||||||
this.checkpointTimestamp = snapshotTimestamp;
|
this.checkpointTimestamp = snapshotTimestamp;
|
||||||
this.latestSequenceNumber = latestSequenceNumber;
|
this.latestSequenceNumber = latestSequenceNumber;
|
||||||
|
this.checkpointCreationTimeTaken = checkpointCreationTimeTaken;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -121,8 +132,14 @@ public class RDBCheckpointManager {
|
||||||
return this.latestSequenceNumber;
|
return this.latestSequenceNumber;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long checkpointCreationTimeTaken() {
|
||||||
|
return checkpointCreationTimeTaken;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void cleanupCheckpoint() throws IOException {
|
public void cleanupCheckpoint() throws IOException {
|
||||||
|
LOG.debug("Cleaning up checkpoint at " + checkpointLocation.toString());
|
||||||
FileUtils.deleteDirectory(checkpointLocation.toFile());
|
FileUtils.deleteDirectory(checkpointLocation.toFile());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -142,6 +142,8 @@ enum Result {
|
||||||
UNKNOWN_BCSID = 37;
|
UNKNOWN_BCSID = 37;
|
||||||
BCSID_MISMATCH = 38;
|
BCSID_MISMATCH = 38;
|
||||||
CONTAINER_NOT_OPEN = 39;
|
CONTAINER_NOT_OPEN = 39;
|
||||||
|
CONTAINER_MISSING = 40;
|
||||||
|
BLOCK_TOKEN_VERIFICATION_FAILED = 41;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -245,6 +247,10 @@ message ContainerDataProto {
|
||||||
optional ContainerType containerType = 10 [default = KeyValueContainer];
|
optional ContainerType containerType = 10 [default = KeyValueContainer];
|
||||||
}
|
}
|
||||||
|
|
||||||
|
message ContainerIdSetProto {
|
||||||
|
repeated int64 containerId = 1;
|
||||||
|
}
|
||||||
|
|
||||||
enum ContainerType {
|
enum ContainerType {
|
||||||
KeyValueContainer = 1;
|
KeyValueContainer = 1;
|
||||||
}
|
}
|
||||||
|
|
|
@ -42,6 +42,7 @@ message AllocateScmBlockRequestProto {
|
||||||
required hadoop.hdds.ReplicationFactor factor = 3;
|
required hadoop.hdds.ReplicationFactor factor = 3;
|
||||||
required string owner = 4;
|
required string owner = 4;
|
||||||
optional string traceID = 5;
|
optional string traceID = 5;
|
||||||
|
optional ExcludeListProto excludeList = 6;
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -199,6 +199,12 @@ enum ScmOps {
|
||||||
queryNode = 11;
|
queryNode = 11;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
message ExcludeListProto {
|
||||||
|
repeated string datanodes = 1;
|
||||||
|
repeated int64 containerIds = 2;
|
||||||
|
repeated PipelineID pipelineIds = 3;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Block ID that uniquely identify a block by SCM.
|
* Block ID that uniquely identify a block by SCM.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -1289,6 +1289,16 @@
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<name>hdds.scm.wait.time.after.chillmode.exit</name>
|
||||||
|
<value>5m</value>
|
||||||
|
<tag>HDDS,SCM,OPERATION</tag>
|
||||||
|
<description> After exiting chillmode, wait for configured interval of
|
||||||
|
time to start replication monitor and cleanup activities of unhealthy
|
||||||
|
pipelines.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
|
||||||
<property>
|
<property>
|
||||||
<name>hdds.scm.chillmode.enabled</name>
|
<name>hdds.scm.chillmode.enabled</name>
|
||||||
<value>true</value>
|
<value>true</value>
|
||||||
|
@ -1325,6 +1335,16 @@
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<name>hdds.scm.chillmode.atleast.one.node.reported.pipeline.pct</name>
|
||||||
|
<value>0.90</value>
|
||||||
|
<tag>HDDS,SCM,OPERATION</tag>
|
||||||
|
<description>
|
||||||
|
Percentage of pipelines, where at least one datanode is reported in the
|
||||||
|
pipeline.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
|
||||||
<property>
|
<property>
|
||||||
<name>hdds.container.action.max.limit</name>
|
<name>hdds.container.action.max.limit</name>
|
||||||
<value>20</value>
|
<value>20</value>
|
||||||
|
@ -1636,6 +1656,16 @@
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<name>ozone.om.ratis.server.role.check.interval</name>
|
||||||
|
<value>15s</value>
|
||||||
|
<tag>OZONE, OM, RATIS, MANAGEMENT</tag>
|
||||||
|
<description>The interval between OM leader performing a role
|
||||||
|
check on its ratis server. Ratis server informs OM if it
|
||||||
|
loses the leader role. The scheduled check is an secondary
|
||||||
|
check to ensure that the leader role is updated periodically
|
||||||
|
.</description>
|
||||||
|
</property>
|
||||||
|
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.acl.authorizer.class</name>
|
<name>ozone.acl.authorizer.class</name>
|
||||||
|
@ -1820,7 +1850,7 @@
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>ozone.scm.security.service.address</name>
|
<name>ozone.scm.security.service.address</name>
|
||||||
<value>0.0.0.0:9961</value>
|
<value/>
|
||||||
<tag>OZONE, HDDS, SECURITY</tag>
|
<tag>OZONE, HDDS, SECURITY</tag>
|
||||||
<description>Address of SCMSecurityProtocolServer.</description>
|
<description>Address of SCMSecurityProtocolServer.</description>
|
||||||
</property>
|
</property>
|
||||||
|
@ -2019,4 +2049,203 @@
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<name>hdds.datanode.http.kerberos.principal</name>
|
||||||
|
<value>HTTP/_HOST@EXAMPLE.COM</value>
|
||||||
|
<tag>HDDS, SECURITY, MANAGEMENT</tag>
|
||||||
|
<description>
|
||||||
|
The kerberos principal for the datanode http server.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hdds.datanode.http.kerberos.keytab</name>
|
||||||
|
<value>/etc/security/keytabs/HTTP.keytab</value>
|
||||||
|
<tag>HDDS, SECURITY, MANAGEMENT</tag>
|
||||||
|
<description>
|
||||||
|
The kerberos keytab file for datanode http server
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hdds.datanode.http-address</name>
|
||||||
|
<value>0.0.0.0:9882</value>
|
||||||
|
<tag>HDDS, MANAGEMENT</tag>
|
||||||
|
<description>
|
||||||
|
The address and the base port where the Datanode web ui will listen on.
|
||||||
|
If the port is 0 then the server will start on a free port.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hdds.datanode.http-bind-host</name>
|
||||||
|
<value>0.0.0.0</value>
|
||||||
|
<tag>HDDS, MANAGEMENT</tag>
|
||||||
|
<description>
|
||||||
|
The actual address the Datanode web server will bind to. If this
|
||||||
|
optional address is set, it overrides only the hostname portion of
|
||||||
|
hdds.datanode.http-address.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hdds.datanode.http.enabled</name>
|
||||||
|
<value>true</value>
|
||||||
|
<tag>HDDS, MANAGEMENT</tag>
|
||||||
|
<description>
|
||||||
|
Property to enable or disable Datanode web ui.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hdds.datanode.https-address</name>
|
||||||
|
<value>0.0.0.0:9883</value>
|
||||||
|
<tag>HDDS, MANAGEMENT, SECURITY</tag>
|
||||||
|
<description>
|
||||||
|
The address and the base port where the Datanode web UI will listen
|
||||||
|
on using HTTPS.
|
||||||
|
|
||||||
|
If the port is 0 then the server will start on a free port.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hdds.datanode.https-bind-host</name>
|
||||||
|
<value>0.0.0.0</value>
|
||||||
|
<tag>HDDS, MANAGEMENT, SECURITY</tag>
|
||||||
|
<description>
|
||||||
|
The actual address the Datanode web server will bind to using HTTPS.
|
||||||
|
If this optional address is set, it overrides only the hostname portion of
|
||||||
|
hdds.datanode.http-address.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>ozone.client.retry.max.attempts</name>
|
||||||
|
<value>10</value>
|
||||||
|
<description>
|
||||||
|
Max retry attempts for Ozone RpcClient talking to OzoneManagers.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>ozone.client.failover.max.attempts</name>
|
||||||
|
<value>15</value>
|
||||||
|
<description>
|
||||||
|
Expert only. The number of client failover attempts that should be
|
||||||
|
made before the failover is considered failed.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>ozone.client.failover.sleep.base.millis</name>
|
||||||
|
<value>500</value>
|
||||||
|
<description>
|
||||||
|
Expert only. The time to wait, in milliseconds, between failover
|
||||||
|
attempts increases exponentially as a function of the number of
|
||||||
|
attempts made so far, with a random factor of +/- 50%. This option
|
||||||
|
specifies the base value used in the failover calculation. The
|
||||||
|
first failover will retry immediately. The 2nd failover attempt
|
||||||
|
will delay at least ozone.client.failover.sleep.base.millis
|
||||||
|
milliseconds. And so on.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>ozone.client.failover.sleep.max.millis</name>
|
||||||
|
<value>15000</value>
|
||||||
|
<description>
|
||||||
|
Expert only. The time to wait, in milliseconds, between failover
|
||||||
|
attempts increases exponentially as a function of the number of
|
||||||
|
attempts made so far, with a random factor of +/- 50%. This option
|
||||||
|
specifies the maximum value to wait between failovers.
|
||||||
|
Specifically, the time between two failover attempts will not
|
||||||
|
exceed +/- 50% of ozone.client.failover.sleep.max.millis
|
||||||
|
milliseconds.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>ozone.recon.http.enabled</name>
|
||||||
|
<value>true</value>
|
||||||
|
<tag>RECON, MANAGEMENT</tag>
|
||||||
|
<description>
|
||||||
|
Property to enable or disable Recon web user interface.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>ozone.recon.http-address</name>
|
||||||
|
<value>0.0.0.0:9888</value>
|
||||||
|
<tag>RECON, MANAGEMENT</tag>
|
||||||
|
<description>
|
||||||
|
The address and the base port where the Recon web UI will listen on.
|
||||||
|
|
||||||
|
If the port is 0, then the server will start on a free port. However, it
|
||||||
|
is best to specify a well-known port, so it is easy to connect and see
|
||||||
|
the Recon management UI.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>ozone.recon.http-bind-host</name>
|
||||||
|
<value>0.0.0.0</value>
|
||||||
|
<tag>RECON, MANAGEMENT</tag>
|
||||||
|
<description>
|
||||||
|
The actual address the Recon server will bind to. If this optional
|
||||||
|
the address is set, it overrides only the hostname portion of
|
||||||
|
ozone.recon.http-address.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>ozone.recon.https-bind-host</name>
|
||||||
|
<value>0.0.0.0</value>
|
||||||
|
<tag>RECON, MANAGEMENT, SECURITY</tag>
|
||||||
|
<description>
|
||||||
|
The actual address the Recon web server will bind to using HTTPS.
|
||||||
|
If this optional address is set, it overrides only the hostname portion of
|
||||||
|
ozone.recon.https-address.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>ozone.recon.https-address</name>
|
||||||
|
<value>0.0.0.0:9889</value>
|
||||||
|
<tag>RECON, MANAGEMENT, SECURITY</tag>
|
||||||
|
<description>
|
||||||
|
The address and the base port where the Recon web UI will listen
|
||||||
|
on using HTTPS. If the port is 0 then the server will start on a free
|
||||||
|
port.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>ozone.recon.keytab.file</name>
|
||||||
|
<value/>
|
||||||
|
<tag>RECON, SECURITY</tag>
|
||||||
|
<description>
|
||||||
|
The keytab file for Kerberos authentication in Recon.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>ozone.recon.authentication.kerberos.principal</name>
|
||||||
|
<value/>
|
||||||
|
<tag>RECON</tag>
|
||||||
|
<description>The server principal used by Ozone Recon server. This is
|
||||||
|
typically set to HTTP/_HOST@REALM.TLD The SPNEGO server principal
|
||||||
|
begins with the prefix HTTP/ by convention.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>ozone.recon.container.db.cache.size.mb</name>
|
||||||
|
<value>128</value>
|
||||||
|
<tag>RECON, PERFORMANCE</tag>
|
||||||
|
<description>
|
||||||
|
The size of Recon DB cache in MB that used for caching files.
|
||||||
|
This value is set to an abnormally low value in the default configuration.
|
||||||
|
That is to make unit testing easy. Generally, this value should be set to
|
||||||
|
something like 16GB or more, if you intend to use Recon at scale.
|
||||||
|
|
||||||
|
A large value for this key allows a proportionally larger amount of Recon
|
||||||
|
container DB to be cached in memory. This makes Recon Container-Key
|
||||||
|
operations faster.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>ozone.recon.db.dirs</name>
|
||||||
|
<value/>
|
||||||
|
<tag>OZONE, RECON, STORAGE, PERFORMANCE</tag>
|
||||||
|
<description>
|
||||||
|
Directory where the Recon Server stores its metadata. This should
|
||||||
|
be specified as a single directory. If the directory does not
|
||||||
|
exist then the Recon will attempt to create it.
|
||||||
|
If undefined, then the Recon will log a warning and fallback to
|
||||||
|
ozone.metadata.dirs.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
</configuration>
|
</configuration>
|
||||||
|
|
|
@ -0,0 +1,143 @@
|
||||||
|
/**
|
||||||
|
* 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.hdds.conf;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.rules.TemporaryFolder;
|
||||||
|
|
||||||
|
import java.io.BufferedWriter;
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.FileWriter;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test class for OzoneConfiguration.
|
||||||
|
*/
|
||||||
|
public class TestOzoneConfiguration {
|
||||||
|
|
||||||
|
private Configuration conf;
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public TemporaryFolder tempConfigs = new TemporaryFolder();
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
conf = new OzoneConfiguration();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void startConfig(BufferedWriter out) throws IOException {
|
||||||
|
out.write("<?xml version=\"1.0\"?>\n");
|
||||||
|
out.write("<configuration>\n");
|
||||||
|
}
|
||||||
|
|
||||||
|
private void endConfig(BufferedWriter out) throws IOException {
|
||||||
|
out.write("</configuration>\n");
|
||||||
|
out.flush();
|
||||||
|
out.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetAllPropertiesByTags() throws Exception {
|
||||||
|
File coreDefault = tempConfigs.newFile("core-default-test.xml");
|
||||||
|
File coreSite = tempConfigs.newFile("core-site-test.xml");
|
||||||
|
try (BufferedWriter out = new BufferedWriter(new FileWriter(coreDefault))) {
|
||||||
|
startConfig(out);
|
||||||
|
appendProperty(out, "hadoop.tags.system", "YARN,HDFS,NAMENODE");
|
||||||
|
appendProperty(out, "hadoop.tags.custom", "MYCUSTOMTAG");
|
||||||
|
appendPropertyByTag(out, "dfs.cblock.trace.io", "false", "YARN");
|
||||||
|
appendPropertyByTag(out, "dfs.replication", "1", "HDFS");
|
||||||
|
appendPropertyByTag(out, "dfs.namenode.logging.level", "INFO",
|
||||||
|
"NAMENODE");
|
||||||
|
appendPropertyByTag(out, "dfs.random.key", "XYZ", "MYCUSTOMTAG");
|
||||||
|
endConfig(out);
|
||||||
|
|
||||||
|
Path fileResource = new Path(coreDefault.getAbsolutePath());
|
||||||
|
conf.addResource(fileResource);
|
||||||
|
Assert.assertEquals(conf.getAllPropertiesByTag("MYCUSTOMTAG")
|
||||||
|
.getProperty("dfs.random.key"), "XYZ");
|
||||||
|
}
|
||||||
|
|
||||||
|
try (BufferedWriter out = new BufferedWriter(new FileWriter(coreSite))) {
|
||||||
|
startConfig(out);
|
||||||
|
appendProperty(out, "dfs.random.key", "ABC");
|
||||||
|
appendProperty(out, "dfs.replication", "3");
|
||||||
|
appendProperty(out, "dfs.cblock.trace.io", "true");
|
||||||
|
endConfig(out);
|
||||||
|
|
||||||
|
Path fileResource = new Path(coreSite.getAbsolutePath());
|
||||||
|
conf.addResource(fileResource);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Test if values are getting overridden even without tags being present
|
||||||
|
Assert.assertEquals("3", conf.getAllPropertiesByTag("HDFS")
|
||||||
|
.getProperty("dfs.replication"));
|
||||||
|
Assert.assertEquals("ABC", conf.getAllPropertiesByTag("MYCUSTOMTAG")
|
||||||
|
.getProperty("dfs.random.key"));
|
||||||
|
Assert.assertEquals("true", conf.getAllPropertiesByTag("YARN")
|
||||||
|
.getProperty("dfs.cblock.trace.io"));
|
||||||
|
}
|
||||||
|
|
||||||
|
private void appendProperty(BufferedWriter out, String name, String val)
|
||||||
|
throws IOException {
|
||||||
|
this.appendProperty(out, name, val, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void appendProperty(BufferedWriter out, String name, String val,
|
||||||
|
boolean isFinal) throws IOException {
|
||||||
|
out.write("<property>");
|
||||||
|
out.write("<name>");
|
||||||
|
out.write(name);
|
||||||
|
out.write("</name>");
|
||||||
|
out.write("<value>");
|
||||||
|
out.write(val);
|
||||||
|
out.write("</value>");
|
||||||
|
if (isFinal) {
|
||||||
|
out.write("<final>true</final>");
|
||||||
|
}
|
||||||
|
out.write("</property>\n");
|
||||||
|
}
|
||||||
|
|
||||||
|
private void appendPropertyByTag(BufferedWriter out, String name, String val,
|
||||||
|
String tags) throws IOException {
|
||||||
|
this.appendPropertyByTag(out, name, val, false, tags);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void appendPropertyByTag(BufferedWriter out, String name, String val,
|
||||||
|
boolean isFinal,
|
||||||
|
String tag) throws IOException {
|
||||||
|
out.write("<property>");
|
||||||
|
out.write("<name>");
|
||||||
|
out.write(name);
|
||||||
|
out.write("</name>");
|
||||||
|
out.write("<value>");
|
||||||
|
out.write(val);
|
||||||
|
out.write("</value>");
|
||||||
|
if (isFinal) {
|
||||||
|
out.write("<final>true</final>");
|
||||||
|
}
|
||||||
|
out.write("<tag>");
|
||||||
|
out.write(tag);
|
||||||
|
out.write("</tag>");
|
||||||
|
out.write("</property>\n");
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,22 @@
|
||||||
|
/*
|
||||||
|
* 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.
|
||||||
|
*/
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This package contains the OzoneConfiguration related tests.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.hdds.conf;
|
|
@ -47,9 +47,10 @@ public class MockApprover extends BaseApprover {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public X509CertificateHolder sign(SecurityConfig config, PrivateKey caPrivate,
|
public X509CertificateHolder sign(SecurityConfig config, PrivateKey caPrivate,
|
||||||
X509CertificateHolder caCertificate,
|
X509CertificateHolder caCertificate,
|
||||||
Date validFrom, Date validTill,
|
Date validFrom, Date validTill,
|
||||||
PKCS10CertificationRequest request)
|
PKCS10CertificationRequest request,
|
||||||
|
String scmId, String clusterId)
|
||||||
throws IOException, OperatorCreationException {
|
throws IOException, OperatorCreationException {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
|
||||||
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
||||||
import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
|
import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
|
||||||
import org.apache.hadoop.hdds.security.x509.keys.HDDSKeyGenerator;
|
import org.apache.hadoop.hdds.security.x509.keys.HDDSKeyGenerator;
|
||||||
|
import org.apache.hadoop.test.LambdaTestUtils;
|
||||||
import org.bouncycastle.cert.X509CertificateHolder;
|
import org.bouncycastle.cert.X509CertificateHolder;
|
||||||
import org.bouncycastle.pkcs.PKCS10CertificationRequest;
|
import org.bouncycastle.pkcs.PKCS10CertificationRequest;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
@ -139,14 +140,57 @@ public class TestDefaultCAServer {
|
||||||
public void testRequestCertificate() throws IOException,
|
public void testRequestCertificate() throws IOException,
|
||||||
ExecutionException, InterruptedException,
|
ExecutionException, InterruptedException,
|
||||||
NoSuchProviderException, NoSuchAlgorithmException {
|
NoSuchProviderException, NoSuchAlgorithmException {
|
||||||
|
String scmId = RandomStringUtils.randomAlphabetic(4);
|
||||||
|
String clusterId = RandomStringUtils.randomAlphabetic(4);
|
||||||
|
KeyPair keyPair =
|
||||||
|
new HDDSKeyGenerator(conf).generateKey();
|
||||||
|
PKCS10CertificationRequest csr = new CertificateSignRequest.Builder()
|
||||||
|
.addDnsName("hadoop.apache.org")
|
||||||
|
.addIpAddress("8.8.8.8")
|
||||||
|
.setCA(false)
|
||||||
|
.setClusterID(clusterId)
|
||||||
|
.setScmID(scmId)
|
||||||
|
.setSubject("Ozone Cluster")
|
||||||
|
.setConfiguration(conf)
|
||||||
|
.setKey(keyPair)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
// Let us convert this to a string to mimic the common use case.
|
||||||
|
String csrString = CertificateSignRequest.getEncodedString(csr);
|
||||||
|
|
||||||
|
CertificateServer testCA = new DefaultCAServer("testCA",
|
||||||
|
clusterId, scmId, caStore);
|
||||||
|
testCA.init(new SecurityConfig(conf),
|
||||||
|
CertificateServer.CAType.SELF_SIGNED_CA);
|
||||||
|
|
||||||
|
Future<X509CertificateHolder> holder = testCA.requestCertificate(csrString,
|
||||||
|
CertificateApprover.ApprovalType.TESTING_AUTOMATIC);
|
||||||
|
// Right now our calls are synchronous. Eventually this will have to wait.
|
||||||
|
assertTrue(holder.isDone());
|
||||||
|
assertNotNull(holder.get());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests that we are able
|
||||||
|
* to create a Test CA, creates it own self-Signed CA and then issue a
|
||||||
|
* certificate based on a CSR when scmId and clusterId are not set in
|
||||||
|
* csr subject.
|
||||||
|
* @throws SCMSecurityException - on ERROR.
|
||||||
|
* @throws ExecutionException - on ERROR.
|
||||||
|
* @throws InterruptedException - on ERROR.
|
||||||
|
* @throws NoSuchProviderException - on ERROR.
|
||||||
|
* @throws NoSuchAlgorithmException - on ERROR.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testRequestCertificateWithInvalidSubject() throws IOException,
|
||||||
|
ExecutionException, InterruptedException,
|
||||||
|
NoSuchProviderException, NoSuchAlgorithmException {
|
||||||
KeyPair keyPair =
|
KeyPair keyPair =
|
||||||
new HDDSKeyGenerator(conf).generateKey();
|
new HDDSKeyGenerator(conf).generateKey();
|
||||||
PKCS10CertificationRequest csr = new CertificateSignRequest.Builder()
|
PKCS10CertificationRequest csr = new CertificateSignRequest.Builder()
|
||||||
.addDnsName("hadoop.apache.org")
|
.addDnsName("hadoop.apache.org")
|
||||||
.addIpAddress("8.8.8.8")
|
.addIpAddress("8.8.8.8")
|
||||||
.setCA(false)
|
.setCA(false)
|
||||||
.setClusterID("ClusterID")
|
|
||||||
.setScmID("SCMID")
|
|
||||||
.setSubject("Ozone Cluster")
|
.setSubject("Ozone Cluster")
|
||||||
.setConfiguration(conf)
|
.setConfiguration(conf)
|
||||||
.setKey(keyPair)
|
.setKey(keyPair)
|
||||||
|
@ -168,4 +212,40 @@ public class TestDefaultCAServer {
|
||||||
assertNotNull(holder.get());
|
assertNotNull(holder.get());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRequestCertificateWithInvalidSubjectFailure()
|
||||||
|
throws Exception {
|
||||||
|
KeyPair keyPair =
|
||||||
|
new HDDSKeyGenerator(conf).generateKey();
|
||||||
|
PKCS10CertificationRequest csr = new CertificateSignRequest.Builder()
|
||||||
|
.addDnsName("hadoop.apache.org")
|
||||||
|
.addIpAddress("8.8.8.8")
|
||||||
|
.setCA(false)
|
||||||
|
.setScmID("wrong one")
|
||||||
|
.setClusterID("223432rf")
|
||||||
|
.setSubject("Ozone Cluster")
|
||||||
|
.setConfiguration(conf)
|
||||||
|
.setKey(keyPair)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
// Let us convert this to a string to mimic the common use case.
|
||||||
|
String csrString = CertificateSignRequest.getEncodedString(csr);
|
||||||
|
|
||||||
|
CertificateServer testCA = new DefaultCAServer("testCA",
|
||||||
|
RandomStringUtils.randomAlphabetic(4),
|
||||||
|
RandomStringUtils.randomAlphabetic(4), caStore);
|
||||||
|
testCA.init(new SecurityConfig(conf),
|
||||||
|
CertificateServer.CAType.SELF_SIGNED_CA);
|
||||||
|
|
||||||
|
LambdaTestUtils.intercept(ExecutionException.class, "ScmId and " +
|
||||||
|
"ClusterId in CSR subject are incorrect",
|
||||||
|
() -> {
|
||||||
|
Future<X509CertificateHolder> holder =
|
||||||
|
testCA.requestCertificate(csrString,
|
||||||
|
CertificateApprover.ApprovalType.TESTING_AUTOMATIC);
|
||||||
|
holder.isDone();
|
||||||
|
holder.get();
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -61,7 +61,6 @@ public class TestCertificateClientInit {
|
||||||
|
|
||||||
private CertificateClient dnCertificateClient;
|
private CertificateClient dnCertificateClient;
|
||||||
private CertificateClient omCertificateClient;
|
private CertificateClient omCertificateClient;
|
||||||
private static final String COMP = "test";
|
|
||||||
private HDDSKeyGenerator keyGenerator;
|
private HDDSKeyGenerator keyGenerator;
|
||||||
private Path metaDirPath;
|
private Path metaDirPath;
|
||||||
private SecurityConfig securityConfig;
|
private SecurityConfig securityConfig;
|
||||||
|
@ -97,11 +96,11 @@ public class TestCertificateClientInit {
|
||||||
metaDirPath = Paths.get(path, "test");
|
metaDirPath = Paths.get(path, "test");
|
||||||
config.set(HDDS_METADATA_DIR_NAME, metaDirPath.toString());
|
config.set(HDDS_METADATA_DIR_NAME, metaDirPath.toString());
|
||||||
securityConfig = new SecurityConfig(config);
|
securityConfig = new SecurityConfig(config);
|
||||||
dnCertificateClient = new DNCertificateClient(securityConfig, COMP);
|
dnCertificateClient = new DNCertificateClient(securityConfig);
|
||||||
omCertificateClient = new OMCertificateClient(securityConfig, COMP);
|
omCertificateClient = new OMCertificateClient(securityConfig);
|
||||||
keyGenerator = new HDDSKeyGenerator(securityConfig);
|
keyGenerator = new HDDSKeyGenerator(securityConfig);
|
||||||
keyCodec = new KeyCodec(securityConfig, COMP);
|
keyCodec = new KeyCodec(securityConfig);
|
||||||
Files.createDirectories(securityConfig.getKeyLocation(COMP));
|
Files.createDirectories(securityConfig.getKeyLocation());
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
@ -118,7 +117,7 @@ public class TestCertificateClientInit {
|
||||||
if (pvtKeyPresent) {
|
if (pvtKeyPresent) {
|
||||||
keyCodec.writePrivateKey(keyPair.getPrivate());
|
keyCodec.writePrivateKey(keyPair.getPrivate());
|
||||||
} else {
|
} else {
|
||||||
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
|
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||||
.toString(), securityConfig.getPrivateKeyFileName()).toFile());
|
.toString(), securityConfig.getPrivateKeyFileName()).toFile());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -127,7 +126,7 @@ public class TestCertificateClientInit {
|
||||||
keyCodec.writePublicKey(keyPair.getPublic());
|
keyCodec.writePublicKey(keyPair.getPublic());
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
|
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||||
.toString(), securityConfig.getPublicKeyFileName()).toFile());
|
.toString(), securityConfig.getPublicKeyFileName()).toFile());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -135,11 +134,11 @@ public class TestCertificateClientInit {
|
||||||
X509Certificate x509Certificate = KeyStoreTestUtil.generateCertificate(
|
X509Certificate x509Certificate = KeyStoreTestUtil.generateCertificate(
|
||||||
"CN=Test", keyPair, 10, securityConfig.getSignatureAlgo());
|
"CN=Test", keyPair, 10, securityConfig.getSignatureAlgo());
|
||||||
|
|
||||||
CertificateCodec codec = new CertificateCodec(securityConfig, COMP);
|
CertificateCodec codec = new CertificateCodec(securityConfig);
|
||||||
codec.writeCertificate(new X509CertificateHolder(
|
codec.writeCertificate(new X509CertificateHolder(
|
||||||
x509Certificate.getEncoded()));
|
x509Certificate.getEncoded()));
|
||||||
} else {
|
} else {
|
||||||
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
|
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||||
.toString(), securityConfig.getCertificateFileName()).toFile());
|
.toString(), securityConfig.getCertificateFileName()).toFile());
|
||||||
}
|
}
|
||||||
InitResponse response = dnCertificateClient.init();
|
InitResponse response = dnCertificateClient.init();
|
||||||
|
@ -148,10 +147,10 @@ public class TestCertificateClientInit {
|
||||||
|
|
||||||
if (!response.equals(FAILURE)) {
|
if (!response.equals(FAILURE)) {
|
||||||
assertTrue(OzoneSecurityUtil.checkIfFileExist(
|
assertTrue(OzoneSecurityUtil.checkIfFileExist(
|
||||||
securityConfig.getKeyLocation(COMP),
|
securityConfig.getKeyLocation(),
|
||||||
securityConfig.getPrivateKeyFileName()));
|
securityConfig.getPrivateKeyFileName()));
|
||||||
assertTrue(OzoneSecurityUtil.checkIfFileExist(
|
assertTrue(OzoneSecurityUtil.checkIfFileExist(
|
||||||
securityConfig.getKeyLocation(COMP),
|
securityConfig.getKeyLocation(),
|
||||||
securityConfig.getPublicKeyFileName()));
|
securityConfig.getPublicKeyFileName()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -162,7 +161,7 @@ public class TestCertificateClientInit {
|
||||||
if (pvtKeyPresent) {
|
if (pvtKeyPresent) {
|
||||||
keyCodec.writePrivateKey(keyPair.getPrivate());
|
keyCodec.writePrivateKey(keyPair.getPrivate());
|
||||||
} else {
|
} else {
|
||||||
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
|
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||||
.toString(), securityConfig.getPrivateKeyFileName()).toFile());
|
.toString(), securityConfig.getPrivateKeyFileName()).toFile());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -171,7 +170,7 @@ public class TestCertificateClientInit {
|
||||||
keyCodec.writePublicKey(keyPair.getPublic());
|
keyCodec.writePublicKey(keyPair.getPublic());
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
|
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||||
.toString(), securityConfig.getPublicKeyFileName()).toFile());
|
.toString(), securityConfig.getPublicKeyFileName()).toFile());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -179,11 +178,11 @@ public class TestCertificateClientInit {
|
||||||
X509Certificate x509Certificate = KeyStoreTestUtil.generateCertificate(
|
X509Certificate x509Certificate = KeyStoreTestUtil.generateCertificate(
|
||||||
"CN=Test", keyPair, 10, securityConfig.getSignatureAlgo());
|
"CN=Test", keyPair, 10, securityConfig.getSignatureAlgo());
|
||||||
|
|
||||||
CertificateCodec codec = new CertificateCodec(securityConfig, COMP);
|
CertificateCodec codec = new CertificateCodec(securityConfig);
|
||||||
codec.writeCertificate(new X509CertificateHolder(
|
codec.writeCertificate(new X509CertificateHolder(
|
||||||
x509Certificate.getEncoded()));
|
x509Certificate.getEncoded()));
|
||||||
} else {
|
} else {
|
||||||
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
|
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||||
.toString(), securityConfig.getCertificateFileName()).toFile());
|
.toString(), securityConfig.getCertificateFileName()).toFile());
|
||||||
}
|
}
|
||||||
InitResponse response = omCertificateClient.init();
|
InitResponse response = omCertificateClient.init();
|
||||||
|
@ -196,10 +195,10 @@ public class TestCertificateClientInit {
|
||||||
|
|
||||||
if (!response.equals(FAILURE)) {
|
if (!response.equals(FAILURE)) {
|
||||||
assertTrue(OzoneSecurityUtil.checkIfFileExist(
|
assertTrue(OzoneSecurityUtil.checkIfFileExist(
|
||||||
securityConfig.getKeyLocation(COMP),
|
securityConfig.getKeyLocation(),
|
||||||
securityConfig.getPrivateKeyFileName()));
|
securityConfig.getPrivateKeyFileName()));
|
||||||
assertTrue(OzoneSecurityUtil.checkIfFileExist(
|
assertTrue(OzoneSecurityUtil.checkIfFileExist(
|
||||||
securityConfig.getKeyLocation(COMP),
|
securityConfig.getKeyLocation(),
|
||||||
securityConfig.getPublicKeyFileName()));
|
securityConfig.getPublicKeyFileName()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -64,7 +64,6 @@ public class TestDefaultCertificateClient {
|
||||||
|
|
||||||
private OMCertificateClient omCertClient;
|
private OMCertificateClient omCertClient;
|
||||||
private DNCertificateClient dnCertClient;
|
private DNCertificateClient dnCertClient;
|
||||||
private static final String COMP = "test";
|
|
||||||
private HDDSKeyGenerator keyGenerator;
|
private HDDSKeyGenerator keyGenerator;
|
||||||
private Path metaDirPath;
|
private Path metaDirPath;
|
||||||
private SecurityConfig securityConfig;
|
private SecurityConfig securityConfig;
|
||||||
|
@ -81,13 +80,13 @@ public class TestDefaultCertificateClient {
|
||||||
securityConfig = new SecurityConfig(config);
|
securityConfig = new SecurityConfig(config);
|
||||||
getCertClient();
|
getCertClient();
|
||||||
keyGenerator = new HDDSKeyGenerator(securityConfig);
|
keyGenerator = new HDDSKeyGenerator(securityConfig);
|
||||||
keyCodec = new KeyCodec(securityConfig, COMP);
|
keyCodec = new KeyCodec(securityConfig);
|
||||||
Files.createDirectories(securityConfig.getKeyLocation(COMP));
|
Files.createDirectories(securityConfig.getKeyLocation());
|
||||||
}
|
}
|
||||||
|
|
||||||
private void getCertClient() {
|
private void getCertClient() {
|
||||||
omCertClient = new OMCertificateClient(securityConfig, COMP);
|
omCertClient = new OMCertificateClient(securityConfig);
|
||||||
dnCertClient = new DNCertificateClient(securityConfig, COMP);
|
dnCertClient = new DNCertificateClient(securityConfig);
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
@ -160,7 +159,7 @@ public class TestDefaultCertificateClient {
|
||||||
() -> omCertClient.signDataStream(IOUtils.toInputStream(data,
|
() -> omCertClient.signDataStream(IOUtils.toInputStream(data,
|
||||||
UTF)));
|
UTF)));
|
||||||
|
|
||||||
KeyPair keyPair = generateKeyPairFiles();
|
generateKeyPairFiles();
|
||||||
byte[] sign = omCertClient.signDataStream(IOUtils.toInputStream(data,
|
byte[] sign = omCertClient.signDataStream(IOUtils.toInputStream(data,
|
||||||
UTF));
|
UTF));
|
||||||
validateHash(sign, data.getBytes());
|
validateHash(sign, data.getBytes());
|
||||||
|
@ -247,11 +246,11 @@ public class TestDefaultCertificateClient {
|
||||||
omClientLog.clearOutput();
|
omClientLog.clearOutput();
|
||||||
|
|
||||||
// Case 1. Expect failure when keypair validation fails.
|
// Case 1. Expect failure when keypair validation fails.
|
||||||
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
|
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||||
.toString(), securityConfig.getPrivateKeyFileName()).toFile());
|
.toString(), securityConfig.getPrivateKeyFileName()).toFile());
|
||||||
keyCodec.writePrivateKey(keyPair.getPrivate());
|
keyCodec.writePrivateKey(keyPair.getPrivate());
|
||||||
|
|
||||||
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
|
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||||
.toString(), securityConfig.getPublicKeyFileName()).toFile());
|
.toString(), securityConfig.getPublicKeyFileName()).toFile());
|
||||||
keyCodec.writePublicKey(keyPair2.getPublic());
|
keyCodec.writePublicKey(keyPair2.getPublic());
|
||||||
|
|
||||||
|
@ -272,12 +271,12 @@ public class TestDefaultCertificateClient {
|
||||||
// Case 2. Expect failure when certificate is generated from different
|
// Case 2. Expect failure when certificate is generated from different
|
||||||
// private key and keypair validation fails.
|
// private key and keypair validation fails.
|
||||||
getCertClient();
|
getCertClient();
|
||||||
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
|
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||||
.toString(), securityConfig.getCertificateFileName()).toFile());
|
.toString(), securityConfig.getCertificateFileName()).toFile());
|
||||||
X509Certificate x509Certificate = KeyStoreTestUtil.generateCertificate(
|
X509Certificate x509Certificate = KeyStoreTestUtil.generateCertificate(
|
||||||
"CN=Test", keyGenerator.generateKey(), 10,
|
"CN=Test", keyGenerator.generateKey(), 10,
|
||||||
securityConfig.getSignatureAlgo());
|
securityConfig.getSignatureAlgo());
|
||||||
CertificateCodec codec = new CertificateCodec(securityConfig, COMP);
|
CertificateCodec codec = new CertificateCodec(securityConfig);
|
||||||
codec.writeCertificate(new X509CertificateHolder(
|
codec.writeCertificate(new X509CertificateHolder(
|
||||||
x509Certificate.getEncoded()));
|
x509Certificate.getEncoded()));
|
||||||
|
|
||||||
|
@ -299,7 +298,7 @@ public class TestDefaultCertificateClient {
|
||||||
|
|
||||||
// Re write the correct public key.
|
// Re write the correct public key.
|
||||||
getCertClient();
|
getCertClient();
|
||||||
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
|
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||||
.toString(), securityConfig.getPublicKeyFileName()).toFile());
|
.toString(), securityConfig.getPublicKeyFileName()).toFile());
|
||||||
keyCodec.writePublicKey(keyPair.getPublic());
|
keyCodec.writePublicKey(keyPair.getPublic());
|
||||||
|
|
||||||
|
@ -319,7 +318,7 @@ public class TestDefaultCertificateClient {
|
||||||
|
|
||||||
// Case 4. Failure when public key recovery fails.
|
// Case 4. Failure when public key recovery fails.
|
||||||
getCertClient();
|
getCertClient();
|
||||||
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
|
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||||
.toString(), securityConfig.getPublicKeyFileName()).toFile());
|
.toString(), securityConfig.getPublicKeyFileName()).toFile());
|
||||||
|
|
||||||
// Check for DN.
|
// Check for DN.
|
||||||
|
|
|
@ -213,24 +213,6 @@ public class TestCertificateSignRequest {
|
||||||
builder.setSubject(subject);
|
builder.setSubject(subject);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Now try with blank/null SCM ID
|
|
||||||
try {
|
|
||||||
builder.setScmID(null);
|
|
||||||
builder.build();
|
|
||||||
Assert.fail("Null/Blank SCM ID should have thrown.");
|
|
||||||
} catch (IllegalArgumentException e) {
|
|
||||||
builder.setScmID(scmID);
|
|
||||||
}
|
|
||||||
|
|
||||||
// Now try with blank/null SCM ID
|
|
||||||
try {
|
|
||||||
builder.setClusterID(null);
|
|
||||||
builder.build();
|
|
||||||
Assert.fail("Null/Blank Cluster ID should have thrown.");
|
|
||||||
} catch (IllegalArgumentException e) {
|
|
||||||
builder.setClusterID(clusterID);
|
|
||||||
}
|
|
||||||
|
|
||||||
// Now try with invalid IP address
|
// Now try with invalid IP address
|
||||||
try {
|
try {
|
||||||
builder.addIpAddress("255.255.255.*");
|
builder.addIpAddress("255.255.255.*");
|
||||||
|
|
|
@ -163,6 +163,58 @@ public class TestMetadataStore {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIteratorPrefixSeek() throws Exception {
|
||||||
|
Configuration conf = new OzoneConfiguration();
|
||||||
|
conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL, storeImpl);
|
||||||
|
File dbDir = GenericTestUtils.getRandomizedTestDir();
|
||||||
|
MetadataStore dbStore = MetadataStoreBuilder.newBuilder()
|
||||||
|
.setConf(conf)
|
||||||
|
.setCreateIfMissing(true)
|
||||||
|
.setDbFile(dbDir)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
for (int i = 0; i < 5; i++) {
|
||||||
|
dbStore.put(getBytes("a" + i), getBytes("a-value" + i));
|
||||||
|
}
|
||||||
|
|
||||||
|
for (int i = 0; i < 5; i++) {
|
||||||
|
dbStore.put(getBytes("b" + i), getBytes("b-value" + i));
|
||||||
|
}
|
||||||
|
|
||||||
|
for (int i = 0; i < 5; i++) {
|
||||||
|
dbStore.put(getBytes("c" + i), getBytes("c-value" + i));
|
||||||
|
}
|
||||||
|
|
||||||
|
for (int i = 5; i < 10; i++) {
|
||||||
|
dbStore.put(getBytes("b" + i), getBytes("b-value" + i));
|
||||||
|
}
|
||||||
|
|
||||||
|
for (int i = 5; i < 10; i++) {
|
||||||
|
dbStore.put(getBytes("a" + i), getBytes("a-value" + i));
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
MetaStoreIterator<KeyValue> metaStoreIterator = dbStore.iterator();
|
||||||
|
metaStoreIterator.prefixSeek(getBytes("b"));
|
||||||
|
int i = 0;
|
||||||
|
while (metaStoreIterator.hasNext()) {
|
||||||
|
KeyValue val = metaStoreIterator.next();
|
||||||
|
String key = getString(val.getKey());
|
||||||
|
if (key.startsWith("b")) {
|
||||||
|
assertEquals("b-value" + i, getString(val.getValue()));
|
||||||
|
} else {
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
i++;
|
||||||
|
}
|
||||||
|
assertTrue(i == 10);
|
||||||
|
dbStore.close();
|
||||||
|
dbStore.destroy();
|
||||||
|
FileUtils.deleteDirectory(dbDir);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMetaStoreConfigDifferentFromType() throws IOException {
|
public void testMetaStoreConfigDifferentFromType() throws IOException {
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,86 @@
|
||||||
|
/**
|
||||||
|
* 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 java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hdds.HddsConfigKeys;
|
||||||
|
import org.apache.hadoop.hdds.server.BaseHttpServer;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Simple http server to provide basic monitoring for hdds datanode.
|
||||||
|
* <p>
|
||||||
|
* This server is used to access default /conf /prom /prof endpoints.
|
||||||
|
*/
|
||||||
|
public class HddsDatanodeHttpServer extends BaseHttpServer {
|
||||||
|
|
||||||
|
public HddsDatanodeHttpServer(Configuration conf) throws IOException {
|
||||||
|
super(conf, "hddsDatanode");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getHttpAddressKey() {
|
||||||
|
return HddsConfigKeys.HDDS_DATANODE_HTTP_ADDRESS_KEY;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getHttpBindHostKey() {
|
||||||
|
return HddsConfigKeys.HDDS_DATANODE_HTTP_BIND_HOST_KEY;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getHttpsAddressKey() {
|
||||||
|
return HddsConfigKeys.HDDS_DATANODE_HTTPS_ADDRESS_KEY;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getHttpsBindHostKey() {
|
||||||
|
return HddsConfigKeys.HDDS_DATANODE_HTTPS_BIND_HOST_KEY;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getBindHostDefault() {
|
||||||
|
return HddsConfigKeys.HDDS_DATANODE_HTTP_BIND_HOST_DEFAULT;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected int getHttpBindPortDefault() {
|
||||||
|
return HddsConfigKeys.HDDS_DATANODE_HTTP_BIND_PORT_DEFAULT;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected int getHttpsBindPortDefault() {
|
||||||
|
return HddsConfigKeys.HDDS_DATANODE_HTTPS_BIND_PORT_DEFAULT;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getKeytabFile() {
|
||||||
|
return HddsConfigKeys.HDDS_DATANODE_HTTP_KERBEROS_KEYTAB_FILE_KEY;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getSpnegoPrincipal() {
|
||||||
|
return HddsConfigKeys.HDDS_DATANODE_HTTP_KERBEROS_PRINCIPAL_KEY;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getEnabledKey() {
|
||||||
|
return HddsConfigKeys.HDDS_DATANODE_HTTP_ENABLED_KEY;
|
||||||
|
}
|
||||||
|
}
|
|
@ -26,17 +26,24 @@ import org.apache.hadoop.hdds.cli.GenericCli;
|
||||||
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
|
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
|
import org.apache.hadoop.hdds.protocol.SCMSecurityProtocol;
|
||||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
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;
|
||||||
|
import org.apache.hadoop.hdds.security.x509.certificate.client.DNCertificateClient;
|
||||||
|
import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
|
||||||
|
import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
|
||||||
import org.apache.hadoop.hdds.tracing.TracingUtil;
|
import org.apache.hadoop.hdds.tracing.TracingUtil;
|
||||||
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
||||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
||||||
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
|
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
|
||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
||||||
import org.apache.hadoop.security.SecurityUtil;
|
import org.apache.hadoop.security.SecurityUtil;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.security.authentication.client.AuthenticationException;
|
import org.apache.hadoop.security.authentication.client.AuthenticationException;
|
||||||
import org.apache.hadoop.util.ServicePlugin;
|
import org.apache.hadoop.util.ServicePlugin;
|
||||||
import org.apache.hadoop.util.StringUtils;
|
import org.apache.hadoop.util.StringUtils;
|
||||||
|
import org.bouncycastle.pkcs.PKCS10CertificationRequest;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
import picocli.CommandLine.Command;
|
import picocli.CommandLine.Command;
|
||||||
|
@ -44,9 +51,13 @@ import picocli.CommandLine.Command;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.InetAddress;
|
import java.net.InetAddress;
|
||||||
|
import java.security.KeyPair;
|
||||||
|
import java.security.cert.CertificateException;
|
||||||
|
import java.security.cert.X509Certificate;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
|
||||||
import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_PLUGINS_KEY;
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_PLUGINS_KEY;
|
||||||
import static org.apache.hadoop.util.ExitUtil.terminate;
|
import static org.apache.hadoop.util.ExitUtil.terminate;
|
||||||
|
|
||||||
|
@ -67,6 +78,9 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
|
||||||
private DatanodeDetails datanodeDetails;
|
private DatanodeDetails datanodeDetails;
|
||||||
private DatanodeStateMachine datanodeStateMachine;
|
private DatanodeStateMachine datanodeStateMachine;
|
||||||
private List<ServicePlugin> plugins;
|
private List<ServicePlugin> plugins;
|
||||||
|
private CertificateClient dnCertClient;
|
||||||
|
private String component;
|
||||||
|
private HddsDatanodeHttpServer httpServer;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Default constructor.
|
* Default constructor.
|
||||||
|
@ -133,6 +147,10 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static Logger getLogger() {
|
||||||
|
return LOG;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Starts HddsDatanode services.
|
* Starts HddsDatanode services.
|
||||||
*
|
*
|
||||||
|
@ -158,13 +176,15 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
|
||||||
.substring(0, 8));
|
.substring(0, 8));
|
||||||
LOG.info("HddsDatanodeService host:{} ip:{}", hostname, ip);
|
LOG.info("HddsDatanodeService host:{} ip:{}", hostname, ip);
|
||||||
// Authenticate Hdds Datanode service if security is enabled
|
// Authenticate Hdds Datanode service if security is enabled
|
||||||
if (conf.getBoolean(OzoneConfigKeys.OZONE_SECURITY_ENABLED_KEY,
|
if (OzoneSecurityUtil.isSecurityEnabled(conf)) {
|
||||||
true)) {
|
component = "dn-" + datanodeDetails.getUuidString();
|
||||||
|
|
||||||
|
dnCertClient = new DNCertificateClient(new SecurityConfig(conf));
|
||||||
|
|
||||||
if (SecurityUtil.getAuthenticationMethod(conf).equals(
|
if (SecurityUtil.getAuthenticationMethod(conf).equals(
|
||||||
UserGroupInformation.AuthenticationMethod.KERBEROS)) {
|
UserGroupInformation.AuthenticationMethod.KERBEROS)) {
|
||||||
LOG.debug("Ozone security is enabled. Attempting login for Hdds " +
|
LOG.info("Ozone security is enabled. Attempting login for Hdds " +
|
||||||
"Datanode user. "
|
"Datanode user. Principal: {},keytab: {}", conf.get(
|
||||||
+ "Principal: {},keytab: {}", conf.get(
|
|
||||||
DFSConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY),
|
DFSConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY),
|
||||||
conf.get(DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY));
|
conf.get(DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY));
|
||||||
|
|
||||||
|
@ -180,9 +200,18 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
|
||||||
LOG.info("Hdds Datanode login successful.");
|
LOG.info("Hdds Datanode login successful.");
|
||||||
}
|
}
|
||||||
datanodeStateMachine = new DatanodeStateMachine(datanodeDetails, conf);
|
datanodeStateMachine = new DatanodeStateMachine(datanodeDetails, conf);
|
||||||
|
try {
|
||||||
|
httpServer = new HddsDatanodeHttpServer(conf);
|
||||||
|
httpServer.start();
|
||||||
|
} catch (Exception ex) {
|
||||||
|
LOG.error("HttpServer failed to start.", ex);
|
||||||
|
}
|
||||||
startPlugins();
|
startPlugins();
|
||||||
// Starting HDDS Daemons
|
// Starting HDDS Daemons
|
||||||
datanodeStateMachine.startDaemon();
|
datanodeStateMachine.startDaemon();
|
||||||
|
if (OzoneSecurityUtil.isSecurityEnabled(conf)) {
|
||||||
|
initializeCertificateClient(conf);
|
||||||
|
}
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new RuntimeException("Can't start the HDDS datanode plugin", e);
|
throw new RuntimeException("Can't start the HDDS datanode plugin", e);
|
||||||
} catch (AuthenticationException ex) {
|
} catch (AuthenticationException ex) {
|
||||||
|
@ -192,6 +221,87 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Initializes secure Datanode.
|
||||||
|
* */
|
||||||
|
@VisibleForTesting
|
||||||
|
public void initializeCertificateClient(OzoneConfiguration config)
|
||||||
|
throws IOException {
|
||||||
|
LOG.info("Initializing secure Datanode.");
|
||||||
|
|
||||||
|
CertificateClient.InitResponse response = dnCertClient.init();
|
||||||
|
LOG.info("Init response: {}", response);
|
||||||
|
switch (response) {
|
||||||
|
case SUCCESS:
|
||||||
|
LOG.info("Initialization successful, case:{}.", response);
|
||||||
|
break;
|
||||||
|
case GETCERT:
|
||||||
|
getSCMSignedCert(config);
|
||||||
|
LOG.info("Successfully stored SCM signed certificate, case:{}.",
|
||||||
|
response);
|
||||||
|
break;
|
||||||
|
case FAILURE:
|
||||||
|
LOG.error("DN security initialization failed, case:{}.", response);
|
||||||
|
throw new RuntimeException("DN security initialization failed.");
|
||||||
|
case RECOVER:
|
||||||
|
LOG.error("DN security initialization failed, case:{}. OM certificate " +
|
||||||
|
"is missing.", response);
|
||||||
|
throw new RuntimeException("DN security initialization failed.");
|
||||||
|
default:
|
||||||
|
LOG.error("DN security initialization failed. Init response: {}",
|
||||||
|
response);
|
||||||
|
throw new RuntimeException("DN security initialization failed.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get SCM signed certificate and store it using certificate client.
|
||||||
|
* @param config
|
||||||
|
* */
|
||||||
|
private void getSCMSignedCert(OzoneConfiguration config) {
|
||||||
|
try {
|
||||||
|
PKCS10CertificationRequest csr = getCSR(config);
|
||||||
|
// TODO: For SCM CA we should fetch certificate from multiple SCMs.
|
||||||
|
SCMSecurityProtocol secureScmClient =
|
||||||
|
HddsUtils.getScmSecurityClient(config,
|
||||||
|
HddsUtils.getScmAddressForSecurityProtocol(config));
|
||||||
|
|
||||||
|
String pemEncodedCert = secureScmClient.getDataNodeCertificate(
|
||||||
|
datanodeDetails.getProtoBufMessage(), getEncodedString(csr));
|
||||||
|
|
||||||
|
X509Certificate x509Certificate =
|
||||||
|
CertificateCodec.getX509Certificate(pemEncodedCert);
|
||||||
|
dnCertClient.storeCertificate(x509Certificate);
|
||||||
|
} catch (IOException | CertificateException e) {
|
||||||
|
LOG.error("Error while storing SCM signed certificate.", e);
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates CSR for DN.
|
||||||
|
* @param config
|
||||||
|
* */
|
||||||
|
@VisibleForTesting
|
||||||
|
public PKCS10CertificationRequest getCSR(Configuration config)
|
||||||
|
throws IOException {
|
||||||
|
CertificateSignRequest.Builder builder = dnCertClient.getCSRBuilder();
|
||||||
|
KeyPair keyPair = new KeyPair(dnCertClient.getPublicKey(),
|
||||||
|
dnCertClient.getPrivateKey());
|
||||||
|
|
||||||
|
String hostname = InetAddress.getLocalHost().getCanonicalHostName();
|
||||||
|
String subject = UserGroupInformation.getCurrentUser()
|
||||||
|
.getShortUserName() + "@" + hostname;
|
||||||
|
|
||||||
|
builder.setCA(false)
|
||||||
|
.setKey(keyPair)
|
||||||
|
.setConfiguration(config)
|
||||||
|
.setSubject(subject);
|
||||||
|
|
||||||
|
LOG.info("Creating csr for DN-> subject:{}", subject);
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns DatanodeDetails or null in case of Error.
|
* Returns DatanodeDetails or null in case of Error.
|
||||||
*
|
*
|
||||||
|
@ -294,6 +404,14 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
|
||||||
if (datanodeStateMachine != null) {
|
if (datanodeStateMachine != null) {
|
||||||
datanodeStateMachine.stopDaemon();
|
datanodeStateMachine.stopDaemon();
|
||||||
}
|
}
|
||||||
|
if (httpServer != null) {
|
||||||
|
try {
|
||||||
|
httpServer.stop();
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.error("Stopping HttpServer is failed.", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -308,4 +426,18 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public String getComponent() {
|
||||||
|
return component;
|
||||||
|
}
|
||||||
|
|
||||||
|
public CertificateClient getCertificateClient() {
|
||||||
|
return dnCertClient;
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public void setCertificateClient(CertificateClient client) {
|
||||||
|
dnCertClient = client;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -35,9 +35,11 @@ import org.slf4j.LoggerFactory;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.ConcurrentNavigableMap;
|
import java.util.concurrent.ConcurrentNavigableMap;
|
||||||
import java.util.concurrent.ConcurrentSkipListMap;
|
import java.util.concurrent.ConcurrentSkipListMap;
|
||||||
|
import java.util.concurrent.ConcurrentSkipListSet;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
|
||||||
|
@ -50,7 +52,8 @@ public class ContainerSet {
|
||||||
|
|
||||||
private final ConcurrentSkipListMap<Long, Container> containerMap = new
|
private final ConcurrentSkipListMap<Long, Container> containerMap = new
|
||||||
ConcurrentSkipListMap<>();
|
ConcurrentSkipListMap<>();
|
||||||
|
private final ConcurrentSkipListSet<Long> missingContainerSet =
|
||||||
|
new ConcurrentSkipListSet<>();
|
||||||
/**
|
/**
|
||||||
* Add Container to container map.
|
* Add Container to container map.
|
||||||
* @param container
|
* @param container
|
||||||
|
@ -128,6 +131,7 @@ public class ContainerSet {
|
||||||
* @return containerMap Iterator
|
* @return containerMap Iterator
|
||||||
*/
|
*/
|
||||||
public Iterator<Map.Entry<Long, Container>> getContainerMapIterator() {
|
public Iterator<Map.Entry<Long, Container>> getContainerMapIterator() {
|
||||||
|
containerMap.keySet().stream().collect(Collectors.toSet());
|
||||||
return containerMap.entrySet().iterator();
|
return containerMap.entrySet().iterator();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -218,4 +222,20 @@ public class ContainerSet {
|
||||||
return deletionPolicy
|
return deletionPolicy
|
||||||
.chooseContainerForBlockDeletion(count, containerDataMap);
|
.chooseContainerForBlockDeletion(count, containerDataMap);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Set<Long> getMissingContainerSet() {
|
||||||
|
return missingContainerSet;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Builds the missing container set by taking a diff total no containers
|
||||||
|
* actually found and number of containers which actually got created.
|
||||||
|
* This will only be called during the initialization of Datanode Service
|
||||||
|
* when it still not a part of any write Pipeline.
|
||||||
|
* @param createdContainerSet ContainerId set persisted in the Ratis snapshot
|
||||||
|
*/
|
||||||
|
public void buildMissingContainerSet(Set<Long> createdContainerSet) {
|
||||||
|
missingContainerSet.addAll(createdContainerSet);
|
||||||
|
missingContainerSet.removeAll(containerMap.keySet());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -66,6 +66,7 @@ import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Ozone Container dispatcher takes a call from the netty server and routes it
|
* Ozone Container dispatcher takes a call from the netty server and routes it
|
||||||
|
@ -101,7 +102,6 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
|
||||||
this.containerCloseThreshold = conf.getFloat(
|
this.containerCloseThreshold = conf.getFloat(
|
||||||
HddsConfigKeys.HDDS_CONTAINER_CLOSE_THRESHOLD,
|
HddsConfigKeys.HDDS_CONTAINER_CLOSE_THRESHOLD,
|
||||||
HddsConfigKeys.HDDS_CONTAINER_CLOSE_THRESHOLD_DEFAULT);
|
HddsConfigKeys.HDDS_CONTAINER_CLOSE_THRESHOLD_DEFAULT);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -132,6 +132,12 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void buildMissingContainerSet(Set<Long> createdContainerSet) {
|
||||||
|
containerSet.buildMissingContainerSet(createdContainerSet);
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("methodlength")
|
||||||
@Override
|
@Override
|
||||||
public ContainerCommandResponseProto dispatch(
|
public ContainerCommandResponseProto dispatch(
|
||||||
ContainerCommandRequestProto msg, DispatcherContext dispatcherContext) {
|
ContainerCommandRequestProto msg, DispatcherContext dispatcherContext) {
|
||||||
|
@ -145,18 +151,61 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
|
||||||
Map<String, String> params =
|
Map<String, String> params =
|
||||||
ContainerCommandRequestPBHelper.getAuditParams(msg);
|
ContainerCommandRequestPBHelper.getAuditParams(msg);
|
||||||
|
|
||||||
Container container = null;
|
Container container;
|
||||||
ContainerType containerType = null;
|
ContainerType containerType;
|
||||||
ContainerCommandResponseProto responseProto = null;
|
ContainerCommandResponseProto responseProto = null;
|
||||||
long startTime = System.nanoTime();
|
long startTime = System.nanoTime();
|
||||||
ContainerProtos.Type cmdType = msg.getCmdType();
|
ContainerProtos.Type cmdType = msg.getCmdType();
|
||||||
long containerID = msg.getContainerID();
|
long containerID = msg.getContainerID();
|
||||||
metrics.incContainerOpsMetrics(cmdType);
|
metrics.incContainerOpsMetrics(cmdType);
|
||||||
|
container = getContainer(containerID);
|
||||||
|
boolean isWriteStage =
|
||||||
|
(cmdType == ContainerProtos.Type.WriteChunk && dispatcherContext != null
|
||||||
|
&& dispatcherContext.getStage()
|
||||||
|
== DispatcherContext.WriteChunkStage.WRITE_DATA);
|
||||||
|
boolean isWriteCommitStage =
|
||||||
|
(cmdType == ContainerProtos.Type.WriteChunk && dispatcherContext != null
|
||||||
|
&& dispatcherContext.getStage()
|
||||||
|
== DispatcherContext.WriteChunkStage.COMMIT_DATA);
|
||||||
|
|
||||||
|
// if the command gets executed other than Ratis, the default wroite stage
|
||||||
|
// is WriteChunkStage.COMBINED
|
||||||
|
boolean isCombinedStage =
|
||||||
|
cmdType == ContainerProtos.Type.WriteChunk && (dispatcherContext == null
|
||||||
|
|| dispatcherContext.getStage()
|
||||||
|
== DispatcherContext.WriteChunkStage.COMBINED);
|
||||||
|
Set<Long> containerIdSet = null;
|
||||||
|
if (dispatcherContext != null) {
|
||||||
|
containerIdSet = dispatcherContext.getCreateContainerSet();
|
||||||
|
}
|
||||||
|
if (isWriteCommitStage) {
|
||||||
|
// check if the container Id exist in the loaded snapshot file. if
|
||||||
|
// it does not , it infers that , this is a restart of dn where
|
||||||
|
// the we are reapplying the transaction which was not captured in the
|
||||||
|
// snapshot.
|
||||||
|
// just add it to the list, and remove it from missing container set
|
||||||
|
// as it might have been added in the list during "init".
|
||||||
|
Preconditions.checkNotNull(containerIdSet);
|
||||||
|
if (!containerIdSet.contains(containerID)) {
|
||||||
|
containerIdSet.add(containerID);
|
||||||
|
containerSet.getMissingContainerSet().remove(containerID);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (getMissingContainerSet().contains(containerID)) {
|
||||||
|
StorageContainerException sce = new StorageContainerException(
|
||||||
|
"ContainerID " + containerID
|
||||||
|
+ " has been lost and and cannot be recreated on this DataNode",
|
||||||
|
ContainerProtos.Result.CONTAINER_MISSING);
|
||||||
|
audit(action, eventType, params, AuditEventStatus.FAILURE, sce);
|
||||||
|
return ContainerUtils.logAndReturnError(LOG, sce, msg);
|
||||||
|
}
|
||||||
|
|
||||||
if (cmdType != ContainerProtos.Type.CreateContainer) {
|
if (cmdType != ContainerProtos.Type.CreateContainer) {
|
||||||
container = getContainer(containerID);
|
/**
|
||||||
|
* Create Container should happen only as part of Write_Data phase of
|
||||||
if (container == null && (cmdType == ContainerProtos.Type.WriteChunk
|
* writeChunk.
|
||||||
|
*/
|
||||||
|
if (container == null && ((isWriteStage || isCombinedStage)
|
||||||
|| cmdType == ContainerProtos.Type.PutSmallFile)) {
|
|| cmdType == ContainerProtos.Type.PutSmallFile)) {
|
||||||
// If container does not exist, create one for WriteChunk and
|
// If container does not exist, create one for WriteChunk and
|
||||||
// PutSmallFile request
|
// PutSmallFile request
|
||||||
|
@ -168,7 +217,12 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
|
||||||
audit(action, eventType, params, AuditEventStatus.FAILURE, sce);
|
audit(action, eventType, params, AuditEventStatus.FAILURE, sce);
|
||||||
return ContainerUtils.logAndReturnError(LOG, sce, msg);
|
return ContainerUtils.logAndReturnError(LOG, sce, msg);
|
||||||
}
|
}
|
||||||
|
Preconditions.checkArgument(isWriteStage && containerIdSet != null
|
||||||
|
|| dispatcherContext == null);
|
||||||
|
if (containerIdSet != null) {
|
||||||
|
// adds this container to list of containers created in the pipeline
|
||||||
|
containerIdSet.add(containerID);
|
||||||
|
}
|
||||||
container = getContainer(containerID);
|
container = getContainer(containerID);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -406,6 +460,11 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
|
||||||
return containerSet.getContainer(containerID);
|
return containerSet.getContainer(containerID);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public Set<Long> getMissingContainerSet() {
|
||||||
|
return containerSet.getMissingContainerSet();
|
||||||
|
}
|
||||||
|
|
||||||
private ContainerType getContainerType(Container container) {
|
private ContainerType getContainerType(Container container) {
|
||||||
return container.getContainerType();
|
return container.getContainerType();
|
||||||
}
|
}
|
||||||
|
|
|
@ -26,6 +26,8 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
|
||||||
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
|
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
|
||||||
|
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Dispatcher acts as the bridge between the transport layer and
|
* Dispatcher acts as the bridge between the transport layer and
|
||||||
* the actual container layer. This layer is capable of transforming
|
* the actual container layer. This layer is capable of transforming
|
||||||
|
@ -58,6 +60,12 @@ public interface ContainerDispatcher {
|
||||||
*/
|
*/
|
||||||
void init();
|
void init();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* finds and builds the missing containers in case of a lost disk etc
|
||||||
|
* in the ContainerSet.
|
||||||
|
*/
|
||||||
|
void buildMissingContainerSet(Set<Long> createdContainers);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Shutdown Dispatcher services.
|
* Shutdown Dispatcher services.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -21,10 +21,13 @@ package org.apache.hadoop.ozone.container.common.transport.server.ratis;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.cache.Cache;
|
import com.google.common.cache.Cache;
|
||||||
import com.google.common.cache.CacheBuilder;
|
import com.google.common.cache.CacheBuilder;
|
||||||
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.hadoop.hdds.HddsUtils;
|
import org.apache.hadoop.hdds.HddsUtils;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
||||||
|
|
||||||
import io.opentracing.Scope;
|
import io.opentracing.Scope;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
|
||||||
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
||||||
import org.apache.ratis.proto.RaftProtos.RaftPeerRole;
|
import org.apache.ratis.proto.RaftProtos.RaftPeerRole;
|
||||||
import org.apache.ratis.protocol.RaftGroup;
|
import org.apache.ratis.protocol.RaftGroup;
|
||||||
import org.apache.ratis.protocol.RaftGroupId;
|
import org.apache.ratis.protocol.RaftGroupId;
|
||||||
|
@ -36,6 +39,8 @@ import org.apache.ratis.statemachine.impl.SingleFileSnapshotInfo;
|
||||||
import org.apache.ratis.thirdparty.com.google.protobuf
|
import org.apache.ratis.thirdparty.com.google.protobuf
|
||||||
.InvalidProtocolBufferException;
|
.InvalidProtocolBufferException;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type;
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type;
|
||||||
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
|
||||||
|
ContainerIdSetProto;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||||
.ContainerCommandRequestProto;
|
.ContainerCommandRequestProto;
|
||||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||||
|
@ -73,10 +78,15 @@ import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.ThreadPoolExecutor;
|
import java.util.concurrent.ThreadPoolExecutor;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.ConcurrentSkipListSet;
|
||||||
|
import java.io.FileOutputStream;
|
||||||
|
import java.io.FileInputStream;
|
||||||
|
import java.io.OutputStream;
|
||||||
|
|
||||||
|
|
||||||
/** A {@link org.apache.ratis.statemachine.StateMachine} for containers.
|
/** A {@link org.apache.ratis.statemachine.StateMachine} for containers.
|
||||||
*
|
*
|
||||||
|
@ -126,6 +136,9 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
private final XceiverServerRatis ratisServer;
|
private final XceiverServerRatis ratisServer;
|
||||||
private final ConcurrentHashMap<Long, CompletableFuture<Message>>
|
private final ConcurrentHashMap<Long, CompletableFuture<Message>>
|
||||||
writeChunkFutureMap;
|
writeChunkFutureMap;
|
||||||
|
|
||||||
|
// keeps track of the containers created per pipeline
|
||||||
|
private final Set<Long> createContainerSet;
|
||||||
private ExecutorService[] executors;
|
private ExecutorService[] executors;
|
||||||
private final int numExecutors;
|
private final int numExecutors;
|
||||||
private final Map<Long, Long> applyTransactionCompletionMap;
|
private final Map<Long, Long> applyTransactionCompletionMap;
|
||||||
|
@ -160,6 +173,7 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
.maximumSize(chunkExecutor.getCorePoolSize()).build();
|
.maximumSize(chunkExecutor.getCorePoolSize()).build();
|
||||||
this.isBlockTokenEnabled = isBlockTokenEnabled;
|
this.isBlockTokenEnabled = isBlockTokenEnabled;
|
||||||
this.tokenVerifier = tokenVerifier;
|
this.tokenVerifier = tokenVerifier;
|
||||||
|
this.createContainerSet = new ConcurrentSkipListSet<>();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -181,26 +195,56 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
loadSnapshot(storage.getLatestSnapshot());
|
loadSnapshot(storage.getLatestSnapshot());
|
||||||
}
|
}
|
||||||
|
|
||||||
private long loadSnapshot(SingleFileSnapshotInfo snapshot) {
|
private long loadSnapshot(SingleFileSnapshotInfo snapshot)
|
||||||
|
throws IOException {
|
||||||
if (snapshot == null) {
|
if (snapshot == null) {
|
||||||
TermIndex empty = TermIndex.newTermIndex(0,
|
TermIndex empty =
|
||||||
RaftServerConstants.INVALID_LOG_INDEX);
|
TermIndex.newTermIndex(0, RaftServerConstants.INVALID_LOG_INDEX);
|
||||||
LOG.info("The snapshot info is null." +
|
LOG.info(
|
||||||
"Setting the last applied index to:" + empty);
|
"The snapshot info is null." + "Setting the last applied index to:"
|
||||||
|
+ empty);
|
||||||
setLastAppliedTermIndex(empty);
|
setLastAppliedTermIndex(empty);
|
||||||
lastIndex = RaftServerConstants.INVALID_LOG_INDEX;
|
lastIndex = RaftServerConstants.INVALID_LOG_INDEX;
|
||||||
return RaftServerConstants.INVALID_LOG_INDEX;
|
return RaftServerConstants.INVALID_LOG_INDEX;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
final File snapshotFile = snapshot.getFile().getPath().toFile();
|
||||||
final TermIndex last =
|
final TermIndex last =
|
||||||
SimpleStateMachineStorage.getTermIndexFromSnapshotFile(
|
SimpleStateMachineStorage.getTermIndexFromSnapshotFile(snapshotFile);
|
||||||
snapshot.getFile().getPath().toFile());
|
|
||||||
LOG.info("Setting the last applied index to " + last);
|
LOG.info("Setting the last applied index to " + last);
|
||||||
setLastAppliedTermIndex(last);
|
setLastAppliedTermIndex(last);
|
||||||
lastIndex = last.getIndex();
|
lastIndex = last.getIndex();
|
||||||
|
|
||||||
|
// initialize the dispatcher with snapshot so that it build the missing
|
||||||
|
// container list
|
||||||
|
try (FileInputStream fin = new FileInputStream(snapshotFile)) {
|
||||||
|
byte[] containerIds = IOUtils.toByteArray(fin);
|
||||||
|
ContainerProtos.ContainerIdSetProto proto =
|
||||||
|
ContainerProtos.ContainerIdSetProto.parseFrom(containerIds);
|
||||||
|
// read the created containers list from the snapshot file and add it to
|
||||||
|
// the createContainerSet here.
|
||||||
|
// createContainerSet will further grow as and when containers get created
|
||||||
|
createContainerSet.addAll(proto.getContainerIdList());
|
||||||
|
dispatcher.buildMissingContainerSet(createContainerSet);
|
||||||
|
}
|
||||||
return last.getIndex();
|
return last.getIndex();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* As a part of taking snapshot with Ratis StateMachine, it will persist
|
||||||
|
* the existing container set in the snapshotFile.
|
||||||
|
* @param out OutputStream mapped to the Ratis snapshot file
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void persistContainerSet(OutputStream out) throws IOException {
|
||||||
|
ContainerIdSetProto.Builder builder = ContainerIdSetProto.newBuilder();
|
||||||
|
builder.addAllContainerId(createContainerSet);
|
||||||
|
// TODO : while snapshot is being taken, deleteContainer call should not
|
||||||
|
// should not happen. Lock protection will be required if delete
|
||||||
|
// container happens outside of Ratis.
|
||||||
|
IOUtils.write(builder.build().toByteArray(), out);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long takeSnapshot() throws IOException {
|
public long takeSnapshot() throws IOException {
|
||||||
TermIndex ti = getLastAppliedTermIndex();
|
TermIndex ti = getLastAppliedTermIndex();
|
||||||
|
@ -211,8 +255,13 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
LOG.info("Taking a snapshot to file {}", snapshotFile);
|
LOG.info("Taking a snapshot to file {}", snapshotFile);
|
||||||
try {
|
try {
|
||||||
//TODO: For now, just create the file to save the term index,
|
//TODO: For now, just create the file to save the term index,
|
||||||
//persist open container info to snapshot later.
|
boolean created = snapshotFile.createNewFile();
|
||||||
snapshotFile.createNewFile();
|
if (!created) {
|
||||||
|
throw new IOException("Failed to create ratis snapshot file");
|
||||||
|
}
|
||||||
|
try (FileOutputStream fos = new FileOutputStream(snapshotFile)) {
|
||||||
|
persistContainerSet(fos);
|
||||||
|
}
|
||||||
} catch(IOException ioe) {
|
} catch(IOException ioe) {
|
||||||
LOG.warn("Failed to write snapshot file \"" + snapshotFile
|
LOG.warn("Failed to write snapshot file \"" + snapshotFile
|
||||||
+ "\", last applied index=" + ti);
|
+ "\", last applied index=" + ti);
|
||||||
|
@ -227,7 +276,7 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
public TransactionContext startTransaction(RaftClientRequest request)
|
public TransactionContext startTransaction(RaftClientRequest request)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
final ContainerCommandRequestProto proto =
|
final ContainerCommandRequestProto proto =
|
||||||
getRequestProto(request.getMessage().getContent());
|
getContainerCommandRequestProto(request.getMessage().getContent());
|
||||||
Preconditions.checkArgument(request.getRaftGroupId().equals(gid));
|
Preconditions.checkArgument(request.getRaftGroupId().equals(gid));
|
||||||
try (Scope scope = TracingUtil
|
try (Scope scope = TracingUtil
|
||||||
.importAndCreateScope(proto.getCmdType().name(), proto.getTraceID())) {
|
.importAndCreateScope(proto.getCmdType().name(), proto.getTraceID())) {
|
||||||
|
@ -244,17 +293,6 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
}
|
}
|
||||||
if (proto.getCmdType() == Type.WriteChunk) {
|
if (proto.getCmdType() == Type.WriteChunk) {
|
||||||
final WriteChunkRequestProto write = proto.getWriteChunk();
|
final WriteChunkRequestProto write = proto.getWriteChunk();
|
||||||
// create the state machine data proto
|
|
||||||
final WriteChunkRequestProto dataWriteChunkProto =
|
|
||||||
WriteChunkRequestProto
|
|
||||||
.newBuilder(write)
|
|
||||||
.build();
|
|
||||||
ContainerCommandRequestProto dataContainerCommandProto =
|
|
||||||
ContainerCommandRequestProto
|
|
||||||
.newBuilder(proto)
|
|
||||||
.setWriteChunk(dataWriteChunkProto)
|
|
||||||
.build();
|
|
||||||
|
|
||||||
// create the log entry proto
|
// create the log entry proto
|
||||||
final WriteChunkRequestProto commitWriteChunkProto =
|
final WriteChunkRequestProto commitWriteChunkProto =
|
||||||
WriteChunkRequestProto.newBuilder()
|
WriteChunkRequestProto.newBuilder()
|
||||||
|
@ -273,7 +311,7 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
.setClientRequest(request)
|
.setClientRequest(request)
|
||||||
.setStateMachine(this)
|
.setStateMachine(this)
|
||||||
.setServerRole(RaftPeerRole.LEADER)
|
.setServerRole(RaftPeerRole.LEADER)
|
||||||
.setStateMachineData(dataContainerCommandProto.toByteString())
|
.setStateMachineData(write.getData())
|
||||||
.setLogData(commitContainerCommandProto.toByteString())
|
.setLogData(commitContainerCommandProto.toByteString())
|
||||||
.build();
|
.build();
|
||||||
} else {
|
} else {
|
||||||
|
@ -291,8 +329,8 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
return entryProto.getStateMachineEntry().getStateMachineData();
|
return entryProto.getStateMachineEntry().getStateMachineData();
|
||||||
}
|
}
|
||||||
|
|
||||||
private ContainerCommandRequestProto getRequestProto(ByteString request)
|
private ContainerCommandRequestProto getContainerCommandRequestProto(
|
||||||
throws InvalidProtocolBufferException {
|
ByteString request) throws InvalidProtocolBufferException {
|
||||||
// TODO: We can avoid creating new builder and set pipeline Id if
|
// TODO: We can avoid creating new builder and set pipeline Id if
|
||||||
// the client is already sending the pipeline id, then we just have to
|
// the client is already sending the pipeline id, then we just have to
|
||||||
// validate the pipeline Id.
|
// validate the pipeline Id.
|
||||||
|
@ -302,13 +340,22 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
}
|
}
|
||||||
|
|
||||||
private ContainerCommandResponseProto dispatchCommand(
|
private ContainerCommandResponseProto dispatchCommand(
|
||||||
ContainerCommandRequestProto requestProto,
|
ContainerCommandRequestProto requestProto, DispatcherContext context) {
|
||||||
DispatcherContext context) throws IOException {
|
LOG.trace("dispatch {} containerID={} pipelineID={} traceID={}",
|
||||||
LOG.trace("dispatch {}", requestProto);
|
requestProto.getCmdType(), requestProto.getContainerID(),
|
||||||
if(isBlockTokenEnabled) {
|
requestProto.getPipelineID(), requestProto.getTraceID());
|
||||||
// ServerInterceptors intercepts incoming request and creates ugi.
|
if (isBlockTokenEnabled) {
|
||||||
tokenVerifier.verify(UserGroupInformation.getCurrentUser()
|
try {
|
||||||
.getShortUserName(), requestProto.getEncodedToken());
|
// ServerInterceptors intercepts incoming request and creates ugi.
|
||||||
|
tokenVerifier
|
||||||
|
.verify(UserGroupInformation.getCurrentUser().getShortUserName(),
|
||||||
|
requestProto.getEncodedToken());
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
StorageContainerException sce = new StorageContainerException(
|
||||||
|
"Block token verification failed. " + ioe.getMessage(), ioe,
|
||||||
|
ContainerProtos.Result.BLOCK_TOKEN_VERIFICATION_FAILED);
|
||||||
|
return ContainerUtils.logAndReturnError(LOG, sce, requestProto);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
ContainerCommandResponseProto response =
|
ContainerCommandResponseProto response =
|
||||||
dispatcher.dispatch(requestProto, context);
|
dispatcher.dispatch(requestProto, context);
|
||||||
|
@ -317,7 +364,7 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
}
|
}
|
||||||
|
|
||||||
private Message runCommand(ContainerCommandRequestProto requestProto,
|
private Message runCommand(ContainerCommandRequestProto requestProto,
|
||||||
DispatcherContext context) throws IOException {
|
DispatcherContext context) {
|
||||||
return dispatchCommand(requestProto, context)::toByteString;
|
return dispatchCommand(requestProto, context)::toByteString;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -344,15 +391,12 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
.setTerm(term)
|
.setTerm(term)
|
||||||
.setLogIndex(entryIndex)
|
.setLogIndex(entryIndex)
|
||||||
.setStage(DispatcherContext.WriteChunkStage.WRITE_DATA)
|
.setStage(DispatcherContext.WriteChunkStage.WRITE_DATA)
|
||||||
|
.setCreateContainerSet(createContainerSet)
|
||||||
.build();
|
.build();
|
||||||
CompletableFuture<Message> writeChunkFuture;
|
// ensure the write chunk happens asynchronously in writeChunkExecutor pool
|
||||||
try {
|
// thread.
|
||||||
Message msg = runCommand(requestProto, context);
|
CompletableFuture<Message> writeChunkFuture = CompletableFuture
|
||||||
writeChunkFuture = CompletableFuture
|
.supplyAsync(() -> runCommand(requestProto, context), chunkExecutor);
|
||||||
.supplyAsync(() -> msg, chunkExecutor);
|
|
||||||
}catch(IOException ie) {
|
|
||||||
writeChunkFuture = completeExceptionally(ie);
|
|
||||||
}
|
|
||||||
|
|
||||||
writeChunkFutureMap.put(entryIndex, writeChunkFuture);
|
writeChunkFutureMap.put(entryIndex, writeChunkFuture);
|
||||||
LOG.debug("writeChunk writeStateMachineData : blockId " + write.getBlockID()
|
LOG.debug("writeChunk writeStateMachineData : blockId " + write.getBlockID()
|
||||||
|
@ -378,8 +422,15 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
public CompletableFuture<Message> writeStateMachineData(LogEntryProto entry) {
|
public CompletableFuture<Message> writeStateMachineData(LogEntryProto entry) {
|
||||||
try {
|
try {
|
||||||
metrics.incNumWriteStateMachineOps();
|
metrics.incNumWriteStateMachineOps();
|
||||||
final ContainerCommandRequestProto requestProto =
|
ContainerCommandRequestProto requestProto =
|
||||||
getRequestProto(getStateMachineData(entry.getStateMachineLogEntry()));
|
getContainerCommandRequestProto(
|
||||||
|
entry.getStateMachineLogEntry().getLogData());
|
||||||
|
WriteChunkRequestProto writeChunk =
|
||||||
|
WriteChunkRequestProto.newBuilder(requestProto.getWriteChunk())
|
||||||
|
.setData(getStateMachineData(entry.getStateMachineLogEntry()))
|
||||||
|
.build();
|
||||||
|
requestProto = ContainerCommandRequestProto.newBuilder(requestProto)
|
||||||
|
.setWriteChunk(writeChunk).build();
|
||||||
Type cmdType = requestProto.getCmdType();
|
Type cmdType = requestProto.getCmdType();
|
||||||
|
|
||||||
// For only writeChunk, there will be writeStateMachineData call.
|
// For only writeChunk, there will be writeStateMachineData call.
|
||||||
|
@ -403,7 +454,7 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
try {
|
try {
|
||||||
metrics.incNumReadStateMachineOps();
|
metrics.incNumReadStateMachineOps();
|
||||||
final ContainerCommandRequestProto requestProto =
|
final ContainerCommandRequestProto requestProto =
|
||||||
getRequestProto(request.getContent());
|
getContainerCommandRequestProto(request.getContent());
|
||||||
return CompletableFuture.completedFuture(runCommand(requestProto, null));
|
return CompletableFuture.completedFuture(runCommand(requestProto, null));
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
metrics.incNumReadStateMachineFails();
|
metrics.incNumReadStateMachineFails();
|
||||||
|
@ -453,34 +504,8 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
*/
|
*/
|
||||||
private ByteString getCachedStateMachineData(Long logIndex, long term,
|
private ByteString getCachedStateMachineData(Long logIndex, long term,
|
||||||
ContainerCommandRequestProto requestProto) throws ExecutionException {
|
ContainerCommandRequestProto requestProto) throws ExecutionException {
|
||||||
try {
|
return stateMachineDataCache.get(logIndex,
|
||||||
return reconstructWriteChunkRequest(
|
() -> readStateMachineData(requestProto, term, logIndex));
|
||||||
stateMachineDataCache.get(logIndex, new Callable<ByteString>() {
|
|
||||||
@Override
|
|
||||||
public ByteString call() throws Exception {
|
|
||||||
return readStateMachineData(requestProto, term, logIndex);
|
|
||||||
}
|
|
||||||
}), requestProto);
|
|
||||||
} catch (ExecutionException e) {
|
|
||||||
throw e;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private ByteString reconstructWriteChunkRequest(ByteString data,
|
|
||||||
ContainerCommandRequestProto requestProto) {
|
|
||||||
WriteChunkRequestProto writeChunkRequestProto =
|
|
||||||
requestProto.getWriteChunk();
|
|
||||||
// reconstruct the write chunk request
|
|
||||||
final WriteChunkRequestProto.Builder dataWriteChunkProto =
|
|
||||||
WriteChunkRequestProto.newBuilder(writeChunkRequestProto)
|
|
||||||
// adding the state machine data
|
|
||||||
.setData(data);
|
|
||||||
|
|
||||||
ContainerCommandRequestProto.Builder newStateMachineProto =
|
|
||||||
ContainerCommandRequestProto.newBuilder(requestProto)
|
|
||||||
.setWriteChunk(dataWriteChunkProto);
|
|
||||||
|
|
||||||
return newStateMachineProto.build().toByteString();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -514,20 +539,23 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
final ContainerCommandRequestProto requestProto =
|
final ContainerCommandRequestProto requestProto =
|
||||||
getRequestProto(entry.getStateMachineLogEntry().getLogData());
|
getContainerCommandRequestProto(
|
||||||
|
entry.getStateMachineLogEntry().getLogData());
|
||||||
// readStateMachineData should only be called for "write" to Ratis.
|
// readStateMachineData should only be called for "write" to Ratis.
|
||||||
Preconditions.checkArgument(!HddsUtils.isReadOnly(requestProto));
|
Preconditions.checkArgument(!HddsUtils.isReadOnly(requestProto));
|
||||||
if (requestProto.getCmdType() == Type.WriteChunk) {
|
if (requestProto.getCmdType() == Type.WriteChunk) {
|
||||||
CompletableFuture<ByteString> future = new CompletableFuture<>();
|
final CompletableFuture<ByteString> future = new CompletableFuture<>();
|
||||||
return future.supplyAsync(() -> {
|
CompletableFuture.supplyAsync(() -> {
|
||||||
try {
|
try {
|
||||||
return getCachedStateMachineData(entry.getIndex(), entry.getTerm(),
|
future.complete(
|
||||||
requestProto);
|
getCachedStateMachineData(entry.getIndex(), entry.getTerm(),
|
||||||
|
requestProto));
|
||||||
} catch (ExecutionException e) {
|
} catch (ExecutionException e) {
|
||||||
future.completeExceptionally(e);
|
future.completeExceptionally(e);
|
||||||
return null;
|
|
||||||
}
|
}
|
||||||
|
return future;
|
||||||
}, chunkExecutor);
|
}, chunkExecutor);
|
||||||
|
return future;
|
||||||
} else {
|
} else {
|
||||||
throw new IllegalStateException("Cmd type:" + requestProto.getCmdType()
|
throw new IllegalStateException("Cmd type:" + requestProto.getCmdType()
|
||||||
+ " cannot have state machine data");
|
+ " cannot have state machine data");
|
||||||
|
@ -576,9 +604,9 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
try {
|
try {
|
||||||
metrics.incNumApplyTransactionsOps();
|
metrics.incNumApplyTransactionsOps();
|
||||||
ContainerCommandRequestProto requestProto =
|
ContainerCommandRequestProto requestProto =
|
||||||
getRequestProto(trx.getStateMachineLogEntry().getLogData());
|
getContainerCommandRequestProto(
|
||||||
|
trx.getStateMachineLogEntry().getLogData());
|
||||||
Type cmdType = requestProto.getCmdType();
|
Type cmdType = requestProto.getCmdType();
|
||||||
CompletableFuture<Message> future;
|
|
||||||
// Make sure that in write chunk, the user data is not set
|
// Make sure that in write chunk, the user data is not set
|
||||||
if (cmdType == Type.WriteChunk) {
|
if (cmdType == Type.WriteChunk) {
|
||||||
Preconditions
|
Preconditions
|
||||||
|
@ -586,13 +614,14 @@ public class ContainerStateMachine extends BaseStateMachine {
|
||||||
builder
|
builder
|
||||||
.setStage(DispatcherContext.WriteChunkStage.COMMIT_DATA);
|
.setStage(DispatcherContext.WriteChunkStage.COMMIT_DATA);
|
||||||
}
|
}
|
||||||
try {
|
if (cmdType == Type.WriteChunk || cmdType ==Type.PutSmallFile) {
|
||||||
Message msg = runCommand(requestProto, builder.build());
|
builder.setCreateContainerSet(createContainerSet);
|
||||||
future = CompletableFuture.supplyAsync(() -> msg,
|
|
||||||
getCommandExecutor(requestProto));
|
|
||||||
} catch (IOException ie) {
|
|
||||||
future = completeExceptionally(ie);
|
|
||||||
}
|
}
|
||||||
|
// Ensure the command gets executed in a separate thread than
|
||||||
|
// stateMachineUpdater thread which is calling applyTransaction here.
|
||||||
|
CompletableFuture<Message> future = CompletableFuture
|
||||||
|
.supplyAsync(() -> runCommand(requestProto, builder.build()),
|
||||||
|
getCommandExecutor(requestProto));
|
||||||
|
|
||||||
lastIndex = index;
|
lastIndex = index;
|
||||||
future.thenAccept(m -> {
|
future.thenAccept(m -> {
|
||||||
|
|
|
@ -20,6 +20,8 @@ package org.apache.hadoop.ozone.container.common.transport.server.ratis;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
|
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* DispatcherContext class holds transport protocol specific context info
|
* DispatcherContext class holds transport protocol specific context info
|
||||||
* required for execution of container commands over the container dispatcher.
|
* required for execution of container commands over the container dispatcher.
|
||||||
|
@ -43,12 +45,15 @@ public final class DispatcherContext {
|
||||||
// the log index in Ratis log to which the request belongs to
|
// the log index in Ratis log to which the request belongs to
|
||||||
private final long logIndex;
|
private final long logIndex;
|
||||||
|
|
||||||
|
private final Set<Long> createContainerSet;
|
||||||
|
|
||||||
private DispatcherContext(long term, long index, WriteChunkStage stage,
|
private DispatcherContext(long term, long index, WriteChunkStage stage,
|
||||||
boolean readFromTmpFile) {
|
boolean readFromTmpFile, Set<Long> containerSet) {
|
||||||
this.term = term;
|
this.term = term;
|
||||||
this.logIndex = index;
|
this.logIndex = index;
|
||||||
this.stage = stage;
|
this.stage = stage;
|
||||||
this.readFromTmpFile = readFromTmpFile;
|
this.readFromTmpFile = readFromTmpFile;
|
||||||
|
this.createContainerSet = containerSet;
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getLogIndex() {
|
public long getLogIndex() {
|
||||||
|
@ -67,6 +72,10 @@ public final class DispatcherContext {
|
||||||
return stage;
|
return stage;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Set<Long> getCreateContainerSet() {
|
||||||
|
return createContainerSet;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Builder class for building DispatcherContext.
|
* Builder class for building DispatcherContext.
|
||||||
*/
|
*/
|
||||||
|
@ -75,11 +84,12 @@ public final class DispatcherContext {
|
||||||
private boolean readFromTmpFile = false;
|
private boolean readFromTmpFile = false;
|
||||||
private long term;
|
private long term;
|
||||||
private long logIndex;
|
private long logIndex;
|
||||||
|
private Set<Long> createContainerSet;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Sets the WriteChunkStage.
|
* Sets the WriteChunkStage.
|
||||||
*
|
*
|
||||||
* @param stage WriteChunk Stage
|
* @param writeChunkStage WriteChunk Stage
|
||||||
* @return DispatcherContext.Builder
|
* @return DispatcherContext.Builder
|
||||||
*/
|
*/
|
||||||
public Builder setStage(WriteChunkStage writeChunkStage) {
|
public Builder setStage(WriteChunkStage writeChunkStage) {
|
||||||
|
@ -90,7 +100,7 @@ public final class DispatcherContext {
|
||||||
/**
|
/**
|
||||||
* Sets the flag for reading from tmp chunk files.
|
* Sets the flag for reading from tmp chunk files.
|
||||||
*
|
*
|
||||||
* @param readFromTmpFile whether to read from tmp chunk file or not
|
* @param setReadFromTmpFile whether to read from tmp chunk file or not
|
||||||
* @return DispatcherContext.Builder
|
* @return DispatcherContext.Builder
|
||||||
*/
|
*/
|
||||||
public Builder setReadFromTmpFile(boolean setReadFromTmpFile) {
|
public Builder setReadFromTmpFile(boolean setReadFromTmpFile) {
|
||||||
|
@ -101,7 +111,7 @@ public final class DispatcherContext {
|
||||||
/**
|
/**
|
||||||
* Sets the current term for the container request from Ratis.
|
* Sets the current term for the container request from Ratis.
|
||||||
*
|
*
|
||||||
* @param term current term
|
* @param currentTerm current term
|
||||||
* @return DispatcherContext.Builder
|
* @return DispatcherContext.Builder
|
||||||
*/
|
*/
|
||||||
public Builder setTerm(long currentTerm) {
|
public Builder setTerm(long currentTerm) {
|
||||||
|
@ -112,7 +122,7 @@ public final class DispatcherContext {
|
||||||
/**
|
/**
|
||||||
* Sets the logIndex for the container request from Ratis.
|
* Sets the logIndex for the container request from Ratis.
|
||||||
*
|
*
|
||||||
* @param logIndex log index
|
* @param index log index
|
||||||
* @return DispatcherContext.Builder
|
* @return DispatcherContext.Builder
|
||||||
*/
|
*/
|
||||||
public Builder setLogIndex(long index) {
|
public Builder setLogIndex(long index) {
|
||||||
|
@ -120,13 +130,24 @@ public final class DispatcherContext {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets the createContainerSet to contain all the containerIds per
|
||||||
|
* RaftGroup.
|
||||||
|
* @param set createContainerSet
|
||||||
|
* @return Builder
|
||||||
|
*/
|
||||||
|
public Builder setCreateContainerSet(Set<Long> set) {
|
||||||
|
this.createContainerSet = set;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
/**
|
/**
|
||||||
* Builds and returns DispatcherContext instance.
|
* Builds and returns DispatcherContext instance.
|
||||||
*
|
*
|
||||||
* @return DispatcherContext
|
* @return DispatcherContext
|
||||||
*/
|
*/
|
||||||
public DispatcherContext build() {
|
public DispatcherContext build() {
|
||||||
return new DispatcherContext(term, logIndex, stage, readFromTmpFile);
|
return new DispatcherContext(term, logIndex, stage, readFromTmpFile,
|
||||||
|
createContainerSet);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,17 @@
|
||||||
|
/**
|
||||||
|
* 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.
|
||||||
|
*/
|
|
@ -0,0 +1,269 @@
|
||||||
|
/**
|
||||||
|
* 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 org.apache.commons.io.FileUtils;
|
||||||
|
import org.apache.hadoop.fs.FileUtil;
|
||||||
|
import org.apache.hadoop.hdds.HddsConfigKeys;
|
||||||
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
|
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
||||||
|
import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
|
||||||
|
import org.apache.hadoop.hdds.security.x509.certificate.client.DNCertificateClient;
|
||||||
|
import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
|
||||||
|
import org.apache.hadoop.hdds.security.x509.keys.KeyCodec;
|
||||||
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
|
import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
|
||||||
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
|
import org.apache.hadoop.test.LambdaTestUtils;
|
||||||
|
import org.apache.hadoop.util.ServicePlugin;
|
||||||
|
import org.bouncycastle.cert.X509CertificateHolder;
|
||||||
|
import org.bouncycastle.pkcs.PKCS10CertificationRequest;
|
||||||
|
import org.junit.AfterClass;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.BeforeClass;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.io.File;
|
||||||
|
import java.nio.file.Paths;
|
||||||
|
import java.security.KeyPair;
|
||||||
|
import java.security.PrivateKey;
|
||||||
|
import java.security.PublicKey;
|
||||||
|
import java.security.cert.X509Certificate;
|
||||||
|
import java.util.concurrent.Callable;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.ozone.HddsDatanodeService.getLogger;
|
||||||
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SECURITY_ENABLED_KEY;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test class for {@link HddsDatanodeService}.
|
||||||
|
*/
|
||||||
|
public class TestHddsSecureDatanodeInit {
|
||||||
|
|
||||||
|
private static File testDir;
|
||||||
|
private static OzoneConfiguration conf;
|
||||||
|
private static HddsDatanodeService service;
|
||||||
|
private static String[] args = new String[]{};
|
||||||
|
private static PrivateKey privateKey;
|
||||||
|
private static PublicKey publicKey;
|
||||||
|
private static GenericTestUtils.LogCapturer dnLogs;
|
||||||
|
private static CertificateClient client;
|
||||||
|
private static SecurityConfig securityConfig;
|
||||||
|
private static KeyCodec keyCodec;
|
||||||
|
private static CertificateCodec certCodec;
|
||||||
|
private static X509CertificateHolder certHolder;
|
||||||
|
|
||||||
|
@BeforeClass
|
||||||
|
public static void setUp() throws Exception {
|
||||||
|
testDir = GenericTestUtils.getRandomizedTestDir();
|
||||||
|
conf = new OzoneConfiguration();
|
||||||
|
conf.setBoolean(OzoneConfigKeys.OZONE_ENABLED, true);
|
||||||
|
conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, testDir.getPath());
|
||||||
|
String volumeDir = testDir + "/disk1";
|
||||||
|
conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, volumeDir);
|
||||||
|
|
||||||
|
conf.setBoolean(OZONE_SECURITY_ENABLED_KEY, true);
|
||||||
|
conf.setClass(OzoneConfigKeys.HDDS_DATANODE_PLUGINS_KEY,
|
||||||
|
TestHddsDatanodeService.MockService.class,
|
||||||
|
ServicePlugin.class);
|
||||||
|
securityConfig = new SecurityConfig(conf);
|
||||||
|
|
||||||
|
service = HddsDatanodeService.createHddsDatanodeService(args, conf);
|
||||||
|
dnLogs = GenericTestUtils.LogCapturer.captureLogs(getLogger());
|
||||||
|
callQuietly(() -> {
|
||||||
|
service.start(null);
|
||||||
|
return null;
|
||||||
|
});
|
||||||
|
callQuietly(() -> {
|
||||||
|
service.initializeCertificateClient(conf);
|
||||||
|
return null;
|
||||||
|
});
|
||||||
|
certCodec = new CertificateCodec(securityConfig);
|
||||||
|
keyCodec = new KeyCodec(securityConfig);
|
||||||
|
dnLogs.clearOutput();
|
||||||
|
privateKey = service.getCertificateClient().getPrivateKey();
|
||||||
|
publicKey = service.getCertificateClient().getPublicKey();
|
||||||
|
X509Certificate x509Certificate = null;
|
||||||
|
|
||||||
|
x509Certificate = KeyStoreTestUtil.generateCertificate(
|
||||||
|
"CN=Test", new KeyPair(publicKey, privateKey), 10,
|
||||||
|
securityConfig.getSignatureAlgo());
|
||||||
|
certHolder = new X509CertificateHolder(x509Certificate.getEncoded());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterClass
|
||||||
|
public static void tearDown() {
|
||||||
|
FileUtil.fullyDelete(testDir);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUpDNCertClient(){
|
||||||
|
client = new DNCertificateClient(securityConfig);
|
||||||
|
service.setCertificateClient(client);
|
||||||
|
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||||
|
.toString(), securityConfig.getPrivateKeyFileName()).toFile());
|
||||||
|
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||||
|
.toString(), securityConfig.getPublicKeyFileName()).toFile());
|
||||||
|
FileUtils.deleteQuietly(Paths.get(securityConfig
|
||||||
|
.getCertificateLocation().toString(),
|
||||||
|
securityConfig.getCertificateFileName()).toFile());
|
||||||
|
dnLogs.clearOutput();
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSecureDnStartupCase0() throws Exception {
|
||||||
|
|
||||||
|
// Case 0: When keypair as well as certificate is missing. Initial keypair
|
||||||
|
// boot-up. Get certificate will fail as no SCM is not running.
|
||||||
|
LambdaTestUtils.intercept(Exception.class, "",
|
||||||
|
() -> service.initializeCertificateClient(conf));
|
||||||
|
|
||||||
|
Assert.assertNotNull(client.getPrivateKey());
|
||||||
|
Assert.assertNotNull(client.getPublicKey());
|
||||||
|
Assert.assertNull(client.getCertificate());
|
||||||
|
Assert.assertTrue(dnLogs.getOutput().contains("Init response: GETCERT"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSecureDnStartupCase1() throws Exception {
|
||||||
|
// Case 1: When only certificate is present.
|
||||||
|
|
||||||
|
certCodec.writeCertificate(certHolder);
|
||||||
|
LambdaTestUtils.intercept(RuntimeException.class, "DN security" +
|
||||||
|
" initialization failed",
|
||||||
|
() -> service.initializeCertificateClient(conf));
|
||||||
|
Assert.assertNull(client.getPrivateKey());
|
||||||
|
Assert.assertNull(client.getPublicKey());
|
||||||
|
Assert.assertNotNull(client.getCertificate());
|
||||||
|
Assert.assertTrue(dnLogs.getOutput().contains("Init response: FAILURE"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSecureDnStartupCase2() throws Exception {
|
||||||
|
// Case 2: When private key and certificate is missing.
|
||||||
|
keyCodec.writePublicKey(publicKey);
|
||||||
|
LambdaTestUtils.intercept(RuntimeException.class, "DN security" +
|
||||||
|
" initialization failed",
|
||||||
|
() -> service.initializeCertificateClient(conf));
|
||||||
|
Assert.assertNull(client.getPrivateKey());
|
||||||
|
Assert.assertNotNull(client.getPublicKey());
|
||||||
|
Assert.assertNull(client.getCertificate());
|
||||||
|
Assert.assertTrue(dnLogs.getOutput().contains("Init response: FAILURE"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSecureDnStartupCase3() throws Exception {
|
||||||
|
// Case 3: When only public key and certificate is present.
|
||||||
|
keyCodec.writePublicKey(publicKey);
|
||||||
|
certCodec.writeCertificate(certHolder);
|
||||||
|
LambdaTestUtils.intercept(RuntimeException.class, "DN security" +
|
||||||
|
" initialization failed",
|
||||||
|
() -> service.initializeCertificateClient(conf));
|
||||||
|
Assert.assertNull(client.getPrivateKey());
|
||||||
|
Assert.assertNotNull(client.getPublicKey());
|
||||||
|
Assert.assertNotNull(client.getCertificate());
|
||||||
|
Assert.assertTrue(dnLogs.getOutput().contains("Init response: FAILURE"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSecureDnStartupCase4() throws Exception {
|
||||||
|
// Case 4: When public key as well as certificate is missing.
|
||||||
|
keyCodec.writePrivateKey(privateKey);
|
||||||
|
LambdaTestUtils.intercept(RuntimeException.class, " DN security" +
|
||||||
|
" initialization failed",
|
||||||
|
() -> service.initializeCertificateClient(conf));
|
||||||
|
Assert.assertNotNull(client.getPrivateKey());
|
||||||
|
Assert.assertNull(client.getPublicKey());
|
||||||
|
Assert.assertNull(client.getCertificate());
|
||||||
|
Assert.assertTrue(dnLogs.getOutput().contains("Init response: FAILURE"));
|
||||||
|
dnLogs.clearOutput();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSecureDnStartupCase5() throws Exception {
|
||||||
|
// Case 5: If private key and certificate is present.
|
||||||
|
certCodec.writeCertificate(certHolder);
|
||||||
|
keyCodec.writePrivateKey(privateKey);
|
||||||
|
service.initializeCertificateClient(conf);
|
||||||
|
Assert.assertNotNull(client.getPrivateKey());
|
||||||
|
Assert.assertNotNull(client.getPublicKey());
|
||||||
|
Assert.assertNotNull(client.getCertificate());
|
||||||
|
Assert.assertTrue(dnLogs.getOutput().contains("Init response: SUCCESS"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSecureDnStartupCase6() throws Exception {
|
||||||
|
// Case 6: If key pair already exist than response should be GETCERT.
|
||||||
|
keyCodec.writePublicKey(publicKey);
|
||||||
|
keyCodec.writePrivateKey(privateKey);
|
||||||
|
LambdaTestUtils.intercept(Exception.class, "",
|
||||||
|
() -> service.initializeCertificateClient(conf));
|
||||||
|
Assert.assertNotNull(client.getPrivateKey());
|
||||||
|
Assert.assertNotNull(client.getPublicKey());
|
||||||
|
Assert.assertNull(client.getCertificate());
|
||||||
|
Assert.assertTrue(dnLogs.getOutput().contains("Init response: GETCERT"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSecureDnStartupCase7() throws Exception {
|
||||||
|
// Case 7 When keypair and certificate is present.
|
||||||
|
keyCodec.writePublicKey(publicKey);
|
||||||
|
keyCodec.writePrivateKey(privateKey);
|
||||||
|
certCodec.writeCertificate(certHolder);
|
||||||
|
|
||||||
|
service.initializeCertificateClient(conf);
|
||||||
|
Assert.assertNotNull(client.getPrivateKey());
|
||||||
|
Assert.assertNotNull(client.getPublicKey());
|
||||||
|
Assert.assertNotNull(client.getCertificate());
|
||||||
|
Assert.assertTrue(dnLogs.getOutput().contains("Init response: SUCCESS"));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Invoke a callable; Ignore all exception.
|
||||||
|
* @param closure closure to execute
|
||||||
|
* @return
|
||||||
|
*/
|
||||||
|
public static void callQuietly(Callable closure) {
|
||||||
|
try {
|
||||||
|
closure.call();
|
||||||
|
} catch (Throwable e) {
|
||||||
|
// Ignore all Throwable,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetCSR() throws Exception {
|
||||||
|
keyCodec.writePublicKey(publicKey);
|
||||||
|
keyCodec.writePrivateKey(privateKey);
|
||||||
|
service.setCertificateClient(client);
|
||||||
|
PKCS10CertificationRequest csr =
|
||||||
|
service.getCSR(conf);
|
||||||
|
Assert.assertNotNull(csr);
|
||||||
|
|
||||||
|
csr = service.getCSR(conf);
|
||||||
|
Assert.assertNotNull(csr);
|
||||||
|
|
||||||
|
csr = service.getCSR(conf);
|
||||||
|
Assert.assertNotNull(csr);
|
||||||
|
|
||||||
|
csr = service.getCSR(conf);
|
||||||
|
Assert.assertNotNull(csr);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -59,6 +59,7 @@ import static org.mockito.Mockito.times;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -88,6 +89,8 @@ public class TestKeyValueHandler {
|
||||||
Mockito.when(dispatcher.dispatch(any(), any())).thenCallRealMethod();
|
Mockito.when(dispatcher.dispatch(any(), any())).thenCallRealMethod();
|
||||||
Mockito.when(dispatcher.getContainer(anyLong())).thenReturn(
|
Mockito.when(dispatcher.getContainer(anyLong())).thenReturn(
|
||||||
Mockito.mock(KeyValueContainer.class));
|
Mockito.mock(KeyValueContainer.class));
|
||||||
|
Mockito.when(dispatcher.getMissingContainerSet())
|
||||||
|
.thenReturn(new HashSet<>());
|
||||||
Mockito.when(handler.handle(any(), any(), any())).thenCallRealMethod();
|
Mockito.when(handler.handle(any(), any(), any())).thenCallRealMethod();
|
||||||
doCallRealMethod().when(dispatcher).setMetricsForTesting(any());
|
doCallRealMethod().when(dispatcher).setMetricsForTesting(any());
|
||||||
dispatcher.setMetricsForTesting(Mockito.mock(ContainerMetrics.class));
|
dispatcher.setMetricsForTesting(Mockito.mock(ContainerMetrics.class));
|
||||||
|
|
|
@ -56,7 +56,7 @@ This will make this bucket to be the default file system for HDFS dfs commands a
|
||||||
You also need to add the ozone-filesystem.jar file to the classpath:
|
You also need to add the ozone-filesystem.jar file to the classpath:
|
||||||
|
|
||||||
{{< highlight bash >}}
|
{{< highlight bash >}}
|
||||||
export HADOOP_CLASSPATH=/opt/ozone/share/ozonefs/lib/hadoop-ozone-filesystem-lib-.*.jar:$HADOOP_CLASSPATH
|
export HADOOP_CLASSPATH=/opt/ozone/share/ozonefs/lib/hadoop-ozone-filesystem-lib-current.*.jar:$HADOOP_CLASSPATH
|
||||||
{{< /highlight >}}
|
{{< /highlight >}}
|
||||||
|
|
||||||
|
|
||||||
|
@ -82,7 +82,7 @@ Please note that any keys created/deleted in the bucket using methods apart from
|
||||||
|
|
||||||
There are two ozonefs files which includes all the dependencies:
|
There are two ozonefs files which includes all the dependencies:
|
||||||
|
|
||||||
* share/ozone/lib/hadoop-ozone-filesystem-lib-VERSION.jar
|
* share/ozone/lib/hadoop-ozone-filesystem-lib-current-VERSION.jar
|
||||||
* share/ozone/lib/hadoop-ozone-filesystem-lib-legacy-VERSION.jar
|
* share/ozone/lib/hadoop-ozone-filesystem-lib-legacy-VERSION.jar
|
||||||
|
|
||||||
The first one contains all the required dependency to use ozonefs with a
|
The first one contains all the required dependency to use ozonefs with a
|
||||||
|
|
|
@ -125,13 +125,29 @@ public final class ServerUtils {
|
||||||
* @return
|
* @return
|
||||||
*/
|
*/
|
||||||
public static File getScmDbDir(Configuration conf) {
|
public static File getScmDbDir(Configuration conf) {
|
||||||
final Collection<String> metadirs = conf.getTrimmedStringCollection(
|
|
||||||
ScmConfigKeys.OZONE_SCM_DB_DIRS);
|
File metadataDir = getDirWithFallBackToOzoneMetadata(conf, ScmConfigKeys
|
||||||
|
.OZONE_SCM_DB_DIRS, "SCM");
|
||||||
|
if (metadataDir != null) {
|
||||||
|
return metadataDir;
|
||||||
|
}
|
||||||
|
|
||||||
|
LOG.warn("{} is not configured. We recommend adding this setting. " +
|
||||||
|
"Falling back to {} instead.",
|
||||||
|
ScmConfigKeys.OZONE_SCM_DB_DIRS, HddsConfigKeys.OZONE_METADATA_DIRS);
|
||||||
|
return getOzoneMetaDirPath(conf);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static File getDirWithFallBackToOzoneMetadata(Configuration conf,
|
||||||
|
String key,
|
||||||
|
String componentName) {
|
||||||
|
final Collection<String> metadirs = conf.getTrimmedStringCollection(key);
|
||||||
|
|
||||||
if (metadirs.size() > 1) {
|
if (metadirs.size() > 1) {
|
||||||
throw new IllegalArgumentException(
|
throw new IllegalArgumentException(
|
||||||
"Bad config setting " + ScmConfigKeys.OZONE_SCM_DB_DIRS +
|
"Bad config setting " + key +
|
||||||
". SCM does not support multiple metadata dirs currently");
|
". " + componentName +
|
||||||
|
" does not support multiple metadata dirs currently");
|
||||||
}
|
}
|
||||||
|
|
||||||
if (metadirs.size() == 1) {
|
if (metadirs.size() == 1) {
|
||||||
|
@ -143,11 +159,7 @@ public final class ServerUtils {
|
||||||
}
|
}
|
||||||
return dbDirPath;
|
return dbDirPath;
|
||||||
}
|
}
|
||||||
|
return null;
|
||||||
LOG.warn("{} is not configured. We recommend adding this setting. " +
|
|
||||||
"Falling back to {} instead.",
|
|
||||||
ScmConfigKeys.OZONE_SCM_DB_DIRS, HddsConfigKeys.OZONE_METADATA_DIRS);
|
|
||||||
return getOzoneMetaDirPath(conf);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -308,7 +308,6 @@
|
||||||
ctrl.convertToArray(response.data);
|
ctrl.convertToArray(response.data);
|
||||||
ctrl.configs = Object.values(ctrl.keyTagMap);
|
ctrl.configs = Object.values(ctrl.keyTagMap);
|
||||||
ctrl.component = 'All';
|
ctrl.component = 'All';
|
||||||
console.log("ajay -> " + JSON.stringify(ctrl.configs));
|
|
||||||
ctrl.sortBy('name');
|
ctrl.sortBy('name');
|
||||||
});
|
});
|
||||||
};
|
};
|
||||||
|
@ -326,7 +325,6 @@
|
||||||
|
|
||||||
if (ctrl.component != 'All' && (item['tag'].indexOf(ctrl
|
if (ctrl.component != 'All' && (item['tag'].indexOf(ctrl
|
||||||
.component) < 0)) {
|
.component) < 0)) {
|
||||||
console.log(item['name'] + " false tag " + item['tag']);
|
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.hadoop.hdds.scm.block;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.hdds.client.BlockID;
|
import org.apache.hadoop.hdds.client.BlockID;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -36,11 +37,14 @@ public interface BlockManager extends Closeable {
|
||||||
* @param size - Block Size
|
* @param size - Block Size
|
||||||
* @param type Replication Type
|
* @param type Replication Type
|
||||||
* @param factor - Replication Factor
|
* @param factor - Replication Factor
|
||||||
|
* @param excludeList List of datanodes/containers to exclude during block
|
||||||
|
* allocation.
|
||||||
* @return AllocatedBlock
|
* @return AllocatedBlock
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
AllocatedBlock allocateBlock(long size, HddsProtos.ReplicationType type,
|
AllocatedBlock allocateBlock(long size, HddsProtos.ReplicationType type,
|
||||||
HddsProtos.ReplicationFactor factor, String owner) throws IOException;
|
HddsProtos.ReplicationFactor factor, String owner,
|
||||||
|
ExcludeList excludeList) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Deletes a list of blocks in an atomic operation. Internally, SCM
|
* Deletes a list of blocks in an atomic operation. Internally, SCM
|
||||||
|
@ -75,4 +79,11 @@ public interface BlockManager extends Closeable {
|
||||||
* @return the block deleting service executed in SCM.
|
* @return the block deleting service executed in SCM.
|
||||||
*/
|
*/
|
||||||
SCMBlockDeletingService getSCMBlockDeletingService();
|
SCMBlockDeletingService getSCMBlockDeletingService();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set ChillMode status.
|
||||||
|
*
|
||||||
|
* @param chillModeStatus
|
||||||
|
*/
|
||||||
|
void setChillModeStatus(boolean chillModeStatus);
|
||||||
}
|
}
|
||||||
|
|
|
@ -35,9 +35,11 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmOps;
|
||||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||||
import org.apache.hadoop.hdds.scm.ScmUtils;
|
import org.apache.hadoop.hdds.scm.ScmUtils;
|
||||||
import org.apache.hadoop.hdds.scm.chillmode.ChillModePrecheck;
|
import org.apache.hadoop.hdds.scm.chillmode.ChillModePrecheck;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
|
||||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
||||||
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
||||||
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
|
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
|
||||||
|
@ -60,10 +62,11 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys
|
||||||
.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT;
|
.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT;
|
||||||
import static org.apache.hadoop.ozone.OzoneConfigKeys
|
import static org.apache.hadoop.ozone.OzoneConfigKeys
|
||||||
.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT;
|
.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT;
|
||||||
|
import java.util.function.Predicate;
|
||||||
|
|
||||||
|
|
||||||
/** Block Manager manages the block access for SCM. */
|
/** Block Manager manages the block access for SCM. */
|
||||||
public class BlockManagerImpl implements EventHandler<Boolean>,
|
public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
|
||||||
BlockManager, BlockmanagerMXBean {
|
|
||||||
private static final Logger LOG =
|
private static final Logger LOG =
|
||||||
LoggerFactory.getLogger(BlockManagerImpl.class);
|
LoggerFactory.getLogger(BlockManagerImpl.class);
|
||||||
// TODO : FIX ME : Hard coding the owner.
|
// TODO : FIX ME : Hard coding the owner.
|
||||||
|
@ -145,12 +148,14 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
||||||
* @param size - Block Size
|
* @param size - Block Size
|
||||||
* @param type Replication Type
|
* @param type Replication Type
|
||||||
* @param factor - Replication Factor
|
* @param factor - Replication Factor
|
||||||
|
* @param excludeList List of datanodes/containers to exclude during block
|
||||||
|
* allocation.
|
||||||
* @return Allocated block
|
* @return Allocated block
|
||||||
* @throws IOException on failure.
|
* @throws IOException on failure.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public AllocatedBlock allocateBlock(final long size,
|
public AllocatedBlock allocateBlock(final long size, ReplicationType type,
|
||||||
ReplicationType type, ReplicationFactor factor, String owner)
|
ReplicationFactor factor, String owner, ExcludeList excludeList)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
LOG.trace("Size;{} , type : {}, factor : {} ", size, type, factor);
|
LOG.trace("Size;{} , type : {}, factor : {} ", size, type, factor);
|
||||||
ScmUtils.preCheck(ScmOps.allocateBlock, chillModePrecheck);
|
ScmUtils.preCheck(ScmOps.allocateBlock, chillModePrecheck);
|
||||||
|
@ -177,8 +182,10 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
||||||
ContainerInfo containerInfo;
|
ContainerInfo containerInfo;
|
||||||
|
|
||||||
while (true) {
|
while (true) {
|
||||||
List<Pipeline> availablePipelines = pipelineManager
|
List<Pipeline> availablePipelines =
|
||||||
.getPipelines(type, factor, Pipeline.PipelineState.OPEN);
|
pipelineManager
|
||||||
|
.getPipelines(type, factor, Pipeline.PipelineState.OPEN,
|
||||||
|
excludeList.getDatanodes(), excludeList.getPipelineIds());
|
||||||
Pipeline pipeline;
|
Pipeline pipeline;
|
||||||
if (availablePipelines.size() == 0) {
|
if (availablePipelines.size() == 0) {
|
||||||
try {
|
try {
|
||||||
|
@ -197,7 +204,13 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
||||||
// look for OPEN containers that match the criteria.
|
// look for OPEN containers that match the criteria.
|
||||||
containerInfo = containerManager
|
containerInfo = containerManager
|
||||||
.getMatchingContainer(size, owner, pipeline);
|
.getMatchingContainer(size, owner, pipeline);
|
||||||
if (containerInfo != null) {
|
|
||||||
|
// TODO: if getMachingContainer results in containers which are in exclude
|
||||||
|
// list, we may end up in this loop forever. This case needs to be
|
||||||
|
// addressed.
|
||||||
|
if (containerInfo != null && (excludeList.getContainerIds() == null
|
||||||
|
|| !discardContainer(containerInfo.containerID(),
|
||||||
|
excludeList.getContainerIds()))) {
|
||||||
return newBlock(containerInfo);
|
return newBlock(containerInfo);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -210,6 +223,11 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private boolean discardContainer(ContainerID containerId,
|
||||||
|
List<ContainerID> containers) {
|
||||||
|
Predicate<ContainerID> predicate = p -> p.equals(containerId);
|
||||||
|
return containers.parallelStream().anyMatch(predicate);
|
||||||
|
}
|
||||||
/**
|
/**
|
||||||
* newBlock - returns a new block assigned to a container.
|
* newBlock - returns a new block assigned to a container.
|
||||||
*
|
*
|
||||||
|
@ -318,8 +336,8 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void onMessage(Boolean inChillMode, EventPublisher publisher) {
|
public void setChillModeStatus(boolean chillModeStatus) {
|
||||||
this.chillModePrecheck.setInChillMode(inChillMode);
|
this.chillModePrecheck.setInChillMode(chillModeStatus);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -0,0 +1,106 @@
|
||||||
|
/*
|
||||||
|
* 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>
|
||||||
|
* <p>http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p>
|
||||||
|
* <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.hdds.scm.chillmode;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hdds.HddsConfigKeys;
|
||||||
|
import org.apache.hadoop.hdds.scm.block.BlockManager;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.replication.
|
||||||
|
ReplicationActivityStatus;
|
||||||
|
import org.apache.hadoop.hdds.scm.server.SCMClientProtocolServer;
|
||||||
|
import org.apache.hadoop.hdds.scm.chillmode.SCMChillModeManager.ChillModeStatus;
|
||||||
|
import org.apache.hadoop.hdds.server.events.EventHandler;
|
||||||
|
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
||||||
|
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Class to handle the activities needed to be performed after exiting chill
|
||||||
|
* mode.
|
||||||
|
*/
|
||||||
|
public class ChillModeHandler implements EventHandler<ChillModeStatus> {
|
||||||
|
|
||||||
|
private final SCMClientProtocolServer scmClientProtocolServer;
|
||||||
|
private final BlockManager scmBlockManager;
|
||||||
|
private final long waitTime;
|
||||||
|
private final AtomicBoolean isInChillMode = new AtomicBoolean(true);
|
||||||
|
private final ReplicationActivityStatus replicationActivityStatus;
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* ChillModeHandler, to handle the logic once we exit chill mode.
|
||||||
|
* @param configuration
|
||||||
|
* @param clientProtocolServer
|
||||||
|
* @param blockManager
|
||||||
|
* @param replicationStatus
|
||||||
|
*/
|
||||||
|
public ChillModeHandler(Configuration configuration,
|
||||||
|
SCMClientProtocolServer clientProtocolServer,
|
||||||
|
BlockManager blockManager,
|
||||||
|
ReplicationActivityStatus replicationStatus) {
|
||||||
|
Objects.requireNonNull(configuration, "Configuration cannot be null");
|
||||||
|
Objects.requireNonNull(clientProtocolServer, "SCMClientProtocolServer " +
|
||||||
|
"object cannot be null");
|
||||||
|
Objects.requireNonNull(blockManager, "BlockManager object cannot be null");
|
||||||
|
Objects.requireNonNull(replicationStatus, "ReplicationActivityStatus " +
|
||||||
|
"object cannot be null");
|
||||||
|
this.waitTime = configuration.getTimeDuration(
|
||||||
|
HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_CHILL_MODE_EXIT,
|
||||||
|
HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_CHILL_MODE_EXIT_DEFAULT,
|
||||||
|
TimeUnit.MILLISECONDS);
|
||||||
|
scmClientProtocolServer = clientProtocolServer;
|
||||||
|
scmBlockManager = blockManager;
|
||||||
|
replicationActivityStatus = replicationStatus;
|
||||||
|
|
||||||
|
boolean chillModeEnabled = configuration.getBoolean(
|
||||||
|
HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED,
|
||||||
|
HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED_DEFAULT);
|
||||||
|
isInChillMode.set(chillModeEnabled);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set ChillMode status based on
|
||||||
|
* {@link org.apache.hadoop.hdds.scm.events.SCMEvents#CHILL_MODE_STATUS}.
|
||||||
|
*
|
||||||
|
* Inform BlockManager, ScmClientProtocolServer and replicationAcitivity
|
||||||
|
* status about chillMode status.
|
||||||
|
*
|
||||||
|
* @param chillModeStatus
|
||||||
|
* @param publisher
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void onMessage(ChillModeStatus chillModeStatus,
|
||||||
|
EventPublisher publisher) {
|
||||||
|
isInChillMode.set(chillModeStatus.getChillModeStatus());
|
||||||
|
|
||||||
|
replicationActivityStatus.fireReplicationStart(isInChillMode.get(),
|
||||||
|
waitTime);
|
||||||
|
scmClientProtocolServer.setChillModeStatus(isInChillMode.get());
|
||||||
|
scmBlockManager.setChillModeStatus(isInChillMode.get());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean getChillModeStatus() {
|
||||||
|
return isInChillMode.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
}
|
|
@ -29,12 +29,15 @@ import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer.NodeRegistrationContainerReport;
|
import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer.NodeRegistrationContainerReport;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
import org.apache.hadoop.hdds.server.events.EventHandler;
|
||||||
|
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Class defining Chill mode exit criteria for Containers.
|
* Class defining Chill mode exit criteria for Containers.
|
||||||
*/
|
*/
|
||||||
public class ContainerChillModeRule implements
|
public class ContainerChillModeRule implements
|
||||||
ChillModeExitRule<NodeRegistrationContainerReport> {
|
ChillModeExitRule<NodeRegistrationContainerReport>,
|
||||||
|
EventHandler<NodeRegistrationContainerReport> {
|
||||||
|
|
||||||
// Required cutoff % for containers with at least 1 reported replica.
|
// Required cutoff % for containers with at least 1 reported replica.
|
||||||
private double chillModeCutoff;
|
private double chillModeCutoff;
|
||||||
|
@ -68,9 +71,6 @@ public class ContainerChillModeRule implements
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean validate() {
|
public boolean validate() {
|
||||||
if (maxContainer == 0) {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
return getCurrentContainerThreshold() >= chillModeCutoff;
|
return getCurrentContainerThreshold() >= chillModeCutoff;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -84,10 +84,6 @@ public class ContainerChillModeRule implements
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void process(NodeRegistrationContainerReport reportsProto) {
|
public void process(NodeRegistrationContainerReport reportsProto) {
|
||||||
if (maxContainer == 0) {
|
|
||||||
// No container to check.
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
reportsProto.getReport().getReportsList().forEach(c -> {
|
reportsProto.getReport().getReportsList().forEach(c -> {
|
||||||
if (containerMap.containsKey(c.getContainerID())) {
|
if (containerMap.containsKey(c.getContainerID())) {
|
||||||
|
@ -96,12 +92,33 @@ public class ContainerChillModeRule implements
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
if(chillModeManager.getInChillMode()) {
|
}
|
||||||
SCMChillModeManager.getLogger().info(
|
|
||||||
"SCM in chill mode. {} % containers have at least one"
|
@Override
|
||||||
+ " reported replica.",
|
public void onMessage(NodeRegistrationContainerReport
|
||||||
(containerWithMinReplicas.get() / maxContainer) * 100);
|
nodeRegistrationContainerReport, EventPublisher publisher) {
|
||||||
|
|
||||||
|
// TODO: when we have remove handlers, we can remove getInChillmode check
|
||||||
|
|
||||||
|
if (chillModeManager.getInChillMode()) {
|
||||||
|
if (validate()) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
process(nodeRegistrationContainerReport);
|
||||||
|
if (chillModeManager.getInChillMode()) {
|
||||||
|
SCMChillModeManager.getLogger().info(
|
||||||
|
"SCM in chill mode. {} % containers have at least one"
|
||||||
|
+ " reported replica.",
|
||||||
|
(containerWithMinReplicas.get() / maxContainer) * 100);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (validate()) {
|
||||||
|
chillModeManager.validateChillModeExitRules(publisher);
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -25,13 +25,16 @@ import org.apache.hadoop.hdds.HddsConfigKeys;
|
||||||
import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer.NodeRegistrationContainerReport;
|
import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer.NodeRegistrationContainerReport;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
import org.apache.hadoop.hdds.server.events.EventHandler;
|
||||||
|
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Class defining Chill mode exit criteria according to number of DataNodes
|
* Class defining Chill mode exit criteria according to number of DataNodes
|
||||||
* registered with SCM.
|
* registered with SCM.
|
||||||
*/
|
*/
|
||||||
public class DataNodeChillModeRule implements
|
public class DataNodeChillModeRule implements
|
||||||
ChillModeExitRule<NodeRegistrationContainerReport> {
|
ChillModeExitRule<NodeRegistrationContainerReport>,
|
||||||
|
EventHandler<NodeRegistrationContainerReport> {
|
||||||
|
|
||||||
// Min DataNodes required to exit chill mode.
|
// Min DataNodes required to exit chill mode.
|
||||||
private int requiredDns;
|
private int requiredDns;
|
||||||
|
@ -62,17 +65,33 @@ public class DataNodeChillModeRule implements
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void process(NodeRegistrationContainerReport reportsProto) {
|
public void process(NodeRegistrationContainerReport reportsProto) {
|
||||||
if (requiredDns == 0) {
|
|
||||||
// No dn check required.
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
if(chillModeManager.getInChillMode()) {
|
registeredDnSet.add(reportsProto.getDatanodeDetails().getUuid());
|
||||||
registeredDnSet.add(reportsProto.getDatanodeDetails().getUuid());
|
registeredDns = registeredDnSet.size();
|
||||||
registeredDns = registeredDnSet.size();
|
|
||||||
SCMChillModeManager.getLogger().info(
|
}
|
||||||
"SCM in chill mode. {} DataNodes registered, {} required.",
|
|
||||||
registeredDns, requiredDns);
|
@Override
|
||||||
|
public void onMessage(NodeRegistrationContainerReport
|
||||||
|
nodeRegistrationContainerReport, EventPublisher publisher) {
|
||||||
|
// TODO: when we have remove handlers, we can remove getInChillmode check
|
||||||
|
|
||||||
|
if (chillModeManager.getInChillMode()) {
|
||||||
|
if (validate()) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
process(nodeRegistrationContainerReport);
|
||||||
|
|
||||||
|
if (chillModeManager.getInChillMode()) {
|
||||||
|
SCMChillModeManager.getLogger().info(
|
||||||
|
"SCM in chill mode. {} DataNodes registered, {} required.",
|
||||||
|
registeredDns, requiredDns);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (validate()) {
|
||||||
|
chillModeManager.validateChillModeExitRules(publisher);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,142 @@
|
||||||
|
/**
|
||||||
|
* 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.hdds.scm.chillmode;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hdds.HddsConfigKeys;
|
||||||
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
|
import org.apache.hadoop.hdds.protocol.proto
|
||||||
|
.StorageContainerDatanodeProtocolProtos.PipelineReport;
|
||||||
|
import org.apache.hadoop.hdds.protocol.proto
|
||||||
|
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
|
||||||
|
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
||||||
|
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
|
||||||
|
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
|
||||||
|
import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException;
|
||||||
|
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.
|
||||||
|
PipelineReportFromDatanode;
|
||||||
|
import org.apache.hadoop.hdds.server.events.EventHandler;
|
||||||
|
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This rule covers whether we have atleast one datanode is reported for each
|
||||||
|
* pipeline. This rule is for all open containers, we have at least one
|
||||||
|
* replica available for read when we exit chill mode.
|
||||||
|
*/
|
||||||
|
public class OneReplicaPipelineChillModeRule implements
|
||||||
|
ChillModeExitRule<PipelineReportFromDatanode>,
|
||||||
|
EventHandler<PipelineReportFromDatanode> {
|
||||||
|
|
||||||
|
private static final Logger LOG =
|
||||||
|
LoggerFactory.getLogger(OneReplicaPipelineChillModeRule.class);
|
||||||
|
|
||||||
|
private int thresholdCount;
|
||||||
|
private Set<PipelineID> reportedPipelineIDSet = new HashSet<>();
|
||||||
|
private final PipelineManager pipelineManager;
|
||||||
|
private final SCMChillModeManager chillModeManager;
|
||||||
|
|
||||||
|
public OneReplicaPipelineChillModeRule(PipelineManager pipelineManager,
|
||||||
|
SCMChillModeManager chillModeManager,
|
||||||
|
Configuration configuration) {
|
||||||
|
this.chillModeManager = chillModeManager;
|
||||||
|
this.pipelineManager = pipelineManager;
|
||||||
|
|
||||||
|
double percent =
|
||||||
|
configuration.getDouble(
|
||||||
|
HddsConfigKeys.HDDS_SCM_CHILLMODE_ONE_NODE_REPORTED_PIPELINE_PCT,
|
||||||
|
HddsConfigKeys.
|
||||||
|
HDDS_SCM_CHILLMODE_ONE_NODE_REPORTED_PIPELINE_PCT_DEFAULT);
|
||||||
|
|
||||||
|
int totalPipelineCount =
|
||||||
|
pipelineManager.getPipelines(HddsProtos.ReplicationType.RATIS,
|
||||||
|
HddsProtos.ReplicationFactor.THREE).size();
|
||||||
|
|
||||||
|
thresholdCount = (int) Math.ceil(percent * totalPipelineCount);
|
||||||
|
|
||||||
|
LOG.info(" Total pipeline count is {}, pipeline's with atleast one " +
|
||||||
|
"datanode reported threshold count is {}", totalPipelineCount,
|
||||||
|
thresholdCount);
|
||||||
|
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
public boolean validate() {
|
||||||
|
if (reportedPipelineIDSet.size() >= thresholdCount) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void process(PipelineReportFromDatanode pipelineReportFromDatanode) {
|
||||||
|
Pipeline pipeline;
|
||||||
|
Preconditions.checkNotNull(pipelineReportFromDatanode);
|
||||||
|
PipelineReportsProto pipelineReport =
|
||||||
|
pipelineReportFromDatanode.getReport();
|
||||||
|
|
||||||
|
for (PipelineReport report : pipelineReport.getPipelineReportList()) {
|
||||||
|
PipelineID pipelineID = PipelineID
|
||||||
|
.getFromProtobuf(report.getPipelineID());
|
||||||
|
try {
|
||||||
|
pipeline = pipelineManager.getPipeline(pipelineID);
|
||||||
|
} catch (PipelineNotFoundException e) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (pipeline.getFactor() == HddsProtos.ReplicationFactor.THREE &&
|
||||||
|
!reportedPipelineIDSet.contains(pipelineID)) {
|
||||||
|
reportedPipelineIDSet.add(pipelineID);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void cleanup() {
|
||||||
|
reportedPipelineIDSet.clear();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onMessage(PipelineReportFromDatanode pipelineReportFromDatanode,
|
||||||
|
EventPublisher publisher) {
|
||||||
|
|
||||||
|
if (validate()) {
|
||||||
|
chillModeManager.validateChillModeExitRules(publisher);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Process pipeline report from datanode
|
||||||
|
process(pipelineReportFromDatanode);
|
||||||
|
|
||||||
|
if (chillModeManager.getInChillMode()) {
|
||||||
|
SCMChillModeManager.getLogger().info(
|
||||||
|
"SCM in chill mode. Pipelines with atleast one datanode reported " +
|
||||||
|
"count is {}, required atleast one datanode reported per " +
|
||||||
|
"pipeline count is {}",
|
||||||
|
reportedPipelineIDSet.size(), thresholdCount);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (validate()) {
|
||||||
|
chillModeManager.validateChillModeExitRules(publisher);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -28,9 +28,6 @@ import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
||||||
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
|
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
|
||||||
import org.apache.hadoop.hdds.scm.pipeline.RatisPipelineUtils;
|
import org.apache.hadoop.hdds.scm.pipeline.RatisPipelineUtils;
|
||||||
import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer
|
|
||||||
.NodeRegistrationContainerReport;
|
|
||||||
import org.apache.hadoop.hdds.server.events.EventHandler;
|
|
||||||
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
||||||
import org.apache.hadoop.hdds.server.events.EventQueue;
|
import org.apache.hadoop.hdds.server.events.EventQueue;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
@ -48,8 +45,7 @@ import org.slf4j.LoggerFactory;
|
||||||
* for reported containers and validates if cutoff threshold for
|
* for reported containers and validates if cutoff threshold for
|
||||||
* containers is meet.
|
* containers is meet.
|
||||||
*/
|
*/
|
||||||
public class SCMChillModeManager implements
|
public class SCMChillModeManager {
|
||||||
EventHandler<NodeRegistrationContainerReport> {
|
|
||||||
|
|
||||||
private static final Logger LOG =
|
private static final Logger LOG =
|
||||||
LoggerFactory.getLogger(SCMChillModeManager.class);
|
LoggerFactory.getLogger(SCMChillModeManager.class);
|
||||||
|
@ -62,6 +58,8 @@ public class SCMChillModeManager implements
|
||||||
private static final String DN_EXIT_RULE = "DataNodeChillModeRule";
|
private static final String DN_EXIT_RULE = "DataNodeChillModeRule";
|
||||||
private static final String HEALTHY_PIPELINE_EXIT_RULE =
|
private static final String HEALTHY_PIPELINE_EXIT_RULE =
|
||||||
"HealthyPipelineChillModeRule";
|
"HealthyPipelineChillModeRule";
|
||||||
|
private static final String ATLEAST_ONE_DATANODE_REPORTED_PIPELINE_EXIT_RULE =
|
||||||
|
"AtleastOneDatanodeReportedRule";
|
||||||
|
|
||||||
private final EventQueue eventPublisher;
|
private final EventQueue eventPublisher;
|
||||||
private final PipelineManager pipelineManager;
|
private final PipelineManager pipelineManager;
|
||||||
|
@ -75,10 +73,18 @@ public class SCMChillModeManager implements
|
||||||
this.isChillModeEnabled = conf.getBoolean(
|
this.isChillModeEnabled = conf.getBoolean(
|
||||||
HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED,
|
HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED,
|
||||||
HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED_DEFAULT);
|
HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED_DEFAULT);
|
||||||
|
|
||||||
if (isChillModeEnabled) {
|
if (isChillModeEnabled) {
|
||||||
exitRules.put(CONT_EXIT_RULE,
|
ContainerChillModeRule containerChillModeRule =
|
||||||
new ContainerChillModeRule(config, allContainers, this));
|
new ContainerChillModeRule(config, allContainers, this);
|
||||||
exitRules.put(DN_EXIT_RULE, new DataNodeChillModeRule(config, this));
|
DataNodeChillModeRule dataNodeChillModeRule =
|
||||||
|
new DataNodeChillModeRule(config, this);
|
||||||
|
exitRules.put(CONT_EXIT_RULE, containerChillModeRule);
|
||||||
|
exitRules.put(DN_EXIT_RULE, dataNodeChillModeRule);
|
||||||
|
eventPublisher.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
|
||||||
|
containerChillModeRule);
|
||||||
|
eventPublisher.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
|
||||||
|
dataNodeChillModeRule);
|
||||||
|
|
||||||
if (conf.getBoolean(
|
if (conf.getBoolean(
|
||||||
HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK,
|
HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK,
|
||||||
|
@ -86,8 +92,14 @@ public class SCMChillModeManager implements
|
||||||
&& pipelineManager != null) {
|
&& pipelineManager != null) {
|
||||||
HealthyPipelineChillModeRule rule = new HealthyPipelineChillModeRule(
|
HealthyPipelineChillModeRule rule = new HealthyPipelineChillModeRule(
|
||||||
pipelineManager, this, config);
|
pipelineManager, this, config);
|
||||||
|
OneReplicaPipelineChillModeRule oneReplicaPipelineChillModeRule =
|
||||||
|
new OneReplicaPipelineChillModeRule(pipelineManager, this, conf);
|
||||||
exitRules.put(HEALTHY_PIPELINE_EXIT_RULE, rule);
|
exitRules.put(HEALTHY_PIPELINE_EXIT_RULE, rule);
|
||||||
|
exitRules.put(ATLEAST_ONE_DATANODE_REPORTED_PIPELINE_EXIT_RULE,
|
||||||
|
oneReplicaPipelineChillModeRule);
|
||||||
eventPublisher.addHandler(SCMEvents.PROCESSED_PIPELINE_REPORT, rule);
|
eventPublisher.addHandler(SCMEvents.PROCESSED_PIPELINE_REPORT, rule);
|
||||||
|
eventPublisher.addHandler(SCMEvents.PROCESSED_PIPELINE_REPORT,
|
||||||
|
oneReplicaPipelineChillModeRule);
|
||||||
}
|
}
|
||||||
emitChillModeStatus();
|
emitChillModeStatus();
|
||||||
} else {
|
} else {
|
||||||
|
@ -100,7 +112,8 @@ public class SCMChillModeManager implements
|
||||||
*/
|
*/
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
public void emitChillModeStatus() {
|
public void emitChillModeStatus() {
|
||||||
eventPublisher.fireEvent(SCMEvents.CHILL_MODE_STATUS, getInChillMode());
|
eventPublisher.fireEvent(SCMEvents.CHILL_MODE_STATUS,
|
||||||
|
new ChillModeStatus(getInChillMode()));
|
||||||
}
|
}
|
||||||
|
|
||||||
public void validateChillModeExitRules(EventPublisher eventQueue) {
|
public void validateChillModeExitRules(EventPublisher eventQueue) {
|
||||||
|
@ -138,17 +151,6 @@ public class SCMChillModeManager implements
|
||||||
.scheduleFixedIntervalPipelineCreator(pipelineManager, config);
|
.scheduleFixedIntervalPipelineCreator(pipelineManager, config);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public void onMessage(
|
|
||||||
NodeRegistrationContainerReport nodeRegistrationContainerReport,
|
|
||||||
EventPublisher publisher) {
|
|
||||||
if (getInChillMode()) {
|
|
||||||
exitRules.get(CONT_EXIT_RULE).process(nodeRegistrationContainerReport);
|
|
||||||
exitRules.get(DN_EXIT_RULE).process(nodeRegistrationContainerReport);
|
|
||||||
validateChillModeExitRules(publisher);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public boolean getInChillMode() {
|
public boolean getInChillMode() {
|
||||||
if (!isChillModeEnabled) {
|
if (!isChillModeEnabled) {
|
||||||
return false;
|
return false;
|
||||||
|
@ -179,4 +181,26 @@ public class SCMChillModeManager implements
|
||||||
exitRules.get(HEALTHY_PIPELINE_EXIT_RULE);
|
exitRules.get(HEALTHY_PIPELINE_EXIT_RULE);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public OneReplicaPipelineChillModeRule getOneReplicaPipelineChillModeRule() {
|
||||||
|
return (OneReplicaPipelineChillModeRule)
|
||||||
|
exitRules.get(ATLEAST_ONE_DATANODE_REPORTED_PIPELINE_EXIT_RULE);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Class used during ChillMode status event.
|
||||||
|
*/
|
||||||
|
public static class ChillModeStatus {
|
||||||
|
|
||||||
|
private boolean chillModeStatus;
|
||||||
|
public ChillModeStatus(boolean chillModeState) {
|
||||||
|
this.chillModeStatus = chillModeState;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean getChillModeStatus() {
|
||||||
|
return chillModeStatus;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,11 +20,13 @@ package org.apache.hadoop.hdds.scm.container.replication;
|
||||||
import javax.management.ObjectName;
|
import javax.management.ObjectName;
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import org.apache.hadoop.hdds.server.events.EventHandler;
|
|
||||||
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
import org.apache.hadoop.hdds.HddsConfigKeys;
|
||||||
import org.apache.hadoop.metrics2.util.MBeans;
|
import org.apache.hadoop.metrics2.util.MBeans;
|
||||||
|
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
@ -39,15 +41,8 @@ public class ReplicationActivityStatus implements
|
||||||
LoggerFactory.getLogger(ReplicationActivityStatus.class);
|
LoggerFactory.getLogger(ReplicationActivityStatus.class);
|
||||||
|
|
||||||
private AtomicBoolean replicationEnabled = new AtomicBoolean();
|
private AtomicBoolean replicationEnabled = new AtomicBoolean();
|
||||||
private AtomicBoolean replicationStatusSetExternally = new AtomicBoolean();
|
|
||||||
private ObjectName jmxObjectName;
|
private ObjectName jmxObjectName;
|
||||||
private ReplicationStatusListener replicationStatusListener;
|
|
||||||
private ChillModeStatusListener chillModeStatusListener;
|
|
||||||
|
|
||||||
public ReplicationActivityStatus(){
|
|
||||||
replicationStatusListener = new ReplicationStatusListener();
|
|
||||||
chillModeStatusListener = new ChillModeStatusListener();
|
|
||||||
}
|
|
||||||
@Override
|
@Override
|
||||||
public boolean isReplicationEnabled() {
|
public boolean isReplicationEnabled() {
|
||||||
return replicationEnabled.get();
|
return replicationEnabled.get();
|
||||||
|
@ -84,35 +79,26 @@ public class ReplicationActivityStatus implements
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Replication status listener.
|
* Waits for
|
||||||
|
* {@link HddsConfigKeys#HDDS_SCM_WAIT_TIME_AFTER_CHILL_MODE_EXIT} and set
|
||||||
|
* replicationEnabled to start replication monitor thread.
|
||||||
*/
|
*/
|
||||||
class ReplicationStatusListener implements EventHandler<Boolean> {
|
public void fireReplicationStart(boolean chillModeStatus,
|
||||||
@Override
|
long waitTime) {
|
||||||
public void onMessage(Boolean status, EventPublisher publisher) {
|
if (!chillModeStatus) {
|
||||||
replicationStatusSetExternally.set(true);
|
CompletableFuture.runAsync(() -> {
|
||||||
replicationEnabled.set(status);
|
try {
|
||||||
|
Thread.sleep(waitTime);
|
||||||
|
} catch (InterruptedException ex) {
|
||||||
|
LOG.error("Interrupted during wait, replication event is not fired",
|
||||||
|
ex);
|
||||||
|
}
|
||||||
|
setReplicationEnabled(true);
|
||||||
|
LOG.info("Replication Timer sleep for {} ms completed. Enable " +
|
||||||
|
"Replication", waitTime);
|
||||||
|
});
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Replication status is influenced by Chill mode status as well.
|
|
||||||
*/
|
|
||||||
class ChillModeStatusListener implements EventHandler<Boolean> {
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void onMessage(Boolean inChillMode, EventPublisher publisher) {
|
|
||||||
if (!replicationStatusSetExternally.get()) {
|
|
||||||
replicationEnabled.set(!inChillMode);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public ReplicationStatusListener getReplicationStatusListener() {
|
|
||||||
return replicationStatusListener;
|
|
||||||
}
|
|
||||||
|
|
||||||
public ChillModeStatusListener getChillModeStatusListener() {
|
|
||||||
return chillModeStatusListener;
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.hdds.scm.events;
|
||||||
|
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.scm.block.PendingDeleteStatusList;
|
import org.apache.hadoop.hdds.scm.block.PendingDeleteStatusList;
|
||||||
|
import org.apache.hadoop.hdds.scm.chillmode.SCMChillModeManager.ChillModeStatus;
|
||||||
import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler;
|
import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler;
|
||||||
import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler
|
import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler
|
||||||
.ReplicationStatus;
|
.ReplicationStatus;
|
||||||
|
@ -253,8 +254,8 @@ public final class SCMEvents {
|
||||||
*/
|
*/
|
||||||
public static final TypedEvent<Boolean> START_REPLICATION =
|
public static final TypedEvent<Boolean> START_REPLICATION =
|
||||||
new TypedEvent<>(Boolean.class);
|
new TypedEvent<>(Boolean.class);
|
||||||
public static final TypedEvent<Boolean> CHILL_MODE_STATUS =
|
public static final TypedEvent<ChillModeStatus> CHILL_MODE_STATUS =
|
||||||
new TypedEvent<>(Boolean.class);
|
new TypedEvent<>(ChillModeStatus.class);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Private Ctor. Never Constructed.
|
* Private Ctor. Never Constructed.
|
||||||
|
|
|
@ -18,6 +18,7 @@
|
||||||
|
|
||||||
package org.apache.hadoop.hdds.scm.pipeline;
|
package org.apache.hadoop.hdds.scm.pipeline;
|
||||||
|
|
||||||
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
||||||
|
@ -45,6 +46,12 @@ public final class PipelineFactory {
|
||||||
new RatisPipelineProvider(nodeManager, stateManager, conf));
|
new RatisPipelineProvider(nodeManager, stateManager, conf));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
void setProvider(ReplicationType replicationType,
|
||||||
|
PipelineProvider provider) {
|
||||||
|
providers.put(replicationType, provider);
|
||||||
|
}
|
||||||
|
|
||||||
public Pipeline create(ReplicationType type, ReplicationFactor factor)
|
public Pipeline create(ReplicationType type, ReplicationFactor factor)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return providers.get(type).create(factor);
|
return providers.get(type).create(factor);
|
||||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.Collection;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.NavigableSet;
|
import java.util.NavigableSet;
|
||||||
|
|
||||||
|
@ -51,6 +52,10 @@ public interface PipelineManager extends Closeable, PipelineManagerMXBean {
|
||||||
List<Pipeline> getPipelines(ReplicationType type,
|
List<Pipeline> getPipelines(ReplicationType type,
|
||||||
ReplicationFactor factor, Pipeline.PipelineState state);
|
ReplicationFactor factor, Pipeline.PipelineState state);
|
||||||
|
|
||||||
|
List<Pipeline> getPipelines(ReplicationType type, ReplicationFactor factor,
|
||||||
|
Pipeline.PipelineState state, Collection<DatanodeDetails> excludeDns,
|
||||||
|
Collection<PipelineID> excludePipelines);
|
||||||
|
|
||||||
void addContainerToPipeline(PipelineID pipelineID, ContainerID containerID)
|
void addContainerToPipeline(PipelineID pipelineID, ContainerID containerID)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
package org.apache.hadoop.hdds.scm.pipeline;
|
package org.apache.hadoop.hdds.scm.pipeline;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
|
@ -27,6 +28,7 @@ import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.Collection;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.NavigableSet;
|
import java.util.NavigableSet;
|
||||||
|
|
||||||
|
@ -81,6 +83,13 @@ class PipelineStateManager {
|
||||||
return pipelineStateMap.getPipelines(type, factor, state);
|
return pipelineStateMap.getPipelines(type, factor, state);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
List<Pipeline> getPipelines(ReplicationType type, ReplicationFactor factor,
|
||||||
|
PipelineState state, Collection<DatanodeDetails> excludeDns,
|
||||||
|
Collection<PipelineID> excludePipelines) {
|
||||||
|
return pipelineStateMap
|
||||||
|
.getPipelines(type, factor, state, excludeDns, excludePipelines);
|
||||||
|
}
|
||||||
|
|
||||||
List<Pipeline> getPipelines(ReplicationType type, PipelineState... states) {
|
List<Pipeline> getPipelines(ReplicationType type, PipelineState... states) {
|
||||||
return pipelineStateMap.getPipelines(type, states);
|
return pipelineStateMap.getPipelines(type, states);
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.hadoop.hdds.scm.pipeline;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
|
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
|
||||||
import org.apache.commons.lang3.builder.HashCodeBuilder;
|
import org.apache.commons.lang3.builder.HashCodeBuilder;
|
||||||
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
|
@ -30,6 +31,7 @@ import org.slf4j.LoggerFactory;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.*;
|
import java.util.*;
|
||||||
import java.util.concurrent.CopyOnWriteArrayList;
|
import java.util.concurrent.CopyOnWriteArrayList;
|
||||||
|
import java.util.function.Predicate;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -217,6 +219,57 @@ class PipelineStateMap {
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get list of pipeline corresponding to specified replication type,
|
||||||
|
* replication factor and pipeline state.
|
||||||
|
*
|
||||||
|
* @param type - ReplicationType
|
||||||
|
* @param state - Required PipelineState
|
||||||
|
* @param excludeDns list of dns to exclude
|
||||||
|
* @param excludePipelines pipelines to exclude
|
||||||
|
* @return List of pipelines with specified replication type,
|
||||||
|
* replication factor and pipeline state
|
||||||
|
*/
|
||||||
|
List<Pipeline> getPipelines(ReplicationType type, ReplicationFactor factor,
|
||||||
|
PipelineState state, Collection<DatanodeDetails> excludeDns,
|
||||||
|
Collection<PipelineID> excludePipelines) {
|
||||||
|
Preconditions.checkNotNull(type, "Replication type cannot be null");
|
||||||
|
Preconditions.checkNotNull(factor, "Replication factor cannot be null");
|
||||||
|
Preconditions.checkNotNull(state, "Pipeline state cannot be null");
|
||||||
|
Preconditions
|
||||||
|
.checkNotNull(excludeDns, "Datanode exclude list cannot be null");
|
||||||
|
Preconditions
|
||||||
|
.checkNotNull(excludeDns, "Pipeline exclude list cannot be null");
|
||||||
|
return getPipelines(type, factor, state).stream().filter(
|
||||||
|
pipeline -> !discardPipeline(pipeline, excludePipelines)
|
||||||
|
&& !discardDatanode(pipeline, excludeDns))
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean discardPipeline(Pipeline pipeline,
|
||||||
|
Collection<PipelineID> excludePipelines) {
|
||||||
|
if (excludePipelines.isEmpty()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
Predicate<PipelineID> predicate = p -> p.equals(pipeline.getId());
|
||||||
|
return excludePipelines.parallelStream().anyMatch(predicate);
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean discardDatanode(Pipeline pipeline,
|
||||||
|
Collection<DatanodeDetails> excludeDns) {
|
||||||
|
if (excludeDns.isEmpty()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
boolean discard = false;
|
||||||
|
for (DatanodeDetails dn : pipeline.getNodes()) {
|
||||||
|
Predicate<DatanodeDetails> predicate = p -> p.equals(dn);
|
||||||
|
discard = excludeDns.parallelStream().anyMatch(predicate);
|
||||||
|
if (discard) {
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return discard;
|
||||||
|
}
|
||||||
/**
|
/**
|
||||||
* Get set of containerIDs corresponding to a pipeline.
|
* Get set of containerIDs corresponding to a pipeline.
|
||||||
*
|
*
|
||||||
|
|
|
@ -143,7 +143,7 @@ public class RatisPipelineProvider implements PipelineProvider {
|
||||||
.build();
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
private void initializePipeline(Pipeline pipeline) throws IOException {
|
protected void initializePipeline(Pipeline pipeline) throws IOException {
|
||||||
RatisPipelineUtils.createPipeline(pipeline, conf);
|
RatisPipelineUtils.createPipeline(pipeline, conf);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||||
import org.apache.ratis.RatisHelper;
|
import org.apache.ratis.RatisHelper;
|
||||||
import org.apache.ratis.client.RaftClient;
|
import org.apache.ratis.client.RaftClient;
|
||||||
import org.apache.ratis.grpc.GrpcTlsConfig;
|
import org.apache.ratis.grpc.GrpcTlsConfig;
|
||||||
|
import org.apache.ratis.protocol.RaftClientReply;
|
||||||
import org.apache.ratis.protocol.RaftGroup;
|
import org.apache.ratis.protocol.RaftGroup;
|
||||||
import org.apache.ratis.protocol.RaftGroupId;
|
import org.apache.ratis.protocol.RaftGroupId;
|
||||||
import org.apache.ratis.protocol.RaftPeer;
|
import org.apache.ratis.protocol.RaftPeer;
|
||||||
|
@ -71,7 +72,15 @@ public final class RatisPipelineUtils {
|
||||||
final RaftGroup group = RatisHelper.newRaftGroup(pipeline);
|
final RaftGroup group = RatisHelper.newRaftGroup(pipeline);
|
||||||
LOG.debug("creating pipeline:{} with {}", pipeline.getId(), group);
|
LOG.debug("creating pipeline:{} with {}", pipeline.getId(), group);
|
||||||
callRatisRpc(pipeline.getNodes(), ozoneConf,
|
callRatisRpc(pipeline.getNodes(), ozoneConf,
|
||||||
(raftClient, peer) -> raftClient.groupAdd(group, peer.getId()));
|
(raftClient, peer) -> {
|
||||||
|
RaftClientReply reply = raftClient.groupAdd(group, peer.getId());
|
||||||
|
if (reply == null || !reply.isSuccess()) {
|
||||||
|
String msg = "Pipeline initialization failed for pipeline:"
|
||||||
|
+ pipeline.getId() + " node:" + peer.getId();
|
||||||
|
LOG.error(msg);
|
||||||
|
throw new IOException(msg);
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -186,8 +195,8 @@ public final class RatisPipelineUtils {
|
||||||
rpc.accept(client, p);
|
rpc.accept(client, p);
|
||||||
} catch (IOException ioe) {
|
} catch (IOException ioe) {
|
||||||
exceptions.add(
|
exceptions.add(
|
||||||
new IOException("Failed invoke Ratis rpc " + rpc + " for " + d,
|
new IOException("Failed invoke Ratis rpc " + rpc + " for " +
|
||||||
ioe));
|
d.getUuid(), ioe));
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
if (!exceptions.isEmpty()) {
|
if (!exceptions.isEmpty()) {
|
||||||
|
|
|
@ -45,6 +45,7 @@ import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.NavigableSet;
|
import java.util.NavigableSet;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.Collection;
|
||||||
import java.util.concurrent.locks.ReadWriteLock;
|
import java.util.concurrent.locks.ReadWriteLock;
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
|
|
||||||
|
@ -99,6 +100,15 @@ public class SCMPipelineManager implements PipelineManager {
|
||||||
initializePipelineState();
|
initializePipelineState();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public PipelineStateManager getStateManager() {
|
||||||
|
return stateManager;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setPipelineProvider(ReplicationType replicationType,
|
||||||
|
PipelineProvider provider) {
|
||||||
|
pipelineFactory.setProvider(replicationType, provider);
|
||||||
|
}
|
||||||
|
|
||||||
private void initializePipelineState() throws IOException {
|
private void initializePipelineState() throws IOException {
|
||||||
if (pipelineStore.isEmpty()) {
|
if (pipelineStore.isEmpty()) {
|
||||||
LOG.info("No pipeline exists in current db");
|
LOG.info("No pipeline exists in current db");
|
||||||
|
@ -205,6 +215,20 @@ public class SCMPipelineManager implements PipelineManager {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<Pipeline> getPipelines(ReplicationType type,
|
||||||
|
ReplicationFactor factor, Pipeline.PipelineState state,
|
||||||
|
Collection<DatanodeDetails> excludeDns,
|
||||||
|
Collection<PipelineID> excludePipelines) {
|
||||||
|
lock.readLock().lock();
|
||||||
|
try {
|
||||||
|
return stateManager
|
||||||
|
.getPipelines(type, factor, state, excludeDns, excludePipelines);
|
||||||
|
} finally {
|
||||||
|
lock.readLock().unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void addContainerToPipeline(PipelineID pipelineID,
|
public void addContainerToPipeline(PipelineID pipelineID,
|
||||||
ContainerID containerID) throws IOException {
|
ContainerID containerID) throws IOException {
|
||||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.hadoop.hdds.scm.HddsServerUtil;
|
||||||
import org.apache.hadoop.hdds.scm.ScmInfo;
|
import org.apache.hadoop.hdds.scm.ScmInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.DeleteBlockResult;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.DeleteBlockResult;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
|
||||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
||||||
import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
|
import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
|
||||||
import org.apache.hadoop.hdds.scm.protocolPB.ScmBlockLocationProtocolPB;
|
import org.apache.hadoop.hdds.scm.protocolPB.ScmBlockLocationProtocolPB;
|
||||||
|
@ -155,9 +156,9 @@ public class SCMBlockProtocolServer implements
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public AllocatedBlock allocateBlock(long size, HddsProtos.ReplicationType
|
public AllocatedBlock allocateBlock(long size,
|
||||||
type, HddsProtos.ReplicationFactor factor, String owner) throws
|
HddsProtos.ReplicationType type, HddsProtos.ReplicationFactor factor,
|
||||||
IOException {
|
String owner, ExcludeList excludeList) throws IOException {
|
||||||
Map<String, String> auditMap = Maps.newHashMap();
|
Map<String, String> auditMap = Maps.newHashMap();
|
||||||
auditMap.put("size", String.valueOf(size));
|
auditMap.put("size", String.valueOf(size));
|
||||||
auditMap.put("type", type.name());
|
auditMap.put("type", type.name());
|
||||||
|
@ -165,7 +166,8 @@ public class SCMBlockProtocolServer implements
|
||||||
auditMap.put("owner", owner);
|
auditMap.put("owner", owner);
|
||||||
boolean auditSuccess = true;
|
boolean auditSuccess = true;
|
||||||
try {
|
try {
|
||||||
return scm.getScmBlockManager().allocateBlock(size, type, factor, owner);
|
return scm.getScmBlockManager()
|
||||||
|
.allocateBlock(size, type, factor, owner, excludeList);
|
||||||
} catch (Exception ex) {
|
} catch (Exception ex) {
|
||||||
auditSuccess = false;
|
auditSuccess = false;
|
||||||
AUDIT.logWriteFailure(
|
AUDIT.logWriteFailure(
|
||||||
|
|
|
@ -49,8 +49,6 @@ import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
|
||||||
import org.apache.hadoop.hdds.scm.pipeline.RatisPipelineUtils;
|
import org.apache.hadoop.hdds.scm.pipeline.RatisPipelineUtils;
|
||||||
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
|
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
|
||||||
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
|
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
|
||||||
import org.apache.hadoop.hdds.server.events.EventHandler;
|
|
||||||
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
||||||
import org.apache.hadoop.ipc.RPC;
|
import org.apache.hadoop.ipc.RPC;
|
||||||
|
@ -95,7 +93,7 @@ import static org.apache.hadoop.hdds.scm.server.StorageContainerManager
|
||||||
* The RPC server that listens to requests from clients.
|
* The RPC server that listens to requests from clients.
|
||||||
*/
|
*/
|
||||||
public class SCMClientProtocolServer implements
|
public class SCMClientProtocolServer implements
|
||||||
StorageContainerLocationProtocol, EventHandler<Boolean>, Auditor {
|
StorageContainerLocationProtocol, Auditor {
|
||||||
private static final Logger LOG =
|
private static final Logger LOG =
|
||||||
LoggerFactory.getLogger(SCMClientProtocolServer.class);
|
LoggerFactory.getLogger(SCMClientProtocolServer.class);
|
||||||
private static final AuditLogger AUDIT =
|
private static final AuditLogger AUDIT =
|
||||||
|
@ -496,14 +494,6 @@ public class SCMClientProtocolServer implements
|
||||||
return scm;
|
return scm;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Set chill mode status based on SCMEvents.CHILL_MODE_STATUS event.
|
|
||||||
*/
|
|
||||||
@Override
|
|
||||||
public void onMessage(Boolean inChillMode, EventPublisher publisher) {
|
|
||||||
chillModePrecheck.setInChillMode(inChillMode);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set chill mode status based on .
|
* Set chill mode status based on .
|
||||||
*/
|
*/
|
||||||
|
@ -561,4 +551,13 @@ public class SCMClientProtocolServer implements
|
||||||
public void close() throws IOException {
|
public void close() throws IOException {
|
||||||
stop();
|
stop();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set ChillMode status.
|
||||||
|
*
|
||||||
|
* @param chillModeStatus
|
||||||
|
*/
|
||||||
|
public void setChillModeStatus(boolean chillModeStatus) {
|
||||||
|
chillModePrecheck.setInChillMode(chillModeStatus);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.hadoop.hdds.scm.block.BlockManager;
|
||||||
import org.apache.hadoop.hdds.scm.block.BlockManagerImpl;
|
import org.apache.hadoop.hdds.scm.block.BlockManagerImpl;
|
||||||
import org.apache.hadoop.hdds.scm.block.DeletedBlockLogImpl;
|
import org.apache.hadoop.hdds.scm.block.DeletedBlockLogImpl;
|
||||||
import org.apache.hadoop.hdds.scm.block.PendingDeleteHandler;
|
import org.apache.hadoop.hdds.scm.block.PendingDeleteHandler;
|
||||||
|
import org.apache.hadoop.hdds.scm.chillmode.ChillModeHandler;
|
||||||
import org.apache.hadoop.hdds.scm.chillmode.SCMChillModeManager;
|
import org.apache.hadoop.hdds.scm.chillmode.SCMChillModeManager;
|
||||||
import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler;
|
import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler;
|
||||||
import org.apache.hadoop.hdds.scm.container.CloseContainerEventHandler;
|
import org.apache.hadoop.hdds.scm.container.CloseContainerEventHandler;
|
||||||
|
@ -202,6 +203,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
||||||
|
|
||||||
private JvmPauseMonitor jvmPauseMonitor;
|
private JvmPauseMonitor jvmPauseMonitor;
|
||||||
private final OzoneConfiguration configuration;
|
private final OzoneConfiguration configuration;
|
||||||
|
private final ChillModeHandler chillModeHandler;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a new StorageContainerManager. Configuration will be
|
* Creates a new StorageContainerManager. Configuration will be
|
||||||
|
@ -336,6 +338,9 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
||||||
clientProtocolServer = new SCMClientProtocolServer(conf, this);
|
clientProtocolServer = new SCMClientProtocolServer(conf, this);
|
||||||
httpServer = new StorageContainerManagerHttpServer(conf);
|
httpServer = new StorageContainerManagerHttpServer(conf);
|
||||||
|
|
||||||
|
chillModeHandler = new ChillModeHandler(configuration,
|
||||||
|
clientProtocolServer, scmBlockManager, replicationStatus);
|
||||||
|
|
||||||
eventQueue.addHandler(SCMEvents.DATANODE_COMMAND, scmNodeManager);
|
eventQueue.addHandler(SCMEvents.DATANODE_COMMAND, scmNodeManager);
|
||||||
eventQueue.addHandler(SCMEvents.RETRIABLE_DATANODE_COMMAND, scmNodeManager);
|
eventQueue.addHandler(SCMEvents.RETRIABLE_DATANODE_COMMAND, scmNodeManager);
|
||||||
eventQueue.addHandler(SCMEvents.NODE_REPORT, nodeReportHandler);
|
eventQueue.addHandler(SCMEvents.NODE_REPORT, nodeReportHandler);
|
||||||
|
@ -350,22 +355,13 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
||||||
nonHealthyToHealthyNodeHandler);
|
nonHealthyToHealthyNodeHandler);
|
||||||
eventQueue.addHandler(SCMEvents.DEAD_NODE, deadNodeHandler);
|
eventQueue.addHandler(SCMEvents.DEAD_NODE, deadNodeHandler);
|
||||||
eventQueue.addHandler(SCMEvents.CMD_STATUS_REPORT, cmdStatusReportHandler);
|
eventQueue.addHandler(SCMEvents.CMD_STATUS_REPORT, cmdStatusReportHandler);
|
||||||
eventQueue.addHandler(SCMEvents.START_REPLICATION,
|
|
||||||
replicationStatus.getReplicationStatusListener());
|
|
||||||
eventQueue
|
eventQueue
|
||||||
.addHandler(SCMEvents.PENDING_DELETE_STATUS, pendingDeleteHandler);
|
.addHandler(SCMEvents.PENDING_DELETE_STATUS, pendingDeleteHandler);
|
||||||
eventQueue.addHandler(SCMEvents.DELETE_BLOCK_STATUS,
|
eventQueue.addHandler(SCMEvents.DELETE_BLOCK_STATUS,
|
||||||
(DeletedBlockLogImpl) scmBlockManager.getDeletedBlockLog());
|
(DeletedBlockLogImpl) scmBlockManager.getDeletedBlockLog());
|
||||||
eventQueue.addHandler(SCMEvents.PIPELINE_ACTIONS, pipelineActionHandler);
|
eventQueue.addHandler(SCMEvents.PIPELINE_ACTIONS, pipelineActionHandler);
|
||||||
eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS, clientProtocolServer);
|
|
||||||
eventQueue.addHandler(SCMEvents.PIPELINE_REPORT, pipelineReportHandler);
|
eventQueue.addHandler(SCMEvents.PIPELINE_REPORT, pipelineReportHandler);
|
||||||
|
eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS, chillModeHandler);
|
||||||
eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS,
|
|
||||||
replicationStatus.getChillModeStatusListener());
|
|
||||||
eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS,
|
|
||||||
(BlockManagerImpl) scmBlockManager);
|
|
||||||
eventQueue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
|
|
||||||
scmChillModeManager);
|
|
||||||
registerMXBean();
|
registerMXBean();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1081,6 +1077,11 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
||||||
return scmBlockManager;
|
return scmBlockManager;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public ChillModeHandler getChillModeHandler() {
|
||||||
|
return chillModeHandler;
|
||||||
|
}
|
||||||
|
|
||||||
public void checkAdminAccess(String remoteUser) throws IOException {
|
public void checkAdminAccess(String remoteUser) throws IOException {
|
||||||
if (remoteUser != null) {
|
if (remoteUser != null) {
|
||||||
if (!scmAdminUsernames.contains(remoteUser)) {
|
if (!scmAdminUsernames.contains(remoteUser)) {
|
||||||
|
|
|
@ -36,25 +36,9 @@
|
||||||
<td>Datanode Rpc port</td>
|
<td>Datanode Rpc port</td>
|
||||||
<td>{{$ctrl.overview.jmx.DatanodeRpcPort}}</td>
|
<td>{{$ctrl.overview.jmx.DatanodeRpcPort}}</td>
|
||||||
</tr>
|
</tr>
|
||||||
<tr>
|
|
||||||
<td>Block Manager: Open containers</td>
|
|
||||||
<td>{{$ctrl.blockmanagermetrics.OpenContainersNo}}</td>
|
|
||||||
</tr>
|
|
||||||
<tr>
|
|
||||||
<td>Node Manager: Minimum chill mode nodes</td>
|
|
||||||
<td>{{$ctrl.nodemanagermetrics.MinimumChillModeNodes}}</td>
|
|
||||||
</tr>
|
|
||||||
<tr>
|
|
||||||
<td>Node Manager: Out-of-node chill mode</td>
|
|
||||||
<td>{{$ctrl.nodemanagermetrics.OutOfNodeChillMode}}</td>
|
|
||||||
</tr>
|
|
||||||
<tr>
|
<tr>
|
||||||
<td>Node Manager: Chill mode status</td>
|
<td>Node Manager: Chill mode status</td>
|
||||||
<td>{{$ctrl.scmmetrics.InChillMode}}</td>
|
<td>{{$ctrl.scmmetrics.InChillMode}}</td>
|
||||||
</tr>
|
</tr>
|
||||||
<tr>
|
|
||||||
<td>Node Manager: Manual chill mode</td>
|
|
||||||
<td>{{$ctrl.nodemanagermetrics.InManualChillMode}}</td>
|
|
||||||
</tr>
|
|
||||||
</tbody>
|
</tbody>
|
||||||
</table>
|
</table>
|
||||||
|
|
|
@ -26,10 +26,6 @@
|
||||||
},
|
},
|
||||||
controller: function ($http) {
|
controller: function ($http) {
|
||||||
var ctrl = this;
|
var ctrl = this;
|
||||||
$http.get("jmx?qry=Hadoop:service=BlockManager,name=*")
|
|
||||||
.then(function (result) {
|
|
||||||
ctrl.blockmanagermetrics = result.data.beans[0];
|
|
||||||
});
|
|
||||||
$http.get("jmx?qry=Hadoop:service=SCMNodeManager,name=SCMNodeManagerInfo")
|
$http.get("jmx?qry=Hadoop:service=SCMNodeManager,name=SCMNodeManagerInfo")
|
||||||
.then(function (result) {
|
.then(function (result) {
|
||||||
ctrl.nodemanagermetrics = result.data.beans[0];
|
ctrl.nodemanagermetrics = result.data.beans[0];
|
||||||
|
|
|
@ -25,11 +25,13 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||||
import org.apache.hadoop.hdds.scm.TestUtils;
|
import org.apache.hadoop.hdds.scm.TestUtils;
|
||||||
|
import org.apache.hadoop.hdds.scm.chillmode.SCMChillModeManager.ChillModeStatus;
|
||||||
import org.apache.hadoop.hdds.scm.container.CloseContainerEventHandler;
|
import org.apache.hadoop.hdds.scm.container.CloseContainerEventHandler;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
import org.apache.hadoop.hdds.scm.container.MockNodeManager;
|
import org.apache.hadoop.hdds.scm.container.MockNodeManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
|
import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
|
||||||
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
|
||||||
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
||||||
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
||||||
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
|
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
|
||||||
|
@ -70,6 +72,7 @@ public class TestBlockManager implements EventHandler<Boolean> {
|
||||||
private static EventQueue eventQueue;
|
private static EventQueue eventQueue;
|
||||||
private int numContainerPerOwnerInPipeline;
|
private int numContainerPerOwnerInPipeline;
|
||||||
private OzoneConfiguration conf;
|
private OzoneConfiguration conf;
|
||||||
|
private ChillModeStatus chillModeStatus = new ChillModeStatus(false);
|
||||||
|
|
||||||
@Rule
|
@Rule
|
||||||
public ExpectedException thrown = ExpectedException.none();
|
public ExpectedException thrown = ExpectedException.none();
|
||||||
|
@ -100,7 +103,7 @@ public class TestBlockManager implements EventHandler<Boolean> {
|
||||||
|
|
||||||
eventQueue = new EventQueue();
|
eventQueue = new EventQueue();
|
||||||
eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS,
|
eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS,
|
||||||
(BlockManagerImpl) scm.getScmBlockManager());
|
scm.getChillModeHandler());
|
||||||
eventQueue.addHandler(SCMEvents.START_REPLICATION, this);
|
eventQueue.addHandler(SCMEvents.START_REPLICATION, this);
|
||||||
CloseContainerEventHandler closeContainerHandler =
|
CloseContainerEventHandler closeContainerHandler =
|
||||||
new CloseContainerEventHandler(pipelineManager, mapping);
|
new CloseContainerEventHandler(pipelineManager, mapping);
|
||||||
|
@ -122,31 +125,32 @@ public class TestBlockManager implements EventHandler<Boolean> {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAllocateBlock() throws Exception {
|
public void testAllocateBlock() throws Exception {
|
||||||
eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, false);
|
eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, chillModeStatus);
|
||||||
GenericTestUtils.waitFor(() -> {
|
GenericTestUtils.waitFor(() -> {
|
||||||
return !blockManager.isScmInChillMode();
|
return !blockManager.isScmInChillMode();
|
||||||
}, 10, 1000 * 5);
|
}, 10, 1000 * 5);
|
||||||
AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
|
AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
|
||||||
type, factor, containerOwner);
|
type, factor, containerOwner, new ExcludeList());
|
||||||
Assert.assertNotNull(block);
|
Assert.assertNotNull(block);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAllocateOversizedBlock() throws Exception {
|
public void testAllocateOversizedBlock() throws Exception {
|
||||||
eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, false);
|
eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, chillModeStatus);
|
||||||
GenericTestUtils.waitFor(() -> {
|
GenericTestUtils.waitFor(() -> {
|
||||||
return !blockManager.isScmInChillMode();
|
return !blockManager.isScmInChillMode();
|
||||||
}, 10, 1000 * 5);
|
}, 10, 1000 * 5);
|
||||||
long size = 6 * GB;
|
long size = 6 * GB;
|
||||||
thrown.expectMessage("Unsupported block size");
|
thrown.expectMessage("Unsupported block size");
|
||||||
AllocatedBlock block = blockManager.allocateBlock(size,
|
AllocatedBlock block = blockManager.allocateBlock(size,
|
||||||
type, factor, containerOwner);
|
type, factor, containerOwner, new ExcludeList());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAllocateBlockFailureInChillMode() throws Exception {
|
public void testAllocateBlockFailureInChillMode() throws Exception {
|
||||||
eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, true);
|
eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS,
|
||||||
|
new ChillModeStatus(true));
|
||||||
GenericTestUtils.waitFor(() -> {
|
GenericTestUtils.waitFor(() -> {
|
||||||
return blockManager.isScmInChillMode();
|
return blockManager.isScmInChillMode();
|
||||||
}, 10, 1000 * 5);
|
}, 10, 1000 * 5);
|
||||||
|
@ -154,24 +158,24 @@ public class TestBlockManager implements EventHandler<Boolean> {
|
||||||
thrown.expectMessage("ChillModePrecheck failed for "
|
thrown.expectMessage("ChillModePrecheck failed for "
|
||||||
+ "allocateBlock");
|
+ "allocateBlock");
|
||||||
blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
|
blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
|
||||||
type, factor, containerOwner);
|
type, factor, containerOwner, new ExcludeList());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAllocateBlockSucInChillMode() throws Exception {
|
public void testAllocateBlockSucInChillMode() throws Exception {
|
||||||
// Test2: Exit chill mode and then try allocateBock again.
|
// Test2: Exit chill mode and then try allocateBock again.
|
||||||
eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, false);
|
eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, chillModeStatus);
|
||||||
GenericTestUtils.waitFor(() -> {
|
GenericTestUtils.waitFor(() -> {
|
||||||
return !blockManager.isScmInChillMode();
|
return !blockManager.isScmInChillMode();
|
||||||
}, 10, 1000 * 5);
|
}, 10, 1000 * 5);
|
||||||
Assert.assertNotNull(blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
|
Assert.assertNotNull(blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
|
||||||
type, factor, containerOwner));
|
type, factor, containerOwner, new ExcludeList()));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 10000)
|
@Test(timeout = 10000)
|
||||||
public void testMultipleBlockAllocation()
|
public void testMultipleBlockAllocation()
|
||||||
throws IOException, TimeoutException, InterruptedException {
|
throws IOException, TimeoutException, InterruptedException {
|
||||||
eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, false);
|
eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, chillModeStatus);
|
||||||
GenericTestUtils
|
GenericTestUtils
|
||||||
.waitFor(() -> !blockManager.isScmInChillMode(), 10, 1000 * 5);
|
.waitFor(() -> !blockManager.isScmInChillMode(), 10, 1000 * 5);
|
||||||
|
|
||||||
|
@ -179,12 +183,14 @@ public class TestBlockManager implements EventHandler<Boolean> {
|
||||||
pipelineManager.createPipeline(type, factor);
|
pipelineManager.createPipeline(type, factor);
|
||||||
|
|
||||||
AllocatedBlock allocatedBlock = blockManager
|
AllocatedBlock allocatedBlock = blockManager
|
||||||
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, containerOwner);
|
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, containerOwner,
|
||||||
|
new ExcludeList());
|
||||||
// block should be allocated in different pipelines
|
// block should be allocated in different pipelines
|
||||||
GenericTestUtils.waitFor(() -> {
|
GenericTestUtils.waitFor(() -> {
|
||||||
try {
|
try {
|
||||||
AllocatedBlock block = blockManager
|
AllocatedBlock block = blockManager
|
||||||
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, containerOwner);
|
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, containerOwner,
|
||||||
|
new ExcludeList());
|
||||||
return !block.getPipeline().getId()
|
return !block.getPipeline().getId()
|
||||||
.equals(allocatedBlock.getPipeline().getId());
|
.equals(allocatedBlock.getPipeline().getId());
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
|
@ -211,7 +217,7 @@ public class TestBlockManager implements EventHandler<Boolean> {
|
||||||
@Test(timeout = 10000)
|
@Test(timeout = 10000)
|
||||||
public void testMultipleBlockAllocationWithClosedContainer()
|
public void testMultipleBlockAllocationWithClosedContainer()
|
||||||
throws IOException, TimeoutException, InterruptedException {
|
throws IOException, TimeoutException, InterruptedException {
|
||||||
eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, false);
|
eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, chillModeStatus);
|
||||||
GenericTestUtils
|
GenericTestUtils
|
||||||
.waitFor(() -> !blockManager.isScmInChillMode(), 10, 1000 * 5);
|
.waitFor(() -> !blockManager.isScmInChillMode(), 10, 1000 * 5);
|
||||||
|
|
||||||
|
@ -227,7 +233,8 @@ public class TestBlockManager implements EventHandler<Boolean> {
|
||||||
GenericTestUtils.waitFor(() -> {
|
GenericTestUtils.waitFor(() -> {
|
||||||
try {
|
try {
|
||||||
blockManager
|
blockManager
|
||||||
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, containerOwner);
|
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, containerOwner,
|
||||||
|
new ExcludeList());
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
}
|
}
|
||||||
return verifyNumberOfContainersInPipelines(
|
return verifyNumberOfContainersInPipelines(
|
||||||
|
@ -250,7 +257,8 @@ public class TestBlockManager implements EventHandler<Boolean> {
|
||||||
GenericTestUtils.waitFor(() -> {
|
GenericTestUtils.waitFor(() -> {
|
||||||
try {
|
try {
|
||||||
blockManager
|
blockManager
|
||||||
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, containerOwner);
|
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, containerOwner,
|
||||||
|
new ExcludeList());
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
}
|
}
|
||||||
return verifyNumberOfContainersInPipelines(
|
return verifyNumberOfContainersInPipelines(
|
||||||
|
@ -261,7 +269,7 @@ public class TestBlockManager implements EventHandler<Boolean> {
|
||||||
@Test(timeout = 10000)
|
@Test(timeout = 10000)
|
||||||
public void testBlockAllocationWithNoAvailablePipelines()
|
public void testBlockAllocationWithNoAvailablePipelines()
|
||||||
throws IOException, TimeoutException, InterruptedException {
|
throws IOException, TimeoutException, InterruptedException {
|
||||||
eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, false);
|
eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, chillModeStatus);
|
||||||
GenericTestUtils
|
GenericTestUtils
|
||||||
.waitFor(() -> !blockManager.isScmInChillMode(), 10, 1000 * 5);
|
.waitFor(() -> !blockManager.isScmInChillMode(), 10, 1000 * 5);
|
||||||
|
|
||||||
|
@ -271,7 +279,8 @@ public class TestBlockManager implements EventHandler<Boolean> {
|
||||||
}
|
}
|
||||||
Assert.assertEquals(0, pipelineManager.getPipelines(type, factor).size());
|
Assert.assertEquals(0, pipelineManager.getPipelines(type, factor).size());
|
||||||
Assert.assertNotNull(blockManager
|
Assert.assertNotNull(blockManager
|
||||||
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, containerOwner));
|
.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, containerOwner,
|
||||||
|
new ExcludeList()));
|
||||||
Assert.assertEquals(1, pipelineManager.getPipelines(type, factor).size());
|
Assert.assertEquals(1, pipelineManager.getPipelines(type, factor).size());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,103 @@
|
||||||
|
/**
|
||||||
|
* 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.hdds.scm.chillmode;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hdds.HddsConfigKeys;
|
||||||
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
|
import org.apache.hadoop.hdds.scm.block.BlockManager;
|
||||||
|
import org.apache.hadoop.hdds.scm.block.BlockManagerImpl;
|
||||||
|
import org.apache.hadoop.hdds.scm.container.replication.ReplicationActivityStatus;
|
||||||
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
||||||
|
import org.apache.hadoop.hdds.scm.server.SCMClientProtocolServer;
|
||||||
|
import org.apache.hadoop.hdds.server.events.EventQueue;
|
||||||
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.mockito.Mockito;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests ChillModeHandler behavior.
|
||||||
|
*/
|
||||||
|
public class TestChillModeHandler {
|
||||||
|
|
||||||
|
|
||||||
|
private OzoneConfiguration configuration;
|
||||||
|
private SCMClientProtocolServer scmClientProtocolServer;
|
||||||
|
private ReplicationActivityStatus replicationActivityStatus;
|
||||||
|
private BlockManager blockManager;
|
||||||
|
private ChillModeHandler chillModeHandler;
|
||||||
|
private EventQueue eventQueue;
|
||||||
|
private SCMChillModeManager.ChillModeStatus chillModeStatus;
|
||||||
|
|
||||||
|
public void setup(boolean enabled) {
|
||||||
|
configuration = new OzoneConfiguration();
|
||||||
|
configuration.setBoolean(HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED,
|
||||||
|
enabled);
|
||||||
|
configuration.set(HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_CHILL_MODE_EXIT,
|
||||||
|
"3s");
|
||||||
|
scmClientProtocolServer =
|
||||||
|
Mockito.mock(SCMClientProtocolServer.class);
|
||||||
|
replicationActivityStatus =
|
||||||
|
new ReplicationActivityStatus();
|
||||||
|
blockManager = Mockito.mock(BlockManagerImpl.class);
|
||||||
|
chillModeHandler =
|
||||||
|
new ChillModeHandler(configuration, scmClientProtocolServer,
|
||||||
|
blockManager, replicationActivityStatus);
|
||||||
|
|
||||||
|
|
||||||
|
eventQueue = new EventQueue();
|
||||||
|
eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS, chillModeHandler);
|
||||||
|
chillModeStatus = new SCMChillModeManager.ChillModeStatus(false);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testChillModeHandlerWithChillModeEnabled() throws Exception {
|
||||||
|
setup(true);
|
||||||
|
|
||||||
|
Assert.assertTrue(chillModeHandler.getChillModeStatus());
|
||||||
|
|
||||||
|
eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, chillModeStatus);
|
||||||
|
|
||||||
|
GenericTestUtils.waitFor(() -> !chillModeHandler.getChillModeStatus(),
|
||||||
|
1000, 5000);
|
||||||
|
|
||||||
|
Assert.assertFalse(scmClientProtocolServer.getChillModeStatus());
|
||||||
|
Assert.assertFalse(((BlockManagerImpl) blockManager).isScmInChillMode());
|
||||||
|
GenericTestUtils.waitFor(() ->
|
||||||
|
replicationActivityStatus.isReplicationEnabled(), 1000, 5000);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testChillModeHandlerWithChillModeDisbaled() throws Exception{
|
||||||
|
|
||||||
|
setup(false);
|
||||||
|
|
||||||
|
Assert.assertFalse(chillModeHandler.getChillModeStatus());
|
||||||
|
|
||||||
|
eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, chillModeStatus);
|
||||||
|
|
||||||
|
Assert.assertFalse(chillModeHandler.getChillModeStatus());
|
||||||
|
Assert.assertFalse(scmClientProtocolServer.getChillModeStatus());
|
||||||
|
Assert.assertFalse(((BlockManagerImpl) blockManager).isScmInChillMode());
|
||||||
|
GenericTestUtils.waitFor(() ->
|
||||||
|
replicationActivityStatus.isReplicationEnabled(), 1000, 5000);
|
||||||
|
}
|
||||||
|
}
|
|
@ -31,7 +31,8 @@ import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.MockNodeManager;
|
import org.apache.hadoop.hdds.scm.container.MockNodeManager;
|
||||||
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
||||||
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
||||||
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
|
import org.apache.hadoop.hdds.scm.pipeline.PipelineProvider;
|
||||||
|
import org.apache.hadoop.hdds.scm.pipeline.MockRatisPipelineProvider;
|
||||||
import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager;
|
import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager;
|
||||||
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher;
|
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher;
|
||||||
import org.apache.hadoop.hdds.server.events.EventQueue;
|
import org.apache.hadoop.hdds.server.events.EventQueue;
|
||||||
|
@ -69,8 +70,13 @@ public class TestHealthyPipelineChillModeRule {
|
||||||
HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK, true);
|
HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK, true);
|
||||||
|
|
||||||
|
|
||||||
PipelineManager pipelineManager = new SCMPipelineManager(config,
|
SCMPipelineManager pipelineManager = new SCMPipelineManager(config,
|
||||||
nodeManager, eventQueue);
|
nodeManager, eventQueue);
|
||||||
|
PipelineProvider mockRatisProvider =
|
||||||
|
new MockRatisPipelineProvider(nodeManager,
|
||||||
|
pipelineManager.getStateManager(), config);
|
||||||
|
pipelineManager.setPipelineProvider(HddsProtos.ReplicationType.RATIS,
|
||||||
|
mockRatisProvider);
|
||||||
SCMChillModeManager scmChillModeManager = new SCMChillModeManager(
|
SCMChillModeManager scmChillModeManager = new SCMChillModeManager(
|
||||||
config, containers, pipelineManager, eventQueue);
|
config, containers, pipelineManager, eventQueue);
|
||||||
|
|
||||||
|
@ -109,9 +115,15 @@ public class TestHealthyPipelineChillModeRule {
|
||||||
HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK, true);
|
HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK, true);
|
||||||
|
|
||||||
|
|
||||||
PipelineManager pipelineManager = new SCMPipelineManager(config,
|
SCMPipelineManager pipelineManager = new SCMPipelineManager(config,
|
||||||
nodeManager, eventQueue);
|
nodeManager, eventQueue);
|
||||||
|
|
||||||
|
PipelineProvider mockRatisProvider =
|
||||||
|
new MockRatisPipelineProvider(nodeManager,
|
||||||
|
pipelineManager.getStateManager(), config);
|
||||||
|
pipelineManager.setPipelineProvider(HddsProtos.ReplicationType.RATIS,
|
||||||
|
mockRatisProvider);
|
||||||
|
|
||||||
// Create 3 pipelines
|
// Create 3 pipelines
|
||||||
Pipeline pipeline1 =
|
Pipeline pipeline1 =
|
||||||
pipelineManager.createPipeline(HddsProtos.ReplicationType.RATIS,
|
pipelineManager.createPipeline(HddsProtos.ReplicationType.RATIS,
|
||||||
|
@ -178,8 +190,13 @@ public class TestHealthyPipelineChillModeRule {
|
||||||
HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK, true);
|
HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK, true);
|
||||||
|
|
||||||
|
|
||||||
PipelineManager pipelineManager = new SCMPipelineManager(config,
|
SCMPipelineManager pipelineManager = new SCMPipelineManager(config,
|
||||||
nodeManager, eventQueue);
|
nodeManager, eventQueue);
|
||||||
|
PipelineProvider mockRatisProvider =
|
||||||
|
new MockRatisPipelineProvider(nodeManager,
|
||||||
|
pipelineManager.getStateManager(), config);
|
||||||
|
pipelineManager.setPipelineProvider(HddsProtos.ReplicationType.RATIS,
|
||||||
|
mockRatisProvider);
|
||||||
|
|
||||||
// Create 3 pipelines
|
// Create 3 pipelines
|
||||||
Pipeline pipeline1 =
|
Pipeline pipeline1 =
|
||||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue