Merge trunk into HDDS-48
This commit is contained in:
commit
c275a9a6a0
4
.gitignore
vendored
4
.gitignore
vendored
@ -50,6 +50,10 @@ patchprocess/
|
||||
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/package-lock.json
|
||||
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/yarn-error.log
|
||||
|
||||
# Ignore files generated by HDDS acceptance tests.
|
||||
hadoop-ozone/acceptance-test/docker-compose.log
|
||||
hadoop-ozone/acceptance-test/junit-results.xml
|
||||
|
||||
#robotframework outputs
|
||||
log.html
|
||||
output.xml
|
||||
|
@ -148,7 +148,7 @@ run copy "${ROOT}/hadoop-ozone/tools/target/hadoop-ozone-tools-${HDDS_VERSION}"
|
||||
mkdir -p "./share/hadoop/ozonefs"
|
||||
cp "${ROOT}/hadoop-ozone/ozonefs/target/hadoop-ozone-filesystem-${HDDS_VERSION}.jar" "./share/hadoop/ozonefs/hadoop-ozone-filesystem.jar"
|
||||
# Optional documentation, could be missing
|
||||
cp -r "${ROOT}/hadoop-ozone/docs/target/classes/webapps/docs" ./share/hadoop/ozone/webapps/ksm/
|
||||
cp -r "${ROOT}/hadoop-ozone/docs/target/classes/webapps/docs" ./share/hadoop/ozone/webapps/ozoneManager/
|
||||
cp -r "${ROOT}/hadoop-ozone/docs/target/classes/webapps/docs" ./share/hadoop/hdds/webapps/scm/
|
||||
|
||||
|
||||
|
@ -38,7 +38,11 @@ public static LanguageVersion languageVersion() {
|
||||
public static boolean start(RootDoc root) {
|
||||
System.out.println(
|
||||
ExcludePrivateAnnotationsStandardDoclet.class.getSimpleName());
|
||||
return Standard.start(RootDocProcessor.process(root));
|
||||
RootDoc excludedDoc = RootDocProcessor.process(root);
|
||||
if (excludedDoc.specifiedPackages().length == 0) {
|
||||
return true;
|
||||
}
|
||||
return Standard.start(excludedDoc);
|
||||
}
|
||||
|
||||
public static int optionLength(String option) {
|
||||
|
@ -404,13 +404,13 @@ esac
|
||||
# export HDFS_DFSROUTER_OPTS=""
|
||||
|
||||
###
|
||||
# HDFS Key Space Manager specific parameters
|
||||
# Ozone Manager specific parameters
|
||||
###
|
||||
# Specify the JVM options to be used when starting the HDFS Key Space Manager.
|
||||
# Specify the JVM options to be used when starting the Ozone Manager.
|
||||
# These options will be appended to the options specified as HADOOP_OPTS
|
||||
# and therefore may override any similar flags set in HADOOP_OPTS
|
||||
#
|
||||
# export HDFS_KSM_OPTS=""
|
||||
# export HDFS_OM_OPTS=""
|
||||
|
||||
###
|
||||
# HDFS StorageContainerManager specific parameters
|
||||
|
@ -41,6 +41,7 @@
|
||||
import java.lang.ref.WeakReference;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.JarURLConnection;
|
||||
import java.net.MalformedURLException;
|
||||
import java.net.URL;
|
||||
import java.net.URLConnection;
|
||||
import java.util.ArrayList;
|
||||
@ -2981,187 +2982,11 @@ private Resource loadResource(Properties properties,
|
||||
if(returnCachedProperties) {
|
||||
toAddTo = new Properties();
|
||||
}
|
||||
DeprecationContext deprecations = deprecationContext.get();
|
||||
|
||||
StringBuilder token = new StringBuilder();
|
||||
String confName = null;
|
||||
String confValue = null;
|
||||
String confInclude = null;
|
||||
String confTag = null;
|
||||
boolean confFinal = false;
|
||||
boolean fallbackAllowed = false;
|
||||
boolean fallbackEntered = false;
|
||||
boolean parseToken = false;
|
||||
LinkedList<String> confSource = new LinkedList<String>();
|
||||
|
||||
while (reader.hasNext()) {
|
||||
switch (reader.next()) {
|
||||
case XMLStreamConstants.START_ELEMENT:
|
||||
switch (reader.getLocalName()) {
|
||||
case "property":
|
||||
confName = null;
|
||||
confValue = null;
|
||||
confFinal = false;
|
||||
confTag = null;
|
||||
confSource.clear();
|
||||
|
||||
// First test for short format configuration
|
||||
int attrCount = reader.getAttributeCount();
|
||||
for (int i = 0; i < attrCount; i++) {
|
||||
String propertyAttr = reader.getAttributeLocalName(i);
|
||||
if ("name".equals(propertyAttr)) {
|
||||
confName = StringInterner.weakIntern(
|
||||
reader.getAttributeValue(i));
|
||||
} else if ("value".equals(propertyAttr)) {
|
||||
confValue = StringInterner.weakIntern(
|
||||
reader.getAttributeValue(i));
|
||||
} else if ("final".equals(propertyAttr)) {
|
||||
confFinal = "true".equals(reader.getAttributeValue(i));
|
||||
} else if ("source".equals(propertyAttr)) {
|
||||
confSource.add(StringInterner.weakIntern(
|
||||
reader.getAttributeValue(i)));
|
||||
} else if ("tag".equals(propertyAttr)) {
|
||||
confTag = StringInterner
|
||||
.weakIntern(reader.getAttributeValue(i));
|
||||
}
|
||||
}
|
||||
break;
|
||||
case "name":
|
||||
case "value":
|
||||
case "final":
|
||||
case "source":
|
||||
case "tag":
|
||||
parseToken = true;
|
||||
token.setLength(0);
|
||||
break;
|
||||
case "include":
|
||||
// Determine href for xi:include
|
||||
confInclude = null;
|
||||
attrCount = reader.getAttributeCount();
|
||||
for (int i = 0; i < attrCount; i++) {
|
||||
String attrName = reader.getAttributeLocalName(i);
|
||||
if ("href".equals(attrName)) {
|
||||
confInclude = reader.getAttributeValue(i);
|
||||
}
|
||||
}
|
||||
if (confInclude == null) {
|
||||
break;
|
||||
}
|
||||
if (isRestricted) {
|
||||
throw new RuntimeException("Error parsing resource " + wrapper
|
||||
+ ": XInclude is not supported for restricted resources");
|
||||
}
|
||||
// Determine if the included resource is a classpath resource
|
||||
// otherwise fallback to a file resource
|
||||
// xi:include are treated as inline and retain current source
|
||||
URL include = getResource(confInclude);
|
||||
if (include != null) {
|
||||
Resource classpathResource = new Resource(include, name,
|
||||
wrapper.isParserRestricted());
|
||||
loadResource(properties, classpathResource, quiet);
|
||||
} else {
|
||||
URL url;
|
||||
try {
|
||||
url = new URL(confInclude);
|
||||
url.openConnection().connect();
|
||||
} catch (IOException ioe) {
|
||||
File href = new File(confInclude);
|
||||
if (!href.isAbsolute()) {
|
||||
// Included resources are relative to the current resource
|
||||
File baseFile = new File(name).getParentFile();
|
||||
href = new File(baseFile, href.getPath());
|
||||
}
|
||||
if (!href.exists()) {
|
||||
// Resource errors are non-fatal iff there is 1 xi:fallback
|
||||
fallbackAllowed = true;
|
||||
break;
|
||||
}
|
||||
url = href.toURI().toURL();
|
||||
}
|
||||
Resource uriResource = new Resource(url, name,
|
||||
wrapper.isParserRestricted());
|
||||
loadResource(properties, uriResource, quiet);
|
||||
}
|
||||
break;
|
||||
case "fallback":
|
||||
fallbackEntered = true;
|
||||
break;
|
||||
case "configuration":
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
break;
|
||||
|
||||
case XMLStreamConstants.CHARACTERS:
|
||||
if (parseToken) {
|
||||
char[] text = reader.getTextCharacters();
|
||||
token.append(text, reader.getTextStart(), reader.getTextLength());
|
||||
}
|
||||
break;
|
||||
|
||||
case XMLStreamConstants.END_ELEMENT:
|
||||
switch (reader.getLocalName()) {
|
||||
case "name":
|
||||
if (token.length() > 0) {
|
||||
confName = StringInterner.weakIntern(token.toString().trim());
|
||||
}
|
||||
break;
|
||||
case "value":
|
||||
if (token.length() > 0) {
|
||||
confValue = StringInterner.weakIntern(token.toString());
|
||||
}
|
||||
break;
|
||||
case "final":
|
||||
confFinal = "true".equals(token.toString());
|
||||
break;
|
||||
case "source":
|
||||
confSource.add(StringInterner.weakIntern(token.toString()));
|
||||
break;
|
||||
case "tag":
|
||||
if (token.length() > 0) {
|
||||
confTag = StringInterner.weakIntern(token.toString());
|
||||
}
|
||||
break;
|
||||
case "include":
|
||||
if (fallbackAllowed && !fallbackEntered) {
|
||||
throw new IOException("Fetch fail on include for '"
|
||||
+ confInclude + "' with no fallback while loading '"
|
||||
+ name + "'");
|
||||
}
|
||||
fallbackAllowed = false;
|
||||
fallbackEntered = false;
|
||||
break;
|
||||
case "property":
|
||||
if (confName == null || (!fallbackAllowed && fallbackEntered)) {
|
||||
break;
|
||||
}
|
||||
confSource.add(name);
|
||||
// Read tags and put them in propertyTagsMap
|
||||
if (confTag != null) {
|
||||
readTagFromConfig(confTag, confName, confValue, confSource);
|
||||
}
|
||||
|
||||
DeprecatedKeyInfo keyInfo =
|
||||
deprecations.getDeprecatedKeyMap().get(confName);
|
||||
if (keyInfo != null) {
|
||||
keyInfo.clearAccessed();
|
||||
for (String key : keyInfo.newKeys) {
|
||||
// update new keys with deprecated key's value
|
||||
loadProperty(toAddTo, name, key, confValue, confFinal,
|
||||
confSource.toArray(new String[confSource.size()]));
|
||||
}
|
||||
} else {
|
||||
loadProperty(toAddTo, name, confName, confValue, confFinal,
|
||||
confSource.toArray(new String[confSource.size()]));
|
||||
}
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
default:
|
||||
break;
|
||||
}
|
||||
List<ParsedItem> items = new Parser(reader, wrapper, quiet).parse();
|
||||
for (ParsedItem item : items) {
|
||||
loadProperty(toAddTo, item.name, item.key, item.value,
|
||||
item.isFinal, item.sources);
|
||||
}
|
||||
reader.close();
|
||||
|
||||
@ -3179,6 +3004,275 @@ private Resource loadResource(Properties properties,
|
||||
}
|
||||
}
|
||||
|
||||
private static class ParsedItem {
|
||||
String name;
|
||||
String key;
|
||||
String value;
|
||||
boolean isFinal;
|
||||
String[] sources;
|
||||
|
||||
ParsedItem(String name, String key, String value,
|
||||
boolean isFinal, String[] sources) {
|
||||
this.name = name;
|
||||
this.key = key;
|
||||
this.value = value;
|
||||
this.isFinal = isFinal;
|
||||
this.sources = sources;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Parser to consume SAX stream of XML elements from a Configuration.
|
||||
*/
|
||||
private class Parser {
|
||||
private final XMLStreamReader2 reader;
|
||||
private final Resource wrapper;
|
||||
private final String name;
|
||||
private final String[] nameSingletonArray;
|
||||
private final boolean isRestricted;
|
||||
private final boolean quiet;
|
||||
|
||||
DeprecationContext deprecations = deprecationContext.get();
|
||||
|
||||
private StringBuilder token = new StringBuilder();
|
||||
private String confName = null;
|
||||
private String confValue = null;
|
||||
private String confInclude = null;
|
||||
private String confTag = null;
|
||||
private boolean confFinal = false;
|
||||
private boolean fallbackAllowed = false;
|
||||
private boolean fallbackEntered = false;
|
||||
private boolean parseToken = false;
|
||||
private List<String> confSource = new ArrayList<>();
|
||||
private List<ParsedItem> results = new ArrayList<>();
|
||||
|
||||
Parser(XMLStreamReader2 reader,
|
||||
Resource wrapper,
|
||||
boolean quiet) {
|
||||
this.reader = reader;
|
||||
this.wrapper = wrapper;
|
||||
this.name = wrapper.getName();
|
||||
this.nameSingletonArray = new String[]{ name };
|
||||
this.isRestricted = wrapper.isParserRestricted();
|
||||
this.quiet = quiet;
|
||||
|
||||
}
|
||||
|
||||
List<ParsedItem> parse() throws IOException, XMLStreamException {
|
||||
while (reader.hasNext()) {
|
||||
parseNext();
|
||||
}
|
||||
return results;
|
||||
}
|
||||
|
||||
private void handleStartElement() throws MalformedURLException {
|
||||
switch (reader.getLocalName()) {
|
||||
case "property":
|
||||
handleStartProperty();
|
||||
break;
|
||||
|
||||
case "name":
|
||||
case "value":
|
||||
case "final":
|
||||
case "source":
|
||||
case "tag":
|
||||
parseToken = true;
|
||||
token.setLength(0);
|
||||
break;
|
||||
case "include":
|
||||
handleInclude();
|
||||
break;
|
||||
case "fallback":
|
||||
fallbackEntered = true;
|
||||
break;
|
||||
case "configuration":
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
private void handleStartProperty() {
|
||||
confName = null;
|
||||
confValue = null;
|
||||
confFinal = false;
|
||||
confTag = null;
|
||||
confSource.clear();
|
||||
|
||||
// First test for short format configuration
|
||||
int attrCount = reader.getAttributeCount();
|
||||
for (int i = 0; i < attrCount; i++) {
|
||||
String propertyAttr = reader.getAttributeLocalName(i);
|
||||
if ("name".equals(propertyAttr)) {
|
||||
confName = StringInterner.weakIntern(
|
||||
reader.getAttributeValue(i));
|
||||
} else if ("value".equals(propertyAttr)) {
|
||||
confValue = StringInterner.weakIntern(
|
||||
reader.getAttributeValue(i));
|
||||
} else if ("final".equals(propertyAttr)) {
|
||||
confFinal = "true".equals(reader.getAttributeValue(i));
|
||||
} else if ("source".equals(propertyAttr)) {
|
||||
confSource.add(StringInterner.weakIntern(
|
||||
reader.getAttributeValue(i)));
|
||||
} else if ("tag".equals(propertyAttr)) {
|
||||
confTag = StringInterner
|
||||
.weakIntern(reader.getAttributeValue(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void handleInclude() throws MalformedURLException {
|
||||
// Determine href for xi:include
|
||||
confInclude = null;
|
||||
int attrCount = reader.getAttributeCount();
|
||||
for (int i = 0; i < attrCount; i++) {
|
||||
String attrName = reader.getAttributeLocalName(i);
|
||||
if ("href".equals(attrName)) {
|
||||
confInclude = reader.getAttributeValue(i);
|
||||
}
|
||||
}
|
||||
if (confInclude == null) {
|
||||
return;
|
||||
}
|
||||
if (isRestricted) {
|
||||
throw new RuntimeException("Error parsing resource " + wrapper
|
||||
+ ": XInclude is not supported for restricted resources");
|
||||
}
|
||||
// Determine if the included resource is a classpath resource
|
||||
// otherwise fallback to a file resource
|
||||
// xi:include are treated as inline and retain current source
|
||||
URL include = getResource(confInclude);
|
||||
if (include != null) {
|
||||
Resource classpathResource = new Resource(include, name,
|
||||
wrapper.isParserRestricted());
|
||||
// This is only called recursively while the lock is already held
|
||||
// by this thread, but synchronizing avoids a findbugs warning.
|
||||
synchronized (Configuration.this) {
|
||||
loadResource(properties, classpathResource, quiet);
|
||||
}
|
||||
} else {
|
||||
URL url;
|
||||
try {
|
||||
url = new URL(confInclude);
|
||||
url.openConnection().connect();
|
||||
} catch (IOException ioe) {
|
||||
File href = new File(confInclude);
|
||||
if (!href.isAbsolute()) {
|
||||
// Included resources are relative to the current resource
|
||||
File baseFile = new File(name).getParentFile();
|
||||
href = new File(baseFile, href.getPath());
|
||||
}
|
||||
if (!href.exists()) {
|
||||
// Resource errors are non-fatal iff there is 1 xi:fallback
|
||||
fallbackAllowed = true;
|
||||
return;
|
||||
}
|
||||
url = href.toURI().toURL();
|
||||
}
|
||||
Resource uriResource = new Resource(url, name,
|
||||
wrapper.isParserRestricted());
|
||||
// This is only called recursively while the lock is already held
|
||||
// by this thread, but synchronizing avoids a findbugs warning.
|
||||
synchronized (Configuration.this) {
|
||||
loadResource(properties, uriResource, quiet);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void handleEndElement() throws IOException {
|
||||
String tokenStr = token.toString();
|
||||
switch (reader.getLocalName()) {
|
||||
case "name":
|
||||
if (token.length() > 0) {
|
||||
confName = StringInterner.weakIntern(tokenStr.trim());
|
||||
}
|
||||
break;
|
||||
case "value":
|
||||
if (token.length() > 0) {
|
||||
confValue = StringInterner.weakIntern(tokenStr);
|
||||
}
|
||||
break;
|
||||
case "final":
|
||||
confFinal = "true".equals(tokenStr);
|
||||
break;
|
||||
case "source":
|
||||
confSource.add(StringInterner.weakIntern(tokenStr));
|
||||
break;
|
||||
case "tag":
|
||||
if (token.length() > 0) {
|
||||
confTag = StringInterner.weakIntern(tokenStr);
|
||||
}
|
||||
break;
|
||||
case "include":
|
||||
if (fallbackAllowed && !fallbackEntered) {
|
||||
throw new IOException("Fetch fail on include for '"
|
||||
+ confInclude + "' with no fallback while loading '"
|
||||
+ name + "'");
|
||||
}
|
||||
fallbackAllowed = false;
|
||||
fallbackEntered = false;
|
||||
break;
|
||||
case "property":
|
||||
handleEndProperty();
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
void handleEndProperty() {
|
||||
if (confName == null || (!fallbackAllowed && fallbackEntered)) {
|
||||
return;
|
||||
}
|
||||
String[] confSourceArray;
|
||||
if (confSource.isEmpty()) {
|
||||
confSourceArray = nameSingletonArray;
|
||||
} else {
|
||||
confSource.add(name);
|
||||
confSourceArray = confSource.toArray(new String[confSource.size()]);
|
||||
}
|
||||
|
||||
// Read tags and put them in propertyTagsMap
|
||||
if (confTag != null) {
|
||||
readTagFromConfig(confTag, confName, confValue, confSourceArray);
|
||||
}
|
||||
|
||||
DeprecatedKeyInfo keyInfo =
|
||||
deprecations.getDeprecatedKeyMap().get(confName);
|
||||
|
||||
if (keyInfo != null) {
|
||||
keyInfo.clearAccessed();
|
||||
for (String key : keyInfo.newKeys) {
|
||||
// update new keys with deprecated key's value
|
||||
results.add(new ParsedItem(
|
||||
name, key, confValue, confFinal, confSourceArray));
|
||||
}
|
||||
} else {
|
||||
results.add(new ParsedItem(name, confName, confValue, confFinal,
|
||||
confSourceArray));
|
||||
}
|
||||
}
|
||||
|
||||
void parseNext() throws IOException, XMLStreamException {
|
||||
switch (reader.next()) {
|
||||
case XMLStreamConstants.START_ELEMENT:
|
||||
handleStartElement();
|
||||
break;
|
||||
case XMLStreamConstants.CHARACTERS:
|
||||
if (parseToken) {
|
||||
char[] text = reader.getTextCharacters();
|
||||
token.append(text, reader.getTextStart(), reader.getTextLength());
|
||||
}
|
||||
break;
|
||||
case XMLStreamConstants.END_ELEMENT:
|
||||
handleEndElement();
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Add tags defined in HADOOP_TAGS_SYSTEM, HADOOP_TAGS_CUSTOM.
|
||||
* @param prop
|
||||
@ -3225,7 +3319,7 @@ public void addTags(Properties prop) {
|
||||
* @param confSource
|
||||
*/
|
||||
private void readTagFromConfig(String attributeValue, String confName, String
|
||||
confValue, List<String> confSource) {
|
||||
confValue, String[] confSource) {
|
||||
for (String tagStr : attributeValue.split(",")) {
|
||||
try {
|
||||
tagStr = tagStr.trim();
|
||||
@ -3243,7 +3337,7 @@ private void readTagFromConfig(String attributeValue, String confName, String
|
||||
} catch (Exception ex) {
|
||||
// Log the exception at trace level.
|
||||
LOG.trace("Tag '{}' for property:{} Source:{}", tagStr, confName,
|
||||
Arrays.toString(confSource.toArray()), ex);
|
||||
confSource, ex);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -219,10 +219,12 @@ public boolean accept(final Path file) {
|
||||
* The FileContext is defined by.
|
||||
* 1) defaultFS (slash)
|
||||
* 2) wd
|
||||
* 3) umask (Obtained by FsPermission.getUMask(conf))
|
||||
* 3) umask (explicitly set via setUMask(),
|
||||
* falling back to FsPermission.getUMask(conf))
|
||||
*/
|
||||
private final AbstractFileSystem defaultFS; //default FS for this FileContext.
|
||||
private Path workingDir; // Fully qualified
|
||||
private FsPermission umask;
|
||||
private final Configuration conf;
|
||||
private final UserGroupInformation ugi;
|
||||
final boolean resolveSymlinks;
|
||||
@ -575,7 +577,7 @@ public Path getHomeDirectory() {
|
||||
* @return the umask of this FileContext
|
||||
*/
|
||||
public FsPermission getUMask() {
|
||||
return FsPermission.getUMask(conf);
|
||||
return (umask != null ? umask : FsPermission.getUMask(conf));
|
||||
}
|
||||
|
||||
/**
|
||||
@ -583,10 +585,9 @@ public FsPermission getUMask() {
|
||||
* @param newUmask the new umask
|
||||
*/
|
||||
public void setUMask(final FsPermission newUmask) {
|
||||
FsPermission.setUMask(conf, newUmask);
|
||||
this.umask = newUmask;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Resolve the path following any symlinks or mount points
|
||||
* @param f to be resolved
|
||||
|
@ -418,7 +418,12 @@ public Path getLocalPathForWrite(String pathStr, long size,
|
||||
}
|
||||
}
|
||||
} else {
|
||||
int dirNum = ctx.getAndIncrDirNumLastAccessed();
|
||||
// Start linear search with random increment if possible
|
||||
int randomInc = 1;
|
||||
if (numDirs > 2) {
|
||||
randomInc += dirIndexRandomizer.nextInt(numDirs - 1);
|
||||
}
|
||||
int dirNum = ctx.getAndIncrDirNumLastAccessed(randomInc);
|
||||
while (numDirsSearched < numDirs) {
|
||||
long capacity = ctx.dirDF[dirNum].getAvailable();
|
||||
if (capacity > size) {
|
||||
|
@ -159,14 +159,17 @@ Each metrics record contains tags such as ProcessName, SessionId, and Hostname a
|
||||
| `TransactionsAvgTime` | Average time of Journal transactions in milliseconds |
|
||||
| `SyncsNumOps` | Total number of Journal syncs |
|
||||
| `SyncsAvgTime` | Average time of Journal syncs in milliseconds |
|
||||
| `SyncsTime`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of Journal sync time in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `TransactionsBatchedInSync` | Total number of Journal transactions batched in sync |
|
||||
| `TransactionsBatchedInSync`*num*`s(50/75/90/95/99)thPercentileCount` | The 50/75/90/95/99th percentile of number of batched Journal transactions (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `StorageBlockReportNumOps` | Total number of processing block reports from individual storages in DataNode |
|
||||
| `StorageBlockReportAvgTime` | Average time of processing block reports in milliseconds |
|
||||
| `StorageBlockReport`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of block report processing time in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `CacheReportNumOps` | Total number of processing cache reports from DataNode |
|
||||
| `CacheReportAvgTime` | Average time of processing cache reports in milliseconds |
|
||||
| `CacheReport`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of cached report processing time in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `SafeModeTime` | The interval between FSNameSystem starts and the last time safemode leaves in milliseconds. (sometimes not equal to the time in SafeMode, see [HDFS-5156](https://issues.apache.org/jira/browse/HDFS-5156)) |
|
||||
| `FsImageLoadTime` | Time loading FS Image at startup in milliseconds |
|
||||
| `FsImageLoadTime` | Time loading FS Image at startup in milliseconds |
|
||||
| `GetEditNumOps` | Total number of edits downloads from SecondaryNameNode |
|
||||
| `GetEditAvgTime` | Average edits download time in milliseconds |
|
||||
| `GetImageNumOps` | Total number of fsimage downloads from SecondaryNameNode |
|
||||
@ -177,22 +180,23 @@ Each metrics record contains tags such as ProcessName, SessionId, and Hostname a
|
||||
| `NNStartedTimeInMillis`| NameNode start time in milliseconds |
|
||||
| `GenerateEDEKTimeNumOps` | Total number of generating EDEK |
|
||||
| `GenerateEDEKTimeAvgTime` | Average time of generating EDEK in milliseconds |
|
||||
| `GenerateEDEKTime`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of time spent in generating EDEK in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `WarmUpEDEKTimeNumOps` | Total number of warming up EDEK |
|
||||
| `WarmUpEDEKTimeAvgTime` | Average time of warming up EDEK in milliseconds |
|
||||
| `ResourceCheckTime`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of NameNode resource check latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `StorageBlockReport`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of storage block report latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `WarmUpEDEKTime`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of time spent in warming up EDEK in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `ResourceCheckTime`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of of NameNode resource check latency in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `EditLogTailTimeNumOps` | Total number of times the standby NameNode tailed the edit log |
|
||||
| `EditLogTailTimeAvgTime` | Average time (in milliseconds) spent by standby NameNode in tailing edit log |
|
||||
| `EditLogTailTime`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of time spent in tailing edit logs by standby NameNode, in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `EditLogTailTime`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of time spent in tailing edit logs by standby NameNode in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `EditLogFetchTimeNumOps` | Total number of times the standby NameNode fetched remote edit streams from journal nodes |
|
||||
| `EditLogFetchTimeAvgTime` | Average time (in milliseconds) spent by standby NameNode in fetching remote edit streams from journal nodes |
|
||||
| `EditLogFetchTime`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of time spent in fetching edit streams from journal nodes by standby NameNode, in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `EditLogFetchTime`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of time spent in fetching edit streams from journal nodes by standby NameNode in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `NumEditLogLoadedNumOps` | Total number of times edits were loaded by standby NameNode |
|
||||
| `NumEditLogLoadedAvgCount` | Average number of edits loaded by standby NameNode in each edit log tailing |
|
||||
| `NumEditLogLoaded`*num*`s(50/75/90/95/99)thPercentileCount` | The 50/75/90/95/99th percentile of number of edits loaded by standby NameNode in each edit log tailing. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `NumEditLogLoaded`*num*`s(50/75/90/95/99)thPercentileCount` | The 50/75/90/95/99th percentile of number of edits loaded by standby NameNode in each edit log tailing (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `EditLogTailIntervalNumOps` | Total number of intervals between edit log tailings by standby NameNode |
|
||||
| `EditLogTailIntervalAvgTime` | Average time of intervals between edit log tailings by standby NameNode in milliseconds |
|
||||
| `EditLogTailInterval`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of time between edit log tailings by standby NameNode, in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `EditLogTailInterval`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of time between edit log tailings by standby NameNode in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
|
||||
FSNamesystem
|
||||
------------
|
||||
@ -338,13 +342,13 @@ Each metrics record contains tags such as SessionId and Hostname as additional i
|
||||
| `RamDiskBlocksEvictedWithoutRead` | Total number of blocks evicted in memory without ever being read from memory |
|
||||
| `RamDiskBlocksEvictionWindowMsNumOps` | Number of blocks evicted in memory|
|
||||
| `RamDiskBlocksEvictionWindowMsAvgTime` | Average time of blocks in memory before being evicted in milliseconds |
|
||||
| `RamDiskBlocksEvictionWindows`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of latency between memory write and eviction in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `RamDiskBlocksEvictionWindows`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of latency between memory write and eviction in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `RamDiskBlocksLazyPersisted` | Total number of blocks written to disk by lazy writer |
|
||||
| `RamDiskBlocksDeletedBeforeLazyPersisted` | Total number of blocks deleted by application before being persisted to disk |
|
||||
| `RamDiskBytesLazyPersisted` | Total number of bytes written to disk by lazy writer |
|
||||
| `RamDiskBlocksLazyPersistWindowMsNumOps` | Number of blocks written to disk by lazy writer |
|
||||
| `RamDiskBlocksLazyPersistWindowMsAvgTime` | Average time of blocks written to disk by lazy writer in milliseconds |
|
||||
| `RamDiskBlocksLazyPersistWindows`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of latency between memory write and disk persist in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `RamDiskBlocksLazyPersistWindows`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of latency between memory write and disk persist in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `FsyncCount` | Total number of fsync |
|
||||
| `VolumeFailures` | Total number of volume failures occurred |
|
||||
| `ReadBlockOpNumOps` | Total number of read operations |
|
||||
@ -371,14 +375,19 @@ Each metrics record contains tags such as SessionId and Hostname as additional i
|
||||
| `CacheReportsAvgTime` | Average time of cache report operations in milliseconds |
|
||||
| `PacketAckRoundTripTimeNanosNumOps` | Total number of ack round trip |
|
||||
| `PacketAckRoundTripTimeNanosAvgTime` | Average time from ack send to receive minus the downstream ack time in nanoseconds |
|
||||
| `PacketAckRoundTripTimeNanos`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile latency from ack send to receive minus the downstream ack time in nanoseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `FlushNanosNumOps` | Total number of flushes |
|
||||
| `FlushNanosAvgTime` | Average flush time in nanoseconds |
|
||||
| `FlushNanos`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile flush time in nanoseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `FsyncNanosNumOps` | Total number of fsync |
|
||||
| `FsyncNanosAvgTime` | Average fsync time in nanoseconds |
|
||||
| `FsyncNanos`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile fsync time in nanoseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `SendDataPacketBlockedOnNetworkNanosNumOps` | Total number of sending packets |
|
||||
| `SendDataPacketBlockedOnNetworkNanosAvgTime` | Average waiting time of sending packets in nanoseconds |
|
||||
| `SendDataPacketBlockedOnNetworkNanos`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile waiting time of sending packets in nanoseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `SendDataPacketTransferNanosNumOps` | Total number of sending packets |
|
||||
| `SendDataPacketTransferNanosAvgTime` | Average transfer time of sending packets in nanoseconds |
|
||||
| `SendDataPacketTransferNanos`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile transfer time of sending packets in nanoseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `TotalWriteTime`| Total number of milliseconds spent on write operation |
|
||||
| `TotalReadTime` | Total number of milliseconds spent on read operation |
|
||||
| `RemoteBytesRead` | Number of bytes read by remote clients |
|
||||
@ -410,23 +419,23 @@ contains tags such as Hostname as additional information along with metrics.
|
||||
| `TotalMetadataOperations` | Total number (monotonically increasing) of metadata operations. Metadata operations include stat, list, mkdir, delete, move, open and posix_fadvise. |
|
||||
| `MetadataOperationRateNumOps` | The number of metadata operations within an interval time of metric |
|
||||
| `MetadataOperationRateAvgTime` | Mean time of metadata operations in milliseconds |
|
||||
| `MetadataOperationLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of metadata operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `MetadataOperationLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of metadata operations latency in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `TotalDataFileIos` | Total number (monotonically increasing) of data file io operations |
|
||||
| `DataFileIoRateNumOps` | The number of data file io operations within an interval time of metric |
|
||||
| `DataFileIoRateAvgTime` | Mean time of data file io operations in milliseconds |
|
||||
| `DataFileIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of data file io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `DataFileIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of data file io operations latency in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `FlushIoRateNumOps` | The number of file flush io operations within an interval time of metric |
|
||||
| `FlushIoRateAvgTime` | Mean time of file flush io operations in milliseconds |
|
||||
| `FlushIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file flush io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `FlushIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file flush io operations latency in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `SyncIoRateNumOps` | The number of file sync io operations within an interval time of metric |
|
||||
| `SyncIoRateAvgTime` | Mean time of file sync io operations in milliseconds |
|
||||
| `SyncIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file sync io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `SyncIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file sync io operations latency in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `ReadIoRateNumOps` | The number of file read io operations within an interval time of metric |
|
||||
| `ReadIoRateAvgTime` | Mean time of file read io operations in milliseconds |
|
||||
| `ReadIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file read io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `ReadIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file read io operations latency in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `WriteIoRateNumOps` | The number of file write io operations within an interval time of metric |
|
||||
| `WriteIoRateAvgTime` | Mean time of file write io operations in milliseconds |
|
||||
| `WriteIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file write io operations latency in milliseconds. Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `WriteIoLatency`*num*`s(50/75/90/95/99)thPercentileLatency` | The 50/75/90/95/99th percentile of file write io operations latency in milliseconds (*num* seconds granularity). Percentile measurement is off by default, by watching no intervals. The intervals are specified by `dfs.metrics.percentiles.intervals`. |
|
||||
| `TotalFileIoErrors` | Total number (monotonically increasing) of file io error operations |
|
||||
| `FileIoErrorRateNumOps` | The number of file io error operations within an interval time of metric |
|
||||
| `FileIoErrorRateAvgTime` | It measures the mean time in milliseconds from the start of an operation to hitting a failure |
|
||||
|
@ -17,13 +17,17 @@
|
||||
*/
|
||||
package org.apache.hadoop.fs;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.net.URI;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.junit.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
public class TestFileContext {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(TestFileContext
|
||||
.class);
|
||||
@ -39,4 +43,40 @@ public void testDefaultURIWithoutScheme() throws Exception {
|
||||
LOG.info("Expected exception: ", ufse);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConfBasedAndAPIBasedSetUMask() throws Exception {
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
|
||||
String defaultlUMask =
|
||||
conf.get(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY);
|
||||
assertEquals("Default UMask changed!", "022", defaultlUMask);
|
||||
|
||||
URI uri1 = new URI("file://mydfs:50070/");
|
||||
URI uri2 = new URI("file://tmp");
|
||||
|
||||
FileContext fc1 = FileContext.getFileContext(uri1, conf);
|
||||
FileContext fc2 = FileContext.getFileContext(uri2, conf);
|
||||
assertEquals("Umask for fc1 is incorrect", 022, fc1.getUMask().toShort());
|
||||
assertEquals("Umask for fc2 is incorrect", 022, fc2.getUMask().toShort());
|
||||
|
||||
// Till a user explicitly calls FileContext.setUMask(), the updates through
|
||||
// configuration should be reflected..
|
||||
conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "011");
|
||||
assertEquals("Umask for fc1 is incorrect", 011, fc1.getUMask().toShort());
|
||||
assertEquals("Umask for fc2 is incorrect", 011, fc2.getUMask().toShort());
|
||||
|
||||
// Stop reflecting the conf update for specific FileContexts, once an
|
||||
// explicit setUMask is done.
|
||||
conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "066");
|
||||
fc1.setUMask(FsPermission.createImmutable((short) 00033));
|
||||
assertEquals("Umask for fc1 is incorrect", 033, fc1.getUMask().toShort());
|
||||
assertEquals("Umask for fc2 is incorrect", 066, fc2.getUMask().toShort());
|
||||
|
||||
conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "077");
|
||||
fc2.setUMask(FsPermission.createImmutable((short) 00044));
|
||||
assertEquals("Umask for fc1 is incorrect", 033, fc1.getUMask().toShort());
|
||||
assertEquals("Umask for fc2 is incorrect", 044, fc2.getUMask().toShort());
|
||||
}
|
||||
}
|
||||
|
@ -265,6 +265,65 @@ public void testCreateManyFiles() throws Exception {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Five buffer dirs, on read-write disk.
|
||||
*
|
||||
* Try to create a whole bunch of files.
|
||||
* Verify that each successive creation uses a different disk
|
||||
* than the previous one (for sized requests).
|
||||
*
|
||||
* Would ideally check statistical properties of distribution, but
|
||||
* we don't have the nerve to risk false-positives here.
|
||||
*
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test (timeout = 30000)
|
||||
public void testCreateManyFilesRandom() throws Exception {
|
||||
assumeNotWindows();
|
||||
final int numDirs = 5;
|
||||
final int numTries = 100;
|
||||
String[] dirs = new String[numDirs];
|
||||
for (int d = 0; d < numDirs; ++d) {
|
||||
dirs[d] = buildBufferDir(ROOT, d);
|
||||
}
|
||||
boolean next_dir_not_selected_at_least_once = false;
|
||||
try {
|
||||
conf.set(CONTEXT, dirs[0] + "," + dirs[1] + "," + dirs[2] + ","
|
||||
+ dirs[3] + "," + dirs[4]);
|
||||
Path[] paths = new Path[5];
|
||||
for (int d = 0; d < numDirs; ++d) {
|
||||
paths[d] = new Path(dirs[d]);
|
||||
assertTrue(localFs.mkdirs(paths[d]));
|
||||
}
|
||||
|
||||
int inDir=0;
|
||||
int prevDir = -1;
|
||||
int[] counts = new int[5];
|
||||
for(int i = 0; i < numTries; ++i) {
|
||||
File result = createTempFile(SMALL_FILE_SIZE);
|
||||
for (int d = 0; d < numDirs; ++d) {
|
||||
if (result.getPath().startsWith(paths[d].toUri().getPath())) {
|
||||
inDir = d;
|
||||
break;
|
||||
}
|
||||
}
|
||||
// Verify we always select a different dir
|
||||
assertNotEquals(prevDir, inDir);
|
||||
// Verify we are not always selecting the next dir - that was the old
|
||||
// algorithm.
|
||||
if ((prevDir != -1) && (inDir != ((prevDir + 1) % numDirs))) {
|
||||
next_dir_not_selected_at_least_once = true;
|
||||
}
|
||||
prevDir = inDir;
|
||||
counts[inDir]++;
|
||||
result.delete();
|
||||
}
|
||||
} finally {
|
||||
rmBufferDirs();
|
||||
}
|
||||
assertTrue(next_dir_not_selected_at_least_once);
|
||||
}
|
||||
|
||||
/** Two buffer dirs. The first dir does not exist & is on a read-only disk;
|
||||
* The second dir exists & is RW
|
||||
* getLocalPathForWrite with checkAccess set to false should create a parent
|
||||
|
@ -25,17 +25,17 @@ services:
|
||||
command: ["/opt/hadoop/bin/ozone","datanode"]
|
||||
env_file:
|
||||
- ./docker-config
|
||||
ksm:
|
||||
ozoneManager:
|
||||
image: apache/hadoop-runner
|
||||
volumes:
|
||||
- ../../ozone:/opt/hadoop
|
||||
ports:
|
||||
- 9874:9874
|
||||
environment:
|
||||
ENSURE_KSM_INITIALIZED: /data/metadata/ksm/current/VERSION
|
||||
ENSURE_OM_INITIALIZED: /data/metadata/ozoneManager/current/VERSION
|
||||
env_file:
|
||||
- ./docker-config
|
||||
command: ["/opt/hadoop/bin/ozone","ksm"]
|
||||
command: ["/opt/hadoop/bin/ozone","om"]
|
||||
scm:
|
||||
image: apache/hadoop-runner
|
||||
volumes:
|
||||
|
@ -14,7 +14,7 @@
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
OZONE-SITE.XML_ozone.ksm.address=ksm
|
||||
OZONE-SITE.XML_ozone.om.address=ozoneManager
|
||||
OZONE-SITE.XML_ozone.scm.names=scm
|
||||
OZONE-SITE.XML_ozone.enabled=True
|
||||
OZONE-SITE.XML_ozone.scm.datanode.id=/data/datanode.id
|
||||
|
@ -67,7 +67,7 @@ http://localhost:9090/graph
|
||||
Example queries:
|
||||
|
||||
```
|
||||
Hadoop_KeySpaceManager_NumKeyCommits
|
||||
rate(Hadoop_KeySpaceManager_NumKeyCommits[10m])
|
||||
Hadoop_OzoneManager_NumKeyCommits
|
||||
rate(Hadoop_OzoneManager_NumKeyCommits[10m])
|
||||
rate(Hadoop_Ozone_BYTES_WRITTEN[10m])
|
||||
```
|
||||
|
@ -26,7 +26,7 @@ services:
|
||||
command: ["/opt/hadoop/bin/ozone","datanode"]
|
||||
env_file:
|
||||
- ./docker-config
|
||||
ksm:
|
||||
ozoneManager:
|
||||
image: apache/hadoop-runner
|
||||
volumes:
|
||||
- ../../ozone:/opt/hadoop
|
||||
@ -34,10 +34,10 @@ services:
|
||||
ports:
|
||||
- 9874:9874
|
||||
environment:
|
||||
ENSURE_KSM_INITIALIZED: /data/metadata/ksm/current/VERSION
|
||||
ENSURE_OM_INITIALIZED: /data/metadata/ozoneManager/current/VERSION
|
||||
env_file:
|
||||
- ./docker-config
|
||||
command: ["/opt/hadoop/bin/ozone","ksm"]
|
||||
command: ["/opt/hadoop/bin/ozone","om"]
|
||||
scm:
|
||||
image: apache/hadoop-runner
|
||||
volumes:
|
||||
|
@ -14,7 +14,7 @@
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
OZONE-SITE.XML_ozone.ksm.address=ksm
|
||||
OZONE-SITE.XML_ozone.om.address=ozoneManager
|
||||
OZONE-SITE.XML_ozone.scm.names=scm
|
||||
OZONE-SITE.XML_ozone.enabled=True
|
||||
OZONE-SITE.XML_ozone.scm.datanode.id=/data/datanode.id
|
||||
|
@ -20,6 +20,7 @@
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
||||
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||
import org.apache.hadoop.hdds.scm.protocolPB
|
||||
@ -36,7 +37,6 @@
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
|
||||
@ -87,16 +87,17 @@ public static void setContainerSizeB(long size) {
|
||||
* @inheritDoc
|
||||
*/
|
||||
@Override
|
||||
public ContainerInfo createContainer(String owner)
|
||||
public ContainerWithPipeline createContainer(String owner)
|
||||
throws IOException {
|
||||
XceiverClientSpi client = null;
|
||||
try {
|
||||
ContainerInfo container =
|
||||
ContainerWithPipeline containerWithPipeline =
|
||||
storageContainerLocationClient.allocateContainer(
|
||||
xceiverClientManager.getType(),
|
||||
xceiverClientManager.getFactor(), owner);
|
||||
Pipeline pipeline = container.getPipeline();
|
||||
client = xceiverClientManager.acquireClient(pipeline, container.getContainerID());
|
||||
Pipeline pipeline = containerWithPipeline.getPipeline();
|
||||
client = xceiverClientManager.acquireClient(pipeline,
|
||||
containerWithPipeline.getContainerInfo().getContainerID());
|
||||
|
||||
// Allocated State means that SCM has allocated this pipeline in its
|
||||
// namespace. The client needs to create the pipeline on the machines
|
||||
@ -106,8 +107,9 @@ public ContainerInfo createContainer(String owner)
|
||||
if (pipeline.getLifeCycleState() == ALLOCATED) {
|
||||
createPipeline(client, pipeline);
|
||||
}
|
||||
createContainer(client, container.getContainerID());
|
||||
return container;
|
||||
createContainer(client,
|
||||
containerWithPipeline.getContainerInfo().getContainerID());
|
||||
return containerWithPipeline;
|
||||
} finally {
|
||||
if (client != null) {
|
||||
xceiverClientManager.releaseClient(client);
|
||||
@ -197,17 +199,17 @@ private void createPipeline(XceiverClientSpi client, Pipeline pipeline)
|
||||
* @inheritDoc
|
||||
*/
|
||||
@Override
|
||||
public ContainerInfo createContainer(HddsProtos.ReplicationType type,
|
||||
public ContainerWithPipeline createContainer(HddsProtos.ReplicationType type,
|
||||
HddsProtos.ReplicationFactor factor, String owner) throws IOException {
|
||||
XceiverClientSpi client = null;
|
||||
try {
|
||||
// allocate container on SCM.
|
||||
ContainerInfo container =
|
||||
ContainerWithPipeline containerWithPipeline =
|
||||
storageContainerLocationClient.allocateContainer(type, factor,
|
||||
owner);
|
||||
Pipeline pipeline = container.getPipeline();
|
||||
Pipeline pipeline = containerWithPipeline.getPipeline();
|
||||
client = xceiverClientManager.acquireClient(pipeline,
|
||||
container.getContainerID());
|
||||
containerWithPipeline.getContainerInfo().getContainerID());
|
||||
|
||||
// Allocated State means that SCM has allocated this pipeline in its
|
||||
// namespace. The client needs to create the pipeline on the machines
|
||||
@ -217,9 +219,10 @@ public ContainerInfo createContainer(HddsProtos.ReplicationType type,
|
||||
}
|
||||
// connect to pipeline leader and allocate container on leader datanode.
|
||||
client = xceiverClientManager.acquireClient(pipeline,
|
||||
container.getContainerID());
|
||||
createContainer(client, container.getContainerID());
|
||||
return container;
|
||||
containerWithPipeline.getContainerInfo().getContainerID());
|
||||
createContainer(client,
|
||||
containerWithPipeline.getContainerInfo().getContainerID());
|
||||
return containerWithPipeline;
|
||||
} finally {
|
||||
if (client != null) {
|
||||
xceiverClientManager.releaseClient(client);
|
||||
@ -230,14 +233,14 @@ public ContainerInfo createContainer(HddsProtos.ReplicationType type,
|
||||
/**
|
||||
* Returns a set of Nodes that meet a query criteria.
|
||||
*
|
||||
* @param nodeStatuses - A set of criteria that we want the node to have.
|
||||
* @param nodeStatuses - Criteria that we want the node to have.
|
||||
* @param queryScope - Query scope - Cluster or pool.
|
||||
* @param poolName - if it is pool, a pool name is required.
|
||||
* @return A set of nodes that meet the requested criteria.
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public HddsProtos.NodePool queryNode(EnumSet<HddsProtos.NodeState>
|
||||
public List<HddsProtos.Node> queryNode(HddsProtos.NodeState
|
||||
nodeStatuses, HddsProtos.QueryScope queryScope, String poolName)
|
||||
throws IOException {
|
||||
return storageContainerLocationClient.queryNode(nodeStatuses, queryScope,
|
||||
@ -256,24 +259,27 @@ public Pipeline createReplicationPipeline(HddsProtos.ReplicationType type,
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete the container, this will release any resource it uses.
|
||||
* @param pipeline - Pipeline that represents the container.
|
||||
* @param force - True to forcibly delete the container.
|
||||
* Deletes an existing container.
|
||||
*
|
||||
* @param containerId - ID of the container.
|
||||
* @param pipeline - Pipeline that represents the container.
|
||||
* @param force - true to forcibly delete the container.
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public void deleteContainer(long containerID, Pipeline pipeline, boolean force)
|
||||
throws IOException {
|
||||
public void deleteContainer(long containerId, Pipeline pipeline,
|
||||
boolean force) throws IOException {
|
||||
XceiverClientSpi client = null;
|
||||
try {
|
||||
client = xceiverClientManager.acquireClient(pipeline, containerID);
|
||||
client = xceiverClientManager.acquireClient(pipeline, containerId);
|
||||
String traceID = UUID.randomUUID().toString();
|
||||
ContainerProtocolCalls.deleteContainer(client, containerID, force, traceID);
|
||||
ContainerProtocolCalls
|
||||
.deleteContainer(client, containerId, force, traceID);
|
||||
storageContainerLocationClient
|
||||
.deleteContainer(containerID);
|
||||
.deleteContainer(containerId);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Deleted container {}, leader: {}, machines: {} ",
|
||||
containerID,
|
||||
containerId,
|
||||
pipeline.getLeader(),
|
||||
pipeline.getMachines());
|
||||
}
|
||||
@ -284,6 +290,19 @@ public void deleteContainer(long containerID, Pipeline pipeline, boolean force)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete the container, this will release any resource it uses.
|
||||
* @param containerID - containerID.
|
||||
* @param force - True to forcibly delete the container.
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public void deleteContainer(long containerID, boolean force)
|
||||
throws IOException {
|
||||
ContainerWithPipeline info = getContainerWithPipeline(containerID);
|
||||
deleteContainer(containerID, info.getPipeline(), force);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@ -297,9 +316,9 @@ public List<ContainerInfo> listContainer(long startContainerID,
|
||||
/**
|
||||
* Get meta data from an existing container.
|
||||
*
|
||||
* @param pipeline - pipeline that represents the container.
|
||||
* @return ContainerInfo - a message of protobuf which has basic info
|
||||
* of a container.
|
||||
* @param containerID - ID of the container.
|
||||
* @param pipeline - Pipeline where the container is located.
|
||||
* @return ContainerInfo
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
@ -325,6 +344,19 @@ public ContainerData readContainer(long containerID,
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get meta data from an existing container.
|
||||
* @param containerID - ID of the container.
|
||||
* @return ContainerInfo - a message of protobuf which has basic info
|
||||
* of a container.
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public ContainerData readContainer(long containerID) throws IOException {
|
||||
ContainerWithPipeline info = getContainerWithPipeline(containerID);
|
||||
return readContainer(containerID, info.getPipeline());
|
||||
}
|
||||
|
||||
/**
|
||||
* Given an id, return the pipeline associated with the container.
|
||||
* @param containerId - String Container ID
|
||||
@ -337,6 +369,19 @@ public ContainerInfo getContainer(long containerId) throws
|
||||
return storageContainerLocationClient.getContainer(containerId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets a container by Name -- Throws if the container does not exist.
|
||||
*
|
||||
* @param containerId - Container ID
|
||||
* @return ContainerWithPipeline
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public ContainerWithPipeline getContainerWithPipeline(long containerId)
|
||||
throws IOException {
|
||||
return storageContainerLocationClient.getContainerWithPipeline(containerId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Close a container.
|
||||
*
|
||||
@ -391,6 +436,19 @@ public void closeContainer(long containerId, Pipeline pipeline)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Close a container.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public void closeContainer(long containerId)
|
||||
throws IOException {
|
||||
ContainerWithPipeline info = getContainerWithPipeline(containerId);
|
||||
Pipeline pipeline = info.getPipeline();
|
||||
closeContainer(containerId, pipeline);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the the current usage information.
|
||||
* @param containerID - ID of the container.
|
||||
@ -399,7 +457,8 @@ public void closeContainer(long containerId, Pipeline pipeline)
|
||||
*/
|
||||
@Override
|
||||
public long getContainerSize(long containerID) throws IOException {
|
||||
// TODO : Fix this, it currently returns the capacity but not the current usage.
|
||||
// TODO : Fix this, it currently returns the capacity
|
||||
// but not the current usage.
|
||||
long size = getContainerSizeB();
|
||||
if (size == -1) {
|
||||
throw new IOException("Container size unknown!");
|
||||
|
@ -31,6 +31,8 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<properties>
|
||||
<hadoop.component>hdds</hadoop.component>
|
||||
<is.hadoop.component>true</is.hadoop.component>
|
||||
<log4j2.version>2.11.0</log4j2.version>
|
||||
<disruptor.version>3.4.2</disruptor.version>
|
||||
</properties>
|
||||
|
||||
<dependencies>
|
||||
@ -81,6 +83,22 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<type>test-jar</type>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
<artifactId>log4j-api</artifactId>
|
||||
<version>${log4j2.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
<artifactId>log4j-core</artifactId>
|
||||
<version>${log4j2.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.lmax</groupId>
|
||||
<artifactId>disruptor</artifactId>
|
||||
<version>${disruptor.version}</version>
|
||||
</dependency>
|
||||
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
|
@ -35,7 +35,7 @@
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public final class DatanodeDetails implements Comparable<DatanodeDetails> {
|
||||
public class DatanodeDetails implements Comparable<DatanodeDetails> {
|
||||
|
||||
/**
|
||||
* DataNode's unique identifier in the cluster.
|
||||
@ -63,6 +63,13 @@ private DatanodeDetails(String uuid, String ipAddress, String hostName,
|
||||
this.ports = ports;
|
||||
}
|
||||
|
||||
protected DatanodeDetails(DatanodeDetails datanodeDetails) {
|
||||
this.uuid = datanodeDetails.uuid;
|
||||
this.ipAddress = datanodeDetails.ipAddress;
|
||||
this.hostName = datanodeDetails.hostName;
|
||||
this.ports = datanodeDetails.ports;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the DataNode UUID.
|
||||
*
|
||||
@ -238,7 +245,7 @@ public static Builder newBuilder() {
|
||||
/**
|
||||
* Builder class for building DatanodeDetails.
|
||||
*/
|
||||
public static class Builder {
|
||||
public static final class Builder {
|
||||
private String id;
|
||||
private String ipAddress;
|
||||
private String hostName;
|
||||
@ -324,7 +331,7 @@ public static Port newPort(Port.Name name, Integer value) {
|
||||
/**
|
||||
* Container to hold DataNode Port details.
|
||||
*/
|
||||
public static class Port {
|
||||
public static final class Port {
|
||||
|
||||
/**
|
||||
* Ports that are supported in DataNode.
|
||||
|
@ -94,7 +94,7 @@ public final class ScmConfigKeys {
|
||||
"ozone.scm.datanode.port";
|
||||
public static final int OZONE_SCM_DATANODE_PORT_DEFAULT = 9861;
|
||||
|
||||
// OZONE_KSM_PORT_DEFAULT = 9862
|
||||
// OZONE_OM_PORT_DEFAULT = 9862
|
||||
public static final String OZONE_SCM_BLOCK_CLIENT_PORT_KEY =
|
||||
"ozone.scm.block.client.port";
|
||||
public static final int OZONE_SCM_BLOCK_CLIENT_PORT_DEFAULT = 9863;
|
||||
@ -166,10 +166,6 @@ public final class ScmConfigKeys {
|
||||
public static final String OZONE_SCM_DEADNODE_INTERVAL_DEFAULT =
|
||||
"10m";
|
||||
|
||||
public static final String OZONE_SCM_MAX_HB_COUNT_TO_PROCESS =
|
||||
"ozone.scm.max.hb.count.to.process";
|
||||
public static final int OZONE_SCM_MAX_HB_COUNT_TO_PROCESS_DEFAULT = 5000;
|
||||
|
||||
public static final String OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL =
|
||||
"ozone.scm.heartbeat.thread.interval";
|
||||
public static final String OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_DEFAULT =
|
||||
|
@ -18,6 +18,7 @@
|
||||
package org.apache.hadoop.hdds.scm.client;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||
@ -25,7 +26,6 @@
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
@ -45,7 +45,7 @@ public interface ScmClient {
|
||||
* @return ContainerInfo
|
||||
* @throws IOException
|
||||
*/
|
||||
ContainerInfo createContainer(String owner) throws IOException;
|
||||
ContainerWithPipeline createContainer(String owner) throws IOException;
|
||||
|
||||
/**
|
||||
* Gets a container by Name -- Throws if the container does not exist.
|
||||
@ -55,6 +55,14 @@ public interface ScmClient {
|
||||
*/
|
||||
ContainerInfo getContainer(long containerId) throws IOException;
|
||||
|
||||
/**
|
||||
* Gets a container by Name -- Throws if the container does not exist.
|
||||
* @param containerId - Container ID
|
||||
* @return ContainerWithPipeline
|
||||
* @throws IOException
|
||||
*/
|
||||
ContainerWithPipeline getContainerWithPipeline(long containerId) throws IOException;
|
||||
|
||||
/**
|
||||
* Close a container.
|
||||
*
|
||||
@ -64,6 +72,14 @@ public interface ScmClient {
|
||||
*/
|
||||
void closeContainer(long containerId, Pipeline pipeline) throws IOException;
|
||||
|
||||
/**
|
||||
* Close a container.
|
||||
*
|
||||
* @param containerId - ID of the container.
|
||||
* @throws IOException
|
||||
*/
|
||||
void closeContainer(long containerId) throws IOException;
|
||||
|
||||
/**
|
||||
* Deletes an existing container.
|
||||
* @param containerId - ID of the container.
|
||||
@ -73,6 +89,14 @@ public interface ScmClient {
|
||||
*/
|
||||
void deleteContainer(long containerId, Pipeline pipeline, boolean force) throws IOException;
|
||||
|
||||
/**
|
||||
* Deletes an existing container.
|
||||
* @param containerId - ID of the container.
|
||||
* @param force - true to forcibly delete the container.
|
||||
* @throws IOException
|
||||
*/
|
||||
void deleteContainer(long containerId, boolean force) throws IOException;
|
||||
|
||||
/**
|
||||
* Lists a range of containers and get their info.
|
||||
*
|
||||
@ -95,6 +119,15 @@ List<ContainerInfo> listContainer(long startContainerID,
|
||||
ContainerData readContainer(long containerID, Pipeline pipeline)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Read meta data from an existing container.
|
||||
* @param containerID - ID of the container.
|
||||
* @return ContainerInfo
|
||||
* @throws IOException
|
||||
*/
|
||||
ContainerData readContainer(long containerID)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Gets the container size -- Computed by SCM from Container Reports.
|
||||
* @param containerID - ID of the container.
|
||||
@ -110,19 +143,19 @@ ContainerData readContainer(long containerID, Pipeline pipeline)
|
||||
* @return ContainerInfo
|
||||
* @throws IOException - in case of error.
|
||||
*/
|
||||
ContainerInfo createContainer(HddsProtos.ReplicationType type,
|
||||
ContainerWithPipeline createContainer(HddsProtos.ReplicationType type,
|
||||
HddsProtos.ReplicationFactor replicationFactor,
|
||||
String owner) throws IOException;
|
||||
|
||||
/**
|
||||
* Returns a set of Nodes that meet a query criteria.
|
||||
* @param nodeStatuses - A set of criteria that we want the node to have.
|
||||
* @param nodeStatuses - Criteria that we want the node to have.
|
||||
* @param queryScope - Query scope - Cluster or pool.
|
||||
* @param poolName - if it is pool, a pool name is required.
|
||||
* @return A set of nodes that meet the requested criteria.
|
||||
* @throws IOException
|
||||
*/
|
||||
HddsProtos.NodePool queryNode(EnumSet<HddsProtos.NodeState> nodeStatuses,
|
||||
List<HddsProtos.Node> queryNode(HddsProtos.NodeState nodeStatuses,
|
||||
HddsProtos.QueryScope queryScope, String poolName) throws IOException;
|
||||
|
||||
/**
|
||||
|
@ -15,34 +15,39 @@
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hdds.scm.container.common.helpers;
|
||||
|
||||
import static java.lang.Math.max;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonAutoDetect;
|
||||
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||
import com.fasterxml.jackson.annotation.PropertyAccessor;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.ObjectWriter;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.commons.lang3.builder.EqualsBuilder;
|
||||
import org.apache.commons.lang3.builder.HashCodeBuilder;
|
||||
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
import java.io.Externalizable;
|
||||
import java.io.IOException;
|
||||
import java.io.ObjectInput;
|
||||
import java.io.ObjectOutput;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
|
||||
import static java.lang.Math.max;
|
||||
import org.apache.commons.lang3.builder.EqualsBuilder;
|
||||
import org.apache.commons.lang3.builder.HashCodeBuilder;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
||||
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
/**
|
||||
* Class wraps ozone container info.
|
||||
*/
|
||||
public class ContainerInfo
|
||||
implements Comparator<ContainerInfo>, Comparable<ContainerInfo> {
|
||||
public class ContainerInfo implements Comparator<ContainerInfo>,
|
||||
Comparable<ContainerInfo>, Externalizable {
|
||||
|
||||
private static final ObjectWriter WRITER;
|
||||
private static final String SERIALIZATION_ERROR_MSG = "Java serialization not"
|
||||
+ " supported. Use protobuf instead.";
|
||||
|
||||
static {
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
@ -53,7 +58,9 @@ public class ContainerInfo
|
||||
}
|
||||
|
||||
private HddsProtos.LifeCycleState state;
|
||||
private Pipeline pipeline;
|
||||
private String pipelineName;
|
||||
private ReplicationFactor replicationFactor;
|
||||
private ReplicationType replicationType;
|
||||
// Bytes allocated by SCM for clients.
|
||||
private long allocatedBytes;
|
||||
// Actual container usage, updated through heartbeat.
|
||||
@ -75,15 +82,17 @@ public class ContainerInfo
|
||||
ContainerInfo(
|
||||
long containerID,
|
||||
HddsProtos.LifeCycleState state,
|
||||
Pipeline pipeline,
|
||||
String pipelineName,
|
||||
long allocatedBytes,
|
||||
long usedBytes,
|
||||
long numberOfKeys,
|
||||
long stateEnterTime,
|
||||
String owner,
|
||||
long deleteTransactionId) {
|
||||
long deleteTransactionId,
|
||||
ReplicationFactor replicationFactor,
|
||||
ReplicationType repType) {
|
||||
this.containerID = containerID;
|
||||
this.pipeline = pipeline;
|
||||
this.pipelineName = pipelineName;
|
||||
this.allocatedBytes = allocatedBytes;
|
||||
this.usedBytes = usedBytes;
|
||||
this.numberOfKeys = numberOfKeys;
|
||||
@ -92,6 +101,8 @@ public class ContainerInfo
|
||||
this.stateEnterTime = stateEnterTime;
|
||||
this.owner = owner;
|
||||
this.deleteTransactionId = deleteTransactionId;
|
||||
this.replicationFactor = replicationFactor;
|
||||
this.replicationType = repType;
|
||||
}
|
||||
|
||||
/**
|
||||
@ -102,16 +113,18 @@ public ContainerInfo() {
|
||||
|
||||
public static ContainerInfo fromProtobuf(HddsProtos.SCMContainerInfo info) {
|
||||
ContainerInfo.Builder builder = new ContainerInfo.Builder();
|
||||
builder.setPipeline(Pipeline.getFromProtoBuf(info.getPipeline()));
|
||||
builder.setAllocatedBytes(info.getAllocatedBytes());
|
||||
builder.setUsedBytes(info.getUsedBytes());
|
||||
builder.setNumberOfKeys(info.getNumberOfKeys());
|
||||
builder.setState(info.getState());
|
||||
builder.setStateEnterTime(info.getStateEnterTime());
|
||||
builder.setOwner(info.getOwner());
|
||||
builder.setContainerID(info.getContainerID());
|
||||
builder.setDeleteTransactionId(info.getDeleteTransactionId());
|
||||
return builder.build();
|
||||
return builder.setPipelineName(info.getPipelineName())
|
||||
.setAllocatedBytes(info.getAllocatedBytes())
|
||||
.setUsedBytes(info.getUsedBytes())
|
||||
.setNumberOfKeys(info.getNumberOfKeys())
|
||||
.setState(info.getState())
|
||||
.setStateEnterTime(info.getStateEnterTime())
|
||||
.setOwner(info.getOwner())
|
||||
.setContainerID(info.getContainerID())
|
||||
.setDeleteTransactionId(info.getDeleteTransactionId())
|
||||
.setReplicationFactor(info.getReplicationFactor())
|
||||
.setReplicationType(info.getReplicationType())
|
||||
.build();
|
||||
}
|
||||
|
||||
public long getContainerID() {
|
||||
@ -130,8 +143,12 @@ public long getStateEnterTime() {
|
||||
return stateEnterTime;
|
||||
}
|
||||
|
||||
public Pipeline getPipeline() {
|
||||
return pipeline;
|
||||
public ReplicationFactor getReplicationFactor() {
|
||||
return replicationFactor;
|
||||
}
|
||||
|
||||
public String getPipelineName() {
|
||||
return pipelineName;
|
||||
}
|
||||
|
||||
public long getAllocatedBytes() {
|
||||
@ -177,6 +194,10 @@ public long getLastUsed() {
|
||||
return lastUsed;
|
||||
}
|
||||
|
||||
public ReplicationType getReplicationType() {
|
||||
return replicationType;
|
||||
}
|
||||
|
||||
public void updateLastUsedTime() {
|
||||
lastUsed = Time.monotonicNow();
|
||||
}
|
||||
@ -190,19 +211,17 @@ public void allocate(long size) {
|
||||
public HddsProtos.SCMContainerInfo getProtobuf() {
|
||||
HddsProtos.SCMContainerInfo.Builder builder =
|
||||
HddsProtos.SCMContainerInfo.newBuilder();
|
||||
builder.setPipeline(getPipeline().getProtobufMessage());
|
||||
builder.setAllocatedBytes(getAllocatedBytes());
|
||||
builder.setUsedBytes(getUsedBytes());
|
||||
builder.setNumberOfKeys(getNumberOfKeys());
|
||||
builder.setState(state);
|
||||
builder.setStateEnterTime(stateEnterTime);
|
||||
builder.setContainerID(getContainerID());
|
||||
builder.setDeleteTransactionId(deleteTransactionId);
|
||||
|
||||
if (getOwner() != null) {
|
||||
builder.setOwner(getOwner());
|
||||
}
|
||||
return builder.build();
|
||||
return builder.setAllocatedBytes(getAllocatedBytes())
|
||||
.setContainerID(getContainerID())
|
||||
.setUsedBytes(getUsedBytes())
|
||||
.setNumberOfKeys(getNumberOfKeys()).setState(getState())
|
||||
.setStateEnterTime(getStateEnterTime()).setContainerID(getContainerID())
|
||||
.setDeleteTransactionId(getDeleteTransactionId())
|
||||
.setPipelineName(getPipelineName())
|
||||
.setReplicationFactor(getReplicationFactor())
|
||||
.setReplicationType(getReplicationType())
|
||||
.setOwner(getOwner())
|
||||
.build();
|
||||
}
|
||||
|
||||
public String getOwner() {
|
||||
@ -217,7 +236,7 @@ public void setOwner(String owner) {
|
||||
public String toString() {
|
||||
return "ContainerInfo{"
|
||||
+ "state=" + state
|
||||
+ ", pipeline=" + pipeline
|
||||
+ ", pipelineName=" + pipelineName
|
||||
+ ", stateEnterTime=" + stateEnterTime
|
||||
+ ", owner=" + owner
|
||||
+ '}';
|
||||
@ -252,9 +271,7 @@ public boolean equals(Object o) {
|
||||
public int hashCode() {
|
||||
return new HashCodeBuilder(11, 811)
|
||||
.append(getContainerID())
|
||||
.append(pipeline.getFactor())
|
||||
.append(pipeline.getType())
|
||||
.append(owner)
|
||||
.append(getOwner())
|
||||
.toHashCode();
|
||||
}
|
||||
|
||||
@ -327,12 +344,44 @@ public void setData(byte[] data) {
|
||||
this.data = Arrays.copyOf(data, data.length);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Throws IOException as default java serialization is not supported. Use
|
||||
* serialization via protobuf instead.
|
||||
*
|
||||
* @param out the stream to write the object to
|
||||
* @throws IOException Includes any I/O exceptions that may occur
|
||||
* @serialData Overriding methods should use this tag to describe
|
||||
* the data layout of this Externalizable object.
|
||||
* List the sequence of element types and, if possible,
|
||||
* relate the element to a public/protected field and/or
|
||||
* method of this Externalizable class.
|
||||
*/
|
||||
@Override
|
||||
public void writeExternal(ObjectOutput out) throws IOException {
|
||||
throw new IOException(SERIALIZATION_ERROR_MSG);
|
||||
}
|
||||
|
||||
/**
|
||||
* Throws IOException as default java serialization is not supported. Use
|
||||
* serialization via protobuf instead.
|
||||
*
|
||||
* @param in the stream to read data from in order to restore the object
|
||||
* @throws IOException if I/O errors occur
|
||||
* @throws ClassNotFoundException If the class for an object being
|
||||
* restored cannot be found.
|
||||
*/
|
||||
@Override
|
||||
public void readExternal(ObjectInput in)
|
||||
throws IOException, ClassNotFoundException {
|
||||
throw new IOException(SERIALIZATION_ERROR_MSG);
|
||||
}
|
||||
|
||||
/**
|
||||
* Builder class for ContainerInfo.
|
||||
*/
|
||||
public static class Builder {
|
||||
private HddsProtos.LifeCycleState state;
|
||||
private Pipeline pipeline;
|
||||
private long allocated;
|
||||
private long used;
|
||||
private long keys;
|
||||
@ -340,6 +389,25 @@ public static class Builder {
|
||||
private String owner;
|
||||
private long containerID;
|
||||
private long deleteTransactionId;
|
||||
private String pipelineName;
|
||||
private ReplicationFactor replicationFactor;
|
||||
private ReplicationType replicationType;
|
||||
|
||||
public Builder setReplicationType(
|
||||
ReplicationType replicationType) {
|
||||
this.replicationType = replicationType;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setPipelineName(String pipelineName) {
|
||||
this.pipelineName = pipelineName;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setReplicationFactor(ReplicationFactor repFactor) {
|
||||
this.replicationFactor = repFactor;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setContainerID(long id) {
|
||||
Preconditions.checkState(id >= 0);
|
||||
@ -352,11 +420,6 @@ public Builder setState(HddsProtos.LifeCycleState lifeCycleState) {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setPipeline(Pipeline containerPipeline) {
|
||||
this.pipeline = containerPipeline;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setAllocatedBytes(long bytesAllocated) {
|
||||
this.allocated = bytesAllocated;
|
||||
return this;
|
||||
@ -388,9 +451,9 @@ public Builder setDeleteTransactionId(long deleteTransactionId) {
|
||||
}
|
||||
|
||||
public ContainerInfo build() {
|
||||
return new
|
||||
ContainerInfo(containerID, state, pipeline, allocated,
|
||||
used, keys, stateEnterTime, owner, deleteTransactionId);
|
||||
return new ContainerInfo(containerID, state, pipelineName, allocated,
|
||||
used, keys, stateEnterTime, owner, deleteTransactionId,
|
||||
replicationFactor, replicationType);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,131 @@
|
||||
/*
|
||||
* 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 java.util.Comparator;
|
||||
import org.apache.commons.lang3.builder.EqualsBuilder;
|
||||
import org.apache.commons.lang3.builder.HashCodeBuilder;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||
|
||||
/**
|
||||
* Class wraps ozone container info.
|
||||
*/
|
||||
public class ContainerWithPipeline
|
||||
implements Comparator<ContainerWithPipeline>, Comparable<ContainerWithPipeline> {
|
||||
|
||||
private final ContainerInfo containerInfo;
|
||||
private final Pipeline pipeline;
|
||||
|
||||
public ContainerWithPipeline(ContainerInfo containerInfo, Pipeline pipeline) {
|
||||
this.containerInfo = containerInfo;
|
||||
this.pipeline = pipeline;
|
||||
}
|
||||
|
||||
public ContainerInfo getContainerInfo() {
|
||||
return containerInfo;
|
||||
}
|
||||
|
||||
public Pipeline getPipeline() {
|
||||
return pipeline;
|
||||
}
|
||||
|
||||
public static ContainerWithPipeline fromProtobuf(HddsProtos.ContainerWithPipeline allocatedContainer) {
|
||||
return new ContainerWithPipeline(
|
||||
ContainerInfo.fromProtobuf(allocatedContainer.getContainerInfo()),
|
||||
Pipeline.getFromProtoBuf(allocatedContainer.getPipeline()));
|
||||
}
|
||||
|
||||
public HddsProtos.ContainerWithPipeline getProtobuf() {
|
||||
HddsProtos.ContainerWithPipeline.Builder builder =
|
||||
HddsProtos.ContainerWithPipeline.newBuilder();
|
||||
builder.setContainerInfo(getContainerInfo().getProtobuf())
|
||||
.setPipeline(getPipeline().getProtobufMessage());
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return containerInfo.toString() + " | " + pipeline.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
ContainerWithPipeline that = (ContainerWithPipeline) o;
|
||||
|
||||
return new EqualsBuilder()
|
||||
.append(getContainerInfo(), that.getContainerInfo())
|
||||
.append(getPipeline(), that.getPipeline())
|
||||
.isEquals();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return new HashCodeBuilder(11, 811)
|
||||
.append(getContainerInfo())
|
||||
.append(getPipeline())
|
||||
.toHashCode();
|
||||
}
|
||||
|
||||
/**
|
||||
* Compares its two arguments for order. Returns a negative integer, zero, or
|
||||
* a positive integer as the first argument is less than, equal to, or greater
|
||||
* than the second.<p>
|
||||
*
|
||||
* @param o1 the first object to be compared.
|
||||
* @param o2 the second object to be compared.
|
||||
* @return a negative integer, zero, or a positive integer as the first
|
||||
* argument is less than, equal to, or greater than the second.
|
||||
* @throws NullPointerException if an argument is null and this comparator
|
||||
* does not permit null arguments
|
||||
* @throws ClassCastException if the arguments' types prevent them from
|
||||
* being compared by this comparator.
|
||||
*/
|
||||
@Override
|
||||
public int compare(ContainerWithPipeline o1, ContainerWithPipeline o2) {
|
||||
return o1.getContainerInfo().compareTo(o2.getContainerInfo());
|
||||
}
|
||||
|
||||
/**
|
||||
* Compares this object with the specified object for order. Returns a
|
||||
* negative integer, zero, or a positive integer as this object is less than,
|
||||
* equal to, or greater than the specified object.
|
||||
*
|
||||
* @param o the object to be compared.
|
||||
* @return a negative integer, zero, or a positive integer as this object is
|
||||
* less than, equal to, or greater than the specified object.
|
||||
* @throws NullPointerException if the specified object is null
|
||||
* @throws ClassCastException if the specified object's type prevents it
|
||||
* from being compared to this object.
|
||||
*/
|
||||
@Override
|
||||
public int compareTo(ContainerWithPipeline o) {
|
||||
return this.compare(this, o);
|
||||
}
|
||||
|
||||
}
|
@ -18,6 +18,7 @@
|
||||
package org.apache.hadoop.hdds.scm.protocol;
|
||||
|
||||
import org.apache.hadoop.hdds.scm.ScmInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||
@ -25,7 +26,6 @@
|
||||
.StorageContainerLocationProtocolProtos.ObjectStageChangeRequestProto;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
@ -38,7 +38,7 @@ public interface StorageContainerLocationProtocol {
|
||||
* set of datanodes that should be used creating this container.
|
||||
*
|
||||
*/
|
||||
ContainerInfo allocateContainer(HddsProtos.ReplicationType replicationType,
|
||||
ContainerWithPipeline allocateContainer(HddsProtos.ReplicationType replicationType,
|
||||
HddsProtos.ReplicationFactor factor, String owner)
|
||||
throws IOException;
|
||||
|
||||
@ -53,6 +53,16 @@ ContainerInfo allocateContainer(HddsProtos.ReplicationType replicationType,
|
||||
*/
|
||||
ContainerInfo getContainer(long containerID) throws IOException;
|
||||
|
||||
/**
|
||||
* Ask SCM the location of the container. SCM responds with a group of
|
||||
* nodes where this container and its replicas are located.
|
||||
*
|
||||
* @param containerID - ID of the container.
|
||||
* @return ContainerWithPipeline - the container info with the pipeline.
|
||||
* @throws IOException
|
||||
*/
|
||||
ContainerWithPipeline getContainerWithPipeline(long containerID) throws IOException;
|
||||
|
||||
/**
|
||||
* Ask SCM a list of containers with a range of container names
|
||||
* and the limit of count.
|
||||
@ -83,10 +93,10 @@ List<ContainerInfo> listContainer(long startContainerID, int count)
|
||||
|
||||
/**
|
||||
* Queries a list of Node Statuses.
|
||||
* @param nodeStatuses
|
||||
* @param state
|
||||
* @return List of Datanodes.
|
||||
*/
|
||||
HddsProtos.NodePool queryNode(EnumSet<HddsProtos.NodeState> nodeStatuses,
|
||||
List<HddsProtos.Node> queryNode(HddsProtos.NodeState state,
|
||||
HddsProtos.QueryScope queryScope, String poolName) throws IOException;
|
||||
|
||||
/**
|
||||
|
@ -20,7 +20,10 @@
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerWithPipelineRequestProto;
|
||||
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerWithPipelineResponseProto;
|
||||
import org.apache.hadoop.hdds.scm.ScmInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
|
||||
@ -56,7 +59,6 @@
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
@ -95,7 +97,7 @@ public StorageContainerLocationProtocolClientSideTranslatorPB(
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public ContainerInfo allocateContainer(HddsProtos.ReplicationType type,
|
||||
public ContainerWithPipeline allocateContainer(HddsProtos.ReplicationType type,
|
||||
HddsProtos.ReplicationFactor factor, String owner) throws IOException {
|
||||
|
||||
ContainerRequestProto request = ContainerRequestProto.newBuilder()
|
||||
@ -114,7 +116,7 @@ public ContainerInfo allocateContainer(HddsProtos.ReplicationType type,
|
||||
throw new IOException(response.hasErrorMessage() ?
|
||||
response.getErrorMessage() : "Allocate container failed.");
|
||||
}
|
||||
return ContainerInfo.fromProtobuf(response.getContainerInfo());
|
||||
return ContainerWithPipeline.fromProtobuf(response.getContainerWithPipeline());
|
||||
}
|
||||
|
||||
public ContainerInfo getContainer(long containerID) throws IOException {
|
||||
@ -133,6 +135,25 @@ public ContainerInfo getContainer(long containerID) throws IOException {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
public ContainerWithPipeline getContainerWithPipeline(long containerID) throws IOException {
|
||||
Preconditions.checkState(containerID >= 0,
|
||||
"Container ID cannot be negative");
|
||||
GetContainerWithPipelineRequestProto request = GetContainerWithPipelineRequestProto
|
||||
.newBuilder()
|
||||
.setContainerID(containerID)
|
||||
.build();
|
||||
try {
|
||||
GetContainerWithPipelineResponseProto response =
|
||||
rpcProxy.getContainerWithPipeline(NULL_RPC_CONTROLLER, request);
|
||||
return ContainerWithPipeline.fromProtobuf(response.getContainerWithPipeline());
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@ -193,20 +214,19 @@ public void deleteContainer(long containerID)
|
||||
* @return List of Datanodes.
|
||||
*/
|
||||
@Override
|
||||
public HddsProtos.NodePool queryNode(EnumSet<HddsProtos.NodeState>
|
||||
public List<HddsProtos.Node> queryNode(HddsProtos.NodeState
|
||||
nodeStatuses, HddsProtos.QueryScope queryScope, String poolName)
|
||||
throws IOException {
|
||||
// TODO : We support only cluster wide query right now. So ignoring checking
|
||||
// queryScope and poolName
|
||||
Preconditions.checkNotNull(nodeStatuses);
|
||||
Preconditions.checkState(nodeStatuses.size() > 0);
|
||||
NodeQueryRequestProto request = NodeQueryRequestProto.newBuilder()
|
||||
.addAllQuery(nodeStatuses)
|
||||
.setState(nodeStatuses)
|
||||
.setScope(queryScope).setPoolName(poolName).build();
|
||||
try {
|
||||
NodeQueryResponseProto response =
|
||||
rpcProxy.queryNode(NULL_RPC_CONTROLLER, request);
|
||||
return response.getDatanodes();
|
||||
return response.getDatanodesList();
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
|
@ -98,7 +98,7 @@ public final class OzoneConsts {
|
||||
public static final String BLOCK_DB = "block.db";
|
||||
public static final String OPEN_CONTAINERS_DB = "openContainers.db";
|
||||
public static final String DELETED_BLOCK_DB = "deletedBlock.db";
|
||||
public static final String KSM_DB_NAME = "ksm.db";
|
||||
public static final String OM_DB_NAME = "om.db";
|
||||
|
||||
public static final String STORAGE_DIR_CHUNKS = "chunks";
|
||||
public static final String CONTAINER_FILE_CHECKSUM_EXTENSION = ".chksm";
|
||||
@ -127,13 +127,13 @@ public static Versioning getVersioning(boolean versioning) {
|
||||
public static final String OPEN_KEY_ID_DELIMINATOR = "#";
|
||||
|
||||
/**
|
||||
* KSM LevelDB prefixes.
|
||||
* OM LevelDB prefixes.
|
||||
*
|
||||
* KSM DB stores metadata as KV pairs with certain prefixes,
|
||||
* OM DB stores metadata as KV pairs with certain prefixes,
|
||||
* prefix is used to improve the performance to get related
|
||||
* metadata.
|
||||
*
|
||||
* KSM DB Schema:
|
||||
* OM DB Schema:
|
||||
* ----------------------------------------------------------
|
||||
* | KEY | VALUE |
|
||||
* ----------------------------------------------------------
|
||||
@ -148,13 +148,13 @@ public static Versioning getVersioning(boolean versioning) {
|
||||
* | #deleting#/volumeName/bucketName/keyName | KeyInfo |
|
||||
* ----------------------------------------------------------
|
||||
*/
|
||||
public static final String KSM_VOLUME_PREFIX = "/#";
|
||||
public static final String KSM_BUCKET_PREFIX = "/#";
|
||||
public static final String KSM_KEY_PREFIX = "/";
|
||||
public static final String KSM_USER_PREFIX = "$";
|
||||
public static final String OM_VOLUME_PREFIX = "/#";
|
||||
public static final String OM_BUCKET_PREFIX = "/#";
|
||||
public static final String OM_KEY_PREFIX = "/";
|
||||
public static final String OM_USER_PREFIX = "$";
|
||||
|
||||
/**
|
||||
* Max KSM Quota size of 1024 PB.
|
||||
* Max OM Quota size of 1024 PB.
|
||||
*/
|
||||
public static final long MAX_QUOTA_IN_BYTES = 1024L * 1024 * TB;
|
||||
|
||||
@ -176,9 +176,9 @@ public static Versioning getVersioning(boolean versioning) {
|
||||
public static final int INVALID_PORT = -1;
|
||||
|
||||
|
||||
// The ServiceListJSONServlet context attribute where KeySpaceManager
|
||||
// The ServiceListJSONServlet context attribute where OzoneManager
|
||||
// instance gets stored.
|
||||
public static final String KSM_CONTEXT_ATTRIBUTE = "ozone.ksm";
|
||||
public static final String OM_CONTEXT_ATTRIBUTE = "ozone.om";
|
||||
|
||||
private OzoneConsts() {
|
||||
// Never Constructed
|
||||
|
@ -0,0 +1,30 @@
|
||||
/**
|
||||
* 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.audit;
|
||||
|
||||
/**
|
||||
* Interface to define AuditAction.
|
||||
*/
|
||||
public interface AuditAction {
|
||||
/**
|
||||
* Implementation must override.
|
||||
* @return String
|
||||
*/
|
||||
String getAction();
|
||||
}
|
||||
|
@ -0,0 +1,36 @@
|
||||
/**
|
||||
* 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.audit;
|
||||
|
||||
/**
|
||||
* Enum to define AuditEventStatus values.
|
||||
*/
|
||||
public enum AuditEventStatus {
|
||||
SUCCESS("SUCCESS"),
|
||||
FAILURE("FAILURE");
|
||||
|
||||
private String status;
|
||||
|
||||
AuditEventStatus(String status){
|
||||
this.status = status;
|
||||
}
|
||||
|
||||
public String getStatus() {
|
||||
return status;
|
||||
}
|
||||
}
|
@ -0,0 +1,128 @@
|
||||
/**
|
||||
* 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.audit;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.logging.log4j.Level;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Marker;
|
||||
import org.apache.logging.log4j.message.StructuredDataMessage;
|
||||
import org.apache.logging.log4j.spi.ExtendedLogger;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Class to define Audit Logger for Ozone.
|
||||
*/
|
||||
public class AuditLogger {
|
||||
|
||||
private ExtendedLogger logger;
|
||||
|
||||
private static final String SUCCESS = AuditEventStatus.SUCCESS.getStatus();
|
||||
private static final String FAILURE = AuditEventStatus.FAILURE.getStatus();
|
||||
private static final String FQCN = AuditLogger.class.getName();
|
||||
private static final Marker WRITE_MARKER = AuditMarker.WRITE.getMarker();
|
||||
private static final Marker READ_MARKER = AuditMarker.READ.getMarker();
|
||||
|
||||
/**
|
||||
* Parametrized Constructor to initialize logger.
|
||||
* @param type
|
||||
*/
|
||||
public AuditLogger(AuditLoggerType type){
|
||||
initializeLogger(type);
|
||||
}
|
||||
|
||||
/**
|
||||
* Initializes the logger with specific type.
|
||||
* @param loggerType specified one of the values from enum AuditLoggerType.
|
||||
*/
|
||||
private void initializeLogger(AuditLoggerType loggerType){
|
||||
this.logger = LogManager.getContext(false).getLogger(loggerType.getType());
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public ExtendedLogger getLogger() {
|
||||
return logger;
|
||||
}
|
||||
|
||||
public void logWriteSuccess(AuditAction type, Map<String, String> data) {
|
||||
logWriteSuccess(type, data, Level.INFO);
|
||||
}
|
||||
|
||||
public void logWriteSuccess(AuditAction type, Map<String, String> data, Level
|
||||
level) {
|
||||
StructuredDataMessage msg = new StructuredDataMessage("", SUCCESS,
|
||||
type.getAction(), data);
|
||||
this.logger.logIfEnabled(FQCN, level, WRITE_MARKER, msg, null);
|
||||
}
|
||||
|
||||
|
||||
public void logWriteFailure(AuditAction type, Map<String, String> data) {
|
||||
logWriteFailure(type, data, Level.INFO, null);
|
||||
}
|
||||
|
||||
public void logWriteFailure(AuditAction type, Map<String, String> data, Level
|
||||
level) {
|
||||
logWriteFailure(type, data, level, null);
|
||||
}
|
||||
|
||||
public void logWriteFailure(AuditAction type, Map<String, String> data,
|
||||
Throwable exception) {
|
||||
logWriteFailure(type, data, Level.INFO, exception);
|
||||
}
|
||||
|
||||
public void logWriteFailure(AuditAction type, Map<String, String> data, Level
|
||||
level, Throwable exception) {
|
||||
StructuredDataMessage msg = new StructuredDataMessage("", FAILURE,
|
||||
type.getAction(), data);
|
||||
this.logger.logIfEnabled(FQCN, level, WRITE_MARKER, msg, exception);
|
||||
}
|
||||
|
||||
public void logReadSuccess(AuditAction type, Map<String, String> data) {
|
||||
logReadSuccess(type, data, Level.INFO);
|
||||
}
|
||||
|
||||
public void logReadSuccess(AuditAction type, Map<String, String> data, Level
|
||||
level) {
|
||||
StructuredDataMessage msg = new StructuredDataMessage("", SUCCESS,
|
||||
type.getAction(), data);
|
||||
this.logger.logIfEnabled(FQCN, level, READ_MARKER, msg, null);
|
||||
}
|
||||
|
||||
public void logReadFailure(AuditAction type, Map<String, String> data) {
|
||||
logReadFailure(type, data, Level.INFO, null);
|
||||
}
|
||||
|
||||
public void logReadFailure(AuditAction type, Map<String, String> data, Level
|
||||
level) {
|
||||
logReadFailure(type, data, level, null);
|
||||
}
|
||||
|
||||
public void logReadFailure(AuditAction type, Map<String, String> data,
|
||||
Throwable exception) {
|
||||
logReadFailure(type, data, Level.INFO, exception);
|
||||
}
|
||||
|
||||
public void logReadFailure(AuditAction type, Map<String, String> data, Level
|
||||
level, Throwable exception) {
|
||||
StructuredDataMessage msg = new StructuredDataMessage("", FAILURE,
|
||||
type.getAction(), data);
|
||||
this.logger.logIfEnabled(FQCN, level, READ_MARKER, msg, exception);
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,37 @@
|
||||
/**
|
||||
* 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.audit;
|
||||
|
||||
/**
|
||||
* Enumeration for defining types of Audit Loggers in Ozone.
|
||||
*/
|
||||
public enum AuditLoggerType {
|
||||
DNLOGGER("DNAudit"),
|
||||
OMLOGGER("OMAudit"),
|
||||
SCMLOGGER("SCMAudit");
|
||||
|
||||
private String type;
|
||||
|
||||
public String getType() {
|
||||
return type;
|
||||
}
|
||||
|
||||
AuditLoggerType(String type){
|
||||
this.type = type;
|
||||
}
|
||||
}
|
@ -0,0 +1,38 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with this
|
||||
* work for additional information regarding copyright ownership. The ASF
|
||||
* licenses this file to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
* <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.audit;
|
||||
|
||||
import org.apache.logging.log4j.Marker;
|
||||
import org.apache.logging.log4j.MarkerManager;
|
||||
|
||||
/**
|
||||
* Defines audit marker types.
|
||||
*/
|
||||
public enum AuditMarker {
|
||||
WRITE(MarkerManager.getMarker("WRITE")),
|
||||
READ(MarkerManager.getMarker("READ"));
|
||||
|
||||
private Marker marker;
|
||||
|
||||
AuditMarker(Marker marker){
|
||||
this.marker = marker;
|
||||
}
|
||||
|
||||
public Marker getMarker(){
|
||||
return marker;
|
||||
}
|
||||
}
|
@ -0,0 +1,32 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with this
|
||||
* work for additional information regarding copyright ownership. The ASF
|
||||
* licenses this file to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
* <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.audit;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Interface to make an entity auditable.
|
||||
*/
|
||||
public interface Auditable {
|
||||
/**
|
||||
* Must override in implementation.
|
||||
* @return Map<String, String> with values to be logged in audit.
|
||||
*/
|
||||
Map<String, String> toAuditMap();
|
||||
}
|
||||
|
@ -0,0 +1,123 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with this
|
||||
* work for additional information regarding copyright ownership. The ASF
|
||||
* licenses this file to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
* <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.audit;
|
||||
/**
|
||||
******************************************************************************
|
||||
* Important
|
||||
* 1. Any changes to classes in this package can render the logging
|
||||
* framework broken.
|
||||
* 2. The logger framework has been designed keeping in mind future
|
||||
* plans to build a log parser.
|
||||
* 3. Please exercise great caution when attempting changes in this package.
|
||||
******************************************************************************
|
||||
*
|
||||
*
|
||||
* This package lays the foundation for Audit logging in Ozone.
|
||||
* AuditLogging in Ozone has been built using log4j2 which brings in new
|
||||
* features that facilitate turning on/off selective audit events by using
|
||||
* MarkerFilter, checking for change in logging configuration periodically
|
||||
* and reloading the changes, use of disruptor framework for improved
|
||||
* Asynchronous logging.
|
||||
*
|
||||
* The log4j2 configurations can be specified in XML, YAML, JSON and
|
||||
* Properties file. For Ozone, we are using the Properties file due to sheer
|
||||
* simplicity, readability and ease of modification.
|
||||
*
|
||||
* log4j2 configuration file can be passed to startup command with option
|
||||
* -Dlog4j.configurationFile unlike -Dlog4j.configuration in log4j 1.x
|
||||
*
|
||||
******************************************************************************
|
||||
* Understanding the Audit Logging framework in Ozone.
|
||||
******************************************************************************
|
||||
* **** Auditable ***
|
||||
* This is an interface to mark an entity as auditable.
|
||||
* This interface must be implemented by entities requiring audit logging.
|
||||
* For example - KSMVolumeArgs, KSMBucketArgs.
|
||||
* The implementing class must override toAuditMap() to return an
|
||||
* instance of Map<Key, Value> where both Key and Value are String.
|
||||
*
|
||||
* Key: must not contain any spaces. If the key is multi word then use
|
||||
* camel case.
|
||||
* Value: if it is a collection/array, then it must be converted to a comma
|
||||
* delimited string
|
||||
*
|
||||
* *** AuditAction ***
|
||||
* This is an interface to define the various type of actions to be audited.
|
||||
* To ensure separation of concern, for each sub-component you must create an
|
||||
* Enum to implement AuditAction.
|
||||
* Structure of Enum can be referred from the test class DummyAction.
|
||||
*
|
||||
* For starters, we expect following 3 implementations of AuditAction:
|
||||
* OMAction - to define action types for Ozone Manager
|
||||
* SCMAction - to define action types for Storage Container manager
|
||||
* DNAction - to define action types for Datanode
|
||||
*
|
||||
* *** AuditEventStatus ***
|
||||
* Enum to define Audit event status like success and failure.
|
||||
* This is used in AuditLogger.logXXX() methods.
|
||||
*
|
||||
* * *** AuditLogger ***
|
||||
* This is where the audit logging magic unfolds.
|
||||
* The class has 2 Markers defined - READ and WRITE.
|
||||
* These markers are used to tag when logging events.
|
||||
*
|
||||
* *** AuditLoggerType ***
|
||||
* Enum to define the various AuditLoggers in Ozone
|
||||
*
|
||||
* *** AuditMarker ***
|
||||
* Enum to define various Audit Markers used in AuditLogging.
|
||||
*
|
||||
* ****************************************************************************
|
||||
* Usage
|
||||
* ****************************************************************************
|
||||
* Using the AuditLogger to log events:
|
||||
* 1. Get a logger by specifying the appropriate logger type
|
||||
* Example: ExtendedLogger AUDIT = new AuditLogger(AuditLoggerType.OMLogger)
|
||||
*
|
||||
* 2. Log Read/Write and Success/Failure event as needed.
|
||||
* Example
|
||||
* AUDIT.logWriteSuccess(AuditAction type, Map<String, String> data, Level
|
||||
* level)
|
||||
*
|
||||
* If logging is done without specifying Level, then Level implicitly
|
||||
* defaults to INFO
|
||||
* AUDIT.logWriteSuccess(AuditAction type, Map<String, String> data)
|
||||
*
|
||||
* See sample invocations in src/test in the following class:
|
||||
* org.apache.hadoop.ozone.audit.TestOzoneAuditLogger
|
||||
*
|
||||
* ****************************************************************************
|
||||
* Defining new Logger types
|
||||
* ****************************************************************************
|
||||
* New Logger type can be added with following steps:
|
||||
* 1. Update AuditLoggerType to add the new type
|
||||
* 2. Create new Enum by implementing AuditAction if needed
|
||||
* 3. Ensure the required entity implements Auditable
|
||||
*
|
||||
* ****************************************************************************
|
||||
* Defining new Marker types
|
||||
* ****************************************************************************
|
||||
* New Markers can be configured as follows:
|
||||
* 1. Define new markers in AuditMarker
|
||||
* 2. Get the Marker in AuditLogger for use in the log methods, example:
|
||||
* private static final Marker WRITE_MARKER = AuditMarker.WRITE.getMarker();
|
||||
* 3. Define log methods in AuditLogger to use the new Marker type
|
||||
* 4. Call these new methods from the required classes to audit with these
|
||||
* new markers
|
||||
* 5. The marker based filtering can be configured in log4j2 configurations
|
||||
* Refer log4j2.properties in src/test/resources for a sample.
|
||||
*/
|
@ -38,7 +38,7 @@
|
||||
* Local storage information is stored in a separate file VERSION.
|
||||
* It contains type of the node,
|
||||
* the storage layout version, the SCM id, and
|
||||
* the KSM/SCM state creation time.
|
||||
* the OM/SCM state creation time.
|
||||
*
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@ -128,7 +128,7 @@ protected StorageInfo getStorageInfo() {
|
||||
abstract protected Properties getNodeProperties();
|
||||
|
||||
/**
|
||||
* Sets the Node properties spaecific to KSM/SCM.
|
||||
* Sets the Node properties spaecific to OM/SCM.
|
||||
*/
|
||||
private void setNodeProperties() {
|
||||
Properties nodeProperties = getNodeProperties();
|
||||
@ -153,7 +153,7 @@ private File getCurrentDir() {
|
||||
* File {@code VERSION} contains the following fields:
|
||||
* <ol>
|
||||
* <li>node type</li>
|
||||
* <li>KSM/SCM state creation time</li>
|
||||
* <li>OM/SCM state creation time</li>
|
||||
* <li>other fields specific for this node type</li>
|
||||
* </ol>
|
||||
* The version file is always written last during storage directory updates.
|
||||
|
@ -21,7 +21,10 @@
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerWithPipelineRequestProto;
|
||||
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerWithPipelineResponseProto;
|
||||
import org.apache.hadoop.hdds.scm.ScmInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
|
||||
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
|
||||
@ -54,7 +57,6 @@
|
||||
.StorageContainerLocationProtocolProtos.SCMListContainerResponseProto;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
@ -82,10 +84,11 @@ public StorageContainerLocationProtocolServerSideTranslatorPB(
|
||||
public ContainerResponseProto allocateContainer(RpcController unused,
|
||||
ContainerRequestProto request) throws ServiceException {
|
||||
try {
|
||||
ContainerInfo container = impl.allocateContainer(request.getReplicationType(),
|
||||
request.getReplicationFactor(), request.getOwner());
|
||||
ContainerWithPipeline containerWithPipeline = impl
|
||||
.allocateContainer(request.getReplicationType(),
|
||||
request.getReplicationFactor(), request.getOwner());
|
||||
return ContainerResponseProto.newBuilder()
|
||||
.setContainerInfo(container.getProtobuf())
|
||||
.setContainerWithPipeline(containerWithPipeline.getProtobuf())
|
||||
.setErrorCode(ContainerResponseProto.Error.success)
|
||||
.build();
|
||||
|
||||
@ -108,6 +111,21 @@ public GetContainerResponseProto getContainer(
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public GetContainerWithPipelineResponseProto getContainerWithPipeline(
|
||||
RpcController controller, GetContainerWithPipelineRequestProto request)
|
||||
throws ServiceException {
|
||||
try {
|
||||
ContainerWithPipeline container = impl
|
||||
.getContainerWithPipeline(request.getContainerID());
|
||||
return GetContainerWithPipelineResponseProto.newBuilder()
|
||||
.setContainerWithPipeline(container.getProtobuf())
|
||||
.build();
|
||||
} catch (IOException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public SCMListContainerResponseProto listContainer(RpcController controller,
|
||||
SCMListContainerRequestProto request) throws ServiceException {
|
||||
@ -152,13 +170,12 @@ public SCMDeleteContainerResponseProto deleteContainer(
|
||||
StorageContainerLocationProtocolProtos.NodeQueryRequestProto request)
|
||||
throws ServiceException {
|
||||
try {
|
||||
EnumSet<HddsProtos.NodeState> nodeStateEnumSet = EnumSet.copyOf(request
|
||||
.getQueryList());
|
||||
HddsProtos.NodePool datanodes = impl.queryNode(nodeStateEnumSet,
|
||||
HddsProtos.NodeState nodeState = request.getState();
|
||||
List<HddsProtos.Node> datanodes = impl.queryNode(nodeState,
|
||||
request.getScope(), request.getPoolName());
|
||||
return StorageContainerLocationProtocolProtos
|
||||
.NodeQueryResponseProto.newBuilder()
|
||||
.setDatanodes(datanodes)
|
||||
.addAllDatanodes(datanodes)
|
||||
.build();
|
||||
} catch (Exception e) {
|
||||
throw new ServiceException(e);
|
||||
|
@ -46,7 +46,7 @@ message AllocateScmBlockRequestProto {
|
||||
}
|
||||
|
||||
/**
|
||||
* A delete key request sent by KSM to SCM, it contains
|
||||
* A delete key request sent by OM to SCM, it contains
|
||||
* multiple number of keys (and their blocks).
|
||||
*/
|
||||
message DeleteScmKeyBlocksRequestProto {
|
||||
@ -56,9 +56,9 @@ message DeleteScmKeyBlocksRequestProto {
|
||||
/**
|
||||
* A object key and all its associated blocks.
|
||||
* We need to encapsulate object key name plus the blocks in this potocol
|
||||
* because SCM needs to response KSM with the keys it has deleted.
|
||||
* because SCM needs to response OM with the keys it has deleted.
|
||||
* If the response only contains blocks, it will be very expensive for
|
||||
* KSM to figure out what keys have been deleted.
|
||||
* OM to figure out what keys have been deleted.
|
||||
*/
|
||||
message KeyBlocks {
|
||||
required string key = 1;
|
||||
@ -66,7 +66,7 @@ message KeyBlocks {
|
||||
}
|
||||
|
||||
/**
|
||||
* A delete key response from SCM to KSM, it contains multiple child-results.
|
||||
* A delete key response from SCM to OM, it contains multiple child-results.
|
||||
* Each child-result represents a key deletion result, only if all blocks of
|
||||
* a key are successfully deleted, this key result is considered as succeed.
|
||||
*/
|
||||
@ -111,7 +111,7 @@ message AllocateScmBlockResponseProto {
|
||||
}
|
||||
|
||||
/**
|
||||
* Protocol used from KeySpaceManager to StorageContainerManager.
|
||||
* Protocol used from OzoneManager to StorageContainerManager.
|
||||
* See request and response messages for details of the RPC calls.
|
||||
*/
|
||||
service ScmBlockLocationProtocolService {
|
||||
|
@ -52,7 +52,7 @@ message ContainerResponseProto {
|
||||
errorContainerMissing = 3;
|
||||
}
|
||||
required Error errorCode = 1;
|
||||
required SCMContainerInfo containerInfo = 2;
|
||||
required ContainerWithPipeline containerWithPipeline = 2;
|
||||
optional string errorMessage = 3;
|
||||
}
|
||||
|
||||
@ -64,6 +64,14 @@ message GetContainerResponseProto {
|
||||
required SCMContainerInfo containerInfo = 1;
|
||||
}
|
||||
|
||||
message GetContainerWithPipelineRequestProto {
|
||||
required int64 containerID = 1;
|
||||
}
|
||||
|
||||
message GetContainerWithPipelineResponseProto {
|
||||
required ContainerWithPipeline containerWithPipeline = 1;
|
||||
}
|
||||
|
||||
message SCMListContainerRequestProto {
|
||||
required uint32 count = 1;
|
||||
optional uint64 startContainerID = 2;
|
||||
@ -110,26 +118,13 @@ message ObjectStageChangeResponseProto {
|
||||
match the NodeState that we are requesting.
|
||||
*/
|
||||
message NodeQueryRequestProto {
|
||||
|
||||
|
||||
// Repeated, So we can specify more than one status type.
|
||||
// These NodeState types are additive for now, in the sense that
|
||||
// if you specify HEALTHY and FREE_NODE members --
|
||||
// Then you get all healthy node which are not raft members.
|
||||
//
|
||||
// if you specify all healthy and dead nodes, you will get nothing
|
||||
// back. Server is not going to dictate what combinations make sense,
|
||||
// it is entirely up to the caller.
|
||||
// TODO: Support operators like OR and NOT. Currently it is always an
|
||||
// implied AND.
|
||||
|
||||
repeated NodeState query = 1;
|
||||
required NodeState state = 1;
|
||||
required QueryScope scope = 2;
|
||||
optional string poolName = 3; // if scope is pool, then pool name is needed.
|
||||
}
|
||||
|
||||
message NodeQueryResponseProto {
|
||||
required NodePool datanodes = 1;
|
||||
repeated Node datanodes = 1;
|
||||
}
|
||||
|
||||
/**
|
||||
@ -171,6 +166,11 @@ service StorageContainerLocationProtocolService {
|
||||
*/
|
||||
rpc getContainer(GetContainerRequestProto) returns (GetContainerResponseProto);
|
||||
|
||||
/**
|
||||
* Returns the pipeline for a given container.
|
||||
*/
|
||||
rpc getContainerWithPipeline(GetContainerWithPipelineRequestProto) returns (GetContainerWithPipelineResponseProto);
|
||||
|
||||
rpc listContainer(SCMListContainerRequestProto) returns (SCMListContainerResponseProto);
|
||||
|
||||
/**
|
||||
@ -181,7 +181,7 @@ service StorageContainerLocationProtocolService {
|
||||
/**
|
||||
* Returns a set of Nodes that meet a criteria.
|
||||
*/
|
||||
rpc queryNode(NodeQueryRequestProto) returns (NodeQueryResponseProto);
|
||||
rpc queryNode(NodeQueryRequestProto) returns (NodeQueryResponseProto);
|
||||
|
||||
/**
|
||||
* Notify from client when begin or finish container or pipeline operations on datanodes.
|
||||
|
@ -58,9 +58,9 @@ message KeyValue {
|
||||
* Type of the node.
|
||||
*/
|
||||
enum NodeType {
|
||||
KSM = 1;
|
||||
SCM = 2;
|
||||
DATANODE = 3;
|
||||
OM = 1; // Ozone Manager
|
||||
SCM = 2; // Storage Container Manager
|
||||
DATANODE = 3; // DataNode
|
||||
}
|
||||
|
||||
// Should we rename NodeState to DatanodeState?
|
||||
@ -69,14 +69,11 @@ enum NodeType {
|
||||
* and getNodeCount.
|
||||
*/
|
||||
enum NodeState {
|
||||
HEALTHY = 1;
|
||||
STALE = 2;
|
||||
DEAD = 3;
|
||||
DECOMMISSIONING = 4;
|
||||
DECOMMISSIONED = 5;
|
||||
RAFT_MEMBER = 6;
|
||||
FREE_NODE = 7; // Not a member in raft.
|
||||
INVALID = 8;
|
||||
HEALTHY = 1;
|
||||
STALE = 2;
|
||||
DEAD = 3;
|
||||
DECOMMISSIONING = 4;
|
||||
DECOMMISSIONED = 5;
|
||||
}
|
||||
|
||||
enum QueryScope {
|
||||
@ -132,7 +129,7 @@ enum LifeCycleEvent {
|
||||
message SCMContainerInfo {
|
||||
required int64 containerID = 1;
|
||||
required LifeCycleState state = 2;
|
||||
required Pipeline pipeline = 3;
|
||||
optional string pipelineName = 3;
|
||||
// This is not total size of container, but space allocated by SCM for
|
||||
// clients to write blocks
|
||||
required uint64 allocatedBytes = 4;
|
||||
@ -141,6 +138,13 @@ message SCMContainerInfo {
|
||||
optional int64 stateEnterTime = 7;
|
||||
required string owner = 8;
|
||||
optional int64 deleteTransactionId = 9;
|
||||
required ReplicationFactor replicationFactor = 10;
|
||||
required ReplicationType replicationType = 11;
|
||||
}
|
||||
|
||||
message ContainerWithPipeline {
|
||||
required SCMContainerInfo containerInfo = 1;
|
||||
required Pipeline pipeline = 2;
|
||||
}
|
||||
|
||||
message GetScmInfoRequestProto {
|
||||
|
@ -21,7 +21,7 @@
|
||||
<!-- there. If ozone-site.xml does not already exist, create it. -->
|
||||
|
||||
<!--Tags supported are OZONE, CBLOCK, MANAGEMENT, SECURITY, PERFORMANCE, -->
|
||||
<!--DEBUG, CLIENT, SERVER, KSM, SCM, CRITICAL, RATIS, CONTAINER, REQUIRED, -->
|
||||
<!--DEBUG, CLIENT, SERVER, OM, SCM, CRITICAL, RATIS, CONTAINER, REQUIRED, -->
|
||||
<!--REST, STORAGE, PIPELINE, STANDALONE -->
|
||||
|
||||
<configuration>
|
||||
@ -266,122 +266,122 @@
|
||||
<description>
|
||||
Tells ozone which storage handler to use. The possible values are:
|
||||
distributed - The Ozone distributed storage handler, which speaks to
|
||||
KSM/SCM on the backend and provides REST services to clients.
|
||||
OM/SCM on the backend and provides REST services to clients.
|
||||
local - Local Storage handler strictly for testing - To be removed.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.key.deleting.limit.per.task</name>
|
||||
<value>1000</value>
|
||||
<tag>KSM, PERFORMANCE</tag>
|
||||
<tag>OM, PERFORMANCE</tag>
|
||||
<description>
|
||||
A maximum number of keys to be scanned by key deleting service
|
||||
per time interval in KSM. Those keys are sent to delete metadata and
|
||||
per time interval in OM. Those keys are sent to delete metadata and
|
||||
generate transactions in SCM for next async deletion between SCM
|
||||
and DataNode.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.ksm.address</name>
|
||||
<name>ozone.om.address</name>
|
||||
<value/>
|
||||
<tag>KSM, REQUIRED</tag>
|
||||
<tag>OM, REQUIRED</tag>
|
||||
<description>
|
||||
The address of the Ozone KSM service. This allows clients to discover
|
||||
the KSMs address.
|
||||
The address of the Ozone OM service. This allows clients to discover
|
||||
the address of the OM.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.ksm.group.rights</name>
|
||||
<name>ozone.om.group.rights</name>
|
||||
<value>READ_WRITE</value>
|
||||
<tag>KSM, SECURITY</tag>
|
||||
<tag>OM, SECURITY</tag>
|
||||
<description>
|
||||
Default group permissions in Ozone KSM.
|
||||
Default group permissions in Ozone OM.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.ksm.handler.count.key</name>
|
||||
<name>ozone.om.handler.count.key</name>
|
||||
<value>20</value>
|
||||
<tag>KSM, PERFORMANCE</tag>
|
||||
<tag>OM, PERFORMANCE</tag>
|
||||
<description>
|
||||
The number of RPC handler threads for KSM service endpoints.
|
||||
The number of RPC handler threads for OM service endpoints.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.ksm.http-address</name>
|
||||
<name>ozone.om.http-address</name>
|
||||
<value>0.0.0.0:9874</value>
|
||||
<tag>KSM, MANAGEMENT</tag>
|
||||
<tag>OM, MANAGEMENT</tag>
|
||||
<description>
|
||||
The address and the base port where the KSM web UI will listen on.
|
||||
The address and the base port where the OM web UI will listen on.
|
||||
|
||||
If the port is 0, then the server will start on a free port. However, it
|
||||
is best to specify a well-known port, so it is easy to connect and see
|
||||
the KSM management UI.
|
||||
the OM management UI.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.ksm.http-bind-host</name>
|
||||
<name>ozone.om.http-bind-host</name>
|
||||
<value>0.0.0.0</value>
|
||||
<tag>KSM, MANAGEMENT</tag>
|
||||
<tag>OM, MANAGEMENT</tag>
|
||||
<description>
|
||||
The actual address the KSM web server will bind to. If this optional
|
||||
The actual address the OM web server will bind to. If this optional
|
||||
the address is set, it overrides only the hostname portion of
|
||||
ozone.ksm.http-address.
|
||||
ozone.om.http-address.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.ksm.http.enabled</name>
|
||||
<name>ozone.om.http.enabled</name>
|
||||
<value>true</value>
|
||||
<tag>KSM, MANAGEMENT</tag>
|
||||
<tag>OM, MANAGEMENT</tag>
|
||||
<description>
|
||||
Property to enable or disable KSM web user interface.
|
||||
Property to enable or disable OM web user interface.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.ksm.https-address</name>
|
||||
<name>ozone.om.https-address</name>
|
||||
<value>0.0.0.0:9875</value>
|
||||
<tag>KSM, MANAGEMENT, SECURITY</tag>
|
||||
<tag>OM, MANAGEMENT, SECURITY</tag>
|
||||
<description>
|
||||
The address and the base port where the KSM web UI will listen
|
||||
The address and the base port where the OM web UI will listen
|
||||
on using HTTPS.
|
||||
If the port is 0 then the server will start on a free port.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.ksm.https-bind-host</name>
|
||||
<name>ozone.om.https-bind-host</name>
|
||||
<value>0.0.0.0</value>
|
||||
<tag>KSM, MANAGEMENT, SECURITY</tag>
|
||||
<tag>OM, MANAGEMENT, SECURITY</tag>
|
||||
<description>
|
||||
The actual address the KSM web server will bind to using HTTPS.
|
||||
The actual address the OM web server will bind to using HTTPS.
|
||||
If this optional address is set, it overrides only the hostname portion of
|
||||
ozone.ksm.http-address.
|
||||
ozone.om.http-address.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.ksm.keytab.file</name>
|
||||
<name>ozone.om.keytab.file</name>
|
||||
<value/>
|
||||
<tag>KSM, SECURITY</tag>
|
||||
<tag>OM, SECURITY</tag>
|
||||
<description>
|
||||
The keytab file for Kerberos authentication in KSM.
|
||||
The keytab file for Kerberos authentication in OM.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.ksm.db.cache.size.mb</name>
|
||||
<name>ozone.om.db.cache.size.mb</name>
|
||||
<value>128</value>
|
||||
<tag>KSM, PERFORMANCE</tag>
|
||||
<tag>OM, PERFORMANCE</tag>
|
||||
<description>
|
||||
The size of KSM DB cache in MB that used for caching files.
|
||||
The size of OM DB cache in MB that used for caching files.
|
||||
This value is set to an abnormally low value in the default configuration.
|
||||
That is to make unit testing easy. Generally, this value should be set to
|
||||
something like 16GB or more, if you intend to use Ozone at scale.
|
||||
|
||||
A large value for this key allows a proportionally larger amount of KSM
|
||||
metadata to be cached in memory. This makes KSM operations faster.
|
||||
A large value for this key allows a proportionally larger amount of OM
|
||||
metadata to be cached in memory. This makes OM operations faster.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.ksm.user.max.volume</name>
|
||||
<name>ozone.om.user.max.volume</name>
|
||||
<value>1024</value>
|
||||
<tag>KSM, MANAGEMENT</tag>
|
||||
<tag>OM, MANAGEMENT</tag>
|
||||
<description>
|
||||
The maximum number of volumes a user can have on a cluster.Increasing or
|
||||
decreasing this number has no real impact on ozone cluster. This is
|
||||
@ -391,11 +391,11 @@
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.ksm.user.rights</name>
|
||||
<name>ozone.om.user.rights</name>
|
||||
<value>READ_WRITE</value>
|
||||
<tag>KSM, SECURITY</tag>
|
||||
<tag>OM, SECURITY</tag>
|
||||
<description>
|
||||
Default user permissions used in KSM.
|
||||
Default user permissions used in OM.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
@ -405,20 +405,20 @@
|
||||
<description>
|
||||
This is used only for testing purposes. This value is used by the local
|
||||
storage handler to simulate a REST backend. This is useful only when
|
||||
debugging the REST front end independent of KSM and SCM. To be removed.
|
||||
debugging the REST front end independent of OM and SCM. To be removed.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.metadata.dirs</name>
|
||||
<value/>
|
||||
<tag>OZONE, KSM, SCM, CONTAINER, REQUIRED, STORAGE</tag>
|
||||
<tag>OZONE, OM, SCM, CONTAINER, REQUIRED, STORAGE</tag>
|
||||
<description>
|
||||
Ozone metadata is shared among KSM, which acts as the namespace
|
||||
Ozone metadata is shared among OM, which acts as the namespace
|
||||
manager for ozone, SCM which acts as the block manager and data nodes
|
||||
which maintain the name of the key(Key Name and BlockIDs). This
|
||||
replicated and distributed metadata store is maintained under the
|
||||
directory pointed by this key. Since metadata can be I/O intensive, at
|
||||
least on KSM and SCM we recommend having SSDs. If you have the luxury
|
||||
least on OM and SCM we recommend having SSDs. If you have the luxury
|
||||
of mapping this path to SSDs on all machines in the cluster, that will
|
||||
be excellent.
|
||||
|
||||
@ -429,10 +429,10 @@
|
||||
<property>
|
||||
<name>ozone.metastore.impl</name>
|
||||
<value>RocksDB</value>
|
||||
<tag>OZONE, KSM, SCM, CONTAINER, STORAGE</tag>
|
||||
<tag>OZONE, OM, SCM, CONTAINER, STORAGE</tag>
|
||||
<description>
|
||||
Ozone metadata store implementation. Ozone metadata are well
|
||||
distributed to multiple services such as ksm, scm. They are stored in
|
||||
distributed to multiple services such as ozoneManager, scm. They are stored in
|
||||
some local key-value databases. This property determines which database
|
||||
library to use. Supported value is either LevelDB or RocksDB.
|
||||
</description>
|
||||
@ -441,7 +441,7 @@
|
||||
<property>
|
||||
<name>ozone.metastore.rocksdb.statistics</name>
|
||||
<value>ALL</value>
|
||||
<tag>OZONE, KSM, SCM, STORAGE, PERFORMANCE</tag>
|
||||
<tag>OZONE, OM, SCM, STORAGE, PERFORMANCE</tag>
|
||||
<description>
|
||||
The statistics level of the rocksdb store. If you use any value from
|
||||
org.rocksdb.StatsLevel (eg. ALL or EXCEPT_DETAILED_TIMERS), the rocksdb
|
||||
@ -684,7 +684,7 @@
|
||||
The heartbeat interval from a data node to SCM. Yes,
|
||||
it is not three but 30, since most data nodes will heart beating via Ratis
|
||||
heartbeats. If a client is not able to talk to a data node, it will notify
|
||||
KSM/SCM eventually. So a 30 second HB seems to work. This assumes that
|
||||
OM/SCM eventually. So a 30 second HB seems to work. This assumes that
|
||||
replication strategy used is Ratis if not, this value should be set to
|
||||
something smaller like 3 seconds.
|
||||
</description>
|
||||
@ -785,17 +785,6 @@
|
||||
The keytab file for Kerberos authentication in SCM.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.scm.max.hb.count.to.process</name>
|
||||
<value>5000</value>
|
||||
<tag>OZONE, MANAGEMENT, PERFORMANCE</tag>
|
||||
<description>
|
||||
The maximum number of heartbeat to process per loop of the
|
||||
heartbeat process thread. Please see
|
||||
ozone.scm.heartbeat.thread.interval
|
||||
for more info.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.scm.names</name>
|
||||
<value/>
|
||||
@ -831,7 +820,7 @@
|
||||
<value/>
|
||||
<tag>OZONE, SECURITY</tag>
|
||||
<description>
|
||||
The server principal used by the SCM and KSM for web UI SPNEGO
|
||||
The server principal used by the SCM and OM for web UI SPNEGO
|
||||
authentication when Kerberos security is enabled. This is typically set to
|
||||
HTTP/_HOST@REALM.TLD The SPNEGO server principal begins with the prefix
|
||||
HTTP/ by convention.
|
||||
@ -890,9 +879,9 @@
|
||||
<property>
|
||||
<name>ozone.key.preallocation.maxsize</name>
|
||||
<value>134217728</value>
|
||||
<tag>OZONE, KSM, PERFORMANCE</tag>
|
||||
<tag>OZONE, OM, PERFORMANCE</tag>
|
||||
<description>
|
||||
When a new key write request is sent to KSM, if a size is requested, at most
|
||||
When a new key write request is sent to OM, if a size is requested, at most
|
||||
128MB of size is allocated at request time. If client needs more space for the
|
||||
write, separate block allocation requests will be made.
|
||||
</description>
|
||||
@ -961,7 +950,7 @@
|
||||
<property>
|
||||
<name>ozone.open.key.cleanup.service.interval.seconds</name>
|
||||
<value>86400</value>
|
||||
<tag>OZONE, KSM, PERFORMANCE</tag>
|
||||
<tag>OZONE, OM, PERFORMANCE</tag>
|
||||
<description>
|
||||
A background job periodically checks open key entries and delete the expired ones. This entry controls the
|
||||
interval of this cleanup check.
|
||||
@ -971,7 +960,7 @@
|
||||
<property>
|
||||
<name>ozone.open.key.expire.threshold</name>
|
||||
<value>86400</value>
|
||||
<tag>OZONE, KSM, PERFORMANCE</tag>
|
||||
<tag>OZONE, OM, PERFORMANCE</tag>
|
||||
<description>
|
||||
Controls how long an open key operation is considered active. Specifically, if a key
|
||||
has been open longer than the value of this config entry, that open key is considered as
|
||||
@ -981,12 +970,12 @@
|
||||
|
||||
<property>
|
||||
<name>hadoop.tags.custom</name>
|
||||
<value>OZONE,MANAGEMENT,SECURITY,PERFORMANCE,DEBUG,CLIENT,SERVER,KSM,SCM,CRITICAL,RATIS,CONTAINER,REQUIRED,REST,STORAGE,PIPELINE,STANDALONE</value>
|
||||
<value>OZONE,MANAGEMENT,SECURITY,PERFORMANCE,DEBUG,CLIENT,SERVER,OM,SCM,CRITICAL,RATIS,CONTAINER,REQUIRED,REST,STORAGE,PIPELINE,STANDALONE</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>ozone.tags.system</name>
|
||||
<value>OZONE,MANAGEMENT,SECURITY,PERFORMANCE,DEBUG,CLIENT,SERVER,KSM,SCM,CRITICAL,RATIS,CONTAINER,REQUIRED,REST,STORAGE,PIPELINE,STANDALONE</value>
|
||||
<value>OZONE,MANAGEMENT,SECURITY,PERFORMANCE,DEBUG,CLIENT,SERVER,OM,SCM,CRITICAL,RATIS,CONTAINER,REQUIRED,REST,STORAGE,PIPELINE,STANDALONE</value>
|
||||
</property>
|
||||
|
||||
|
||||
|
@ -0,0 +1,51 @@
|
||||
/**
|
||||
* 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.audit;
|
||||
|
||||
/**
|
||||
* Enum to define Dummy AuditAction Type for test.
|
||||
*/
|
||||
public enum DummyAction implements AuditAction {
|
||||
|
||||
CREATE_VOLUME("CREATE_VOLUME"),
|
||||
CREATE_BUCKET("CREATE_BUCKET"),
|
||||
CREATE_KEY("CREATE_KEY"),
|
||||
READ_VOLUME("READ_VOLUME"),
|
||||
READ_BUCKET("READ_BUCKET"),
|
||||
READ_KEY("READ_BUCKET"),
|
||||
UPDATE_VOLUME("UPDATE_VOLUME"),
|
||||
UPDATE_BUCKET("UPDATE_BUCKET"),
|
||||
UPDATE_KEY("UPDATE_KEY"),
|
||||
DELETE_VOLUME("DELETE_VOLUME"),
|
||||
DELETE_BUCKET("DELETE_BUCKET"),
|
||||
DELETE_KEY("DELETE_KEY"),
|
||||
SET_OWNER("SET_OWNER"),
|
||||
SET_QUOTA("SET_QUOTA");
|
||||
|
||||
private String action;
|
||||
|
||||
DummyAction(String action) {
|
||||
this.action = action;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getAction() {
|
||||
return this.action;
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,57 @@
|
||||
/**
|
||||
* 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.audit;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* DummyEntity that implements Auditable for test purpose.
|
||||
*/
|
||||
public class DummyEntity implements Auditable {
|
||||
|
||||
private String key1;
|
||||
private String key2;
|
||||
|
||||
public DummyEntity(){
|
||||
this.key1 = "value1";
|
||||
this.key2 = "value2";
|
||||
}
|
||||
public String getKey1() {
|
||||
return key1;
|
||||
}
|
||||
|
||||
public void setKey1(String key1) {
|
||||
this.key1 = key1;
|
||||
}
|
||||
|
||||
public String getKey2() {
|
||||
return key2;
|
||||
}
|
||||
|
||||
public void setKey2(String key2) {
|
||||
this.key2 = key2;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, String> toAuditMap() {
|
||||
Map<String, String> auditMap = new HashMap<>();
|
||||
auditMap.put("key1", this.key1);
|
||||
auditMap.put("key2", this.key2);
|
||||
return auditMap;
|
||||
}
|
||||
}
|
@ -0,0 +1,147 @@
|
||||
/**
|
||||
* 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.audit;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.logging.log4j.Level;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
/**
|
||||
* Test Ozone Audit Logger.
|
||||
*/
|
||||
public class TestOzoneAuditLogger {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger
|
||||
(TestOzoneAuditLogger.class.getName());
|
||||
private static AuditLogger AUDIT = new AuditLogger(AuditLoggerType.OMLOGGER);
|
||||
public DummyEntity auditableObj = new DummyEntity();
|
||||
|
||||
@BeforeClass
|
||||
public static void setUp(){
|
||||
System.setProperty("log4j.configurationFile", "log4j2.properties");
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
File file = new File("audit.log");
|
||||
if (FileUtils.deleteQuietly(file)) {
|
||||
LOG.info(file.getName() +
|
||||
" has been deleted as all tests have completed.");
|
||||
} else {
|
||||
LOG.info("audit.log could not be deleted.");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensures WriteSuccess events are logged @ INFO and above.
|
||||
*/
|
||||
@Test
|
||||
public void logInfoWriteSuccess() throws IOException {
|
||||
AUDIT.logWriteSuccess(DummyAction.CREATE_VOLUME, auditableObj.toAuditMap(), Level.INFO);
|
||||
String expected = "[INFO ] OMAudit - CREATE_VOLUME [ key1=\"value1\" " +
|
||||
"key2=\"value2\"] SUCCESS";
|
||||
verifyLog(expected);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test to verify default log level is INFO
|
||||
*/
|
||||
@Test
|
||||
public void verifyDefaultLogLevel() throws IOException {
|
||||
AUDIT.logWriteSuccess(DummyAction.CREATE_VOLUME, auditableObj.toAuditMap());
|
||||
String expected = "[INFO ] OMAudit - CREATE_VOLUME [ key1=\"value1\" " +
|
||||
"key2=\"value2\"] SUCCESS";
|
||||
verifyLog(expected);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test to verify WriteFailure events are logged as ERROR.
|
||||
*/
|
||||
@Test
|
||||
public void logErrorWriteFailure() throws IOException {
|
||||
AUDIT.logWriteFailure(DummyAction.CREATE_VOLUME, auditableObj.toAuditMap(), Level.ERROR);
|
||||
String expected = "[ERROR] OMAudit - CREATE_VOLUME [ key1=\"value1\" " +
|
||||
"key2=\"value2\"] FAILURE";
|
||||
verifyLog(expected);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test to verify no READ event is logged.
|
||||
*/
|
||||
@Test
|
||||
public void notLogReadEvents() throws IOException {
|
||||
AUDIT.logReadSuccess(DummyAction.READ_VOLUME, auditableObj.toAuditMap(), Level.INFO);
|
||||
AUDIT.logReadFailure(DummyAction.READ_VOLUME, auditableObj.toAuditMap(), Level.INFO);
|
||||
AUDIT.logReadFailure(DummyAction.READ_VOLUME, auditableObj.toAuditMap(), Level.ERROR);
|
||||
AUDIT.logReadFailure(DummyAction.READ_VOLUME, auditableObj.toAuditMap(), Level.ERROR,
|
||||
new Exception("test"));
|
||||
verifyNoLog();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test to ensure DEBUG level messages are not logged when INFO is enabled.
|
||||
*/
|
||||
@Test
|
||||
public void notLogDebugEvents() throws IOException {
|
||||
AUDIT.logWriteSuccess(DummyAction.CREATE_VOLUME, auditableObj.toAuditMap(), Level.DEBUG);
|
||||
AUDIT.logReadSuccess(DummyAction.READ_VOLUME, auditableObj.toAuditMap(), Level.DEBUG);
|
||||
verifyNoLog();
|
||||
}
|
||||
|
||||
private void verifyLog(String expected) throws IOException {
|
||||
File file = new File("audit.log");
|
||||
List<String> lines = FileUtils.readLines(file, (String)null);
|
||||
final int retry = 5;
|
||||
int i = 0;
|
||||
while (lines.isEmpty() && i < retry) {
|
||||
lines = FileUtils.readLines(file, (String)null);
|
||||
try {
|
||||
Thread.sleep( 500 * (i + 1));
|
||||
} catch(InterruptedException ie) {
|
||||
Thread.currentThread().interrupt();
|
||||
break;
|
||||
}
|
||||
i++;
|
||||
}
|
||||
|
||||
// When log entry is expected, the log file will contain one line and
|
||||
// that must be equal to the expected string
|
||||
assertTrue(lines.size() != 0);
|
||||
assertTrue(expected.equalsIgnoreCase(lines.get(0)));
|
||||
//empty the file
|
||||
lines.remove(0);
|
||||
FileUtils.writeLines(file, lines, false);
|
||||
}
|
||||
|
||||
private void verifyNoLog() throws IOException {
|
||||
File file = new File("audit.log");
|
||||
List<String> lines = FileUtils.readLines(file, (String)null);
|
||||
// When no log entry is expected, the log file must be empty
|
||||
assertTrue(lines.size() == 0);
|
||||
}
|
||||
}
|
@ -0,0 +1,23 @@
|
||||
/**
|
||||
* 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.audit;
|
||||
/**
|
||||
* Unit tests of Ozone Audit Logger.
|
||||
* For test purpose, the log4j2 configuration is loaded from file at:
|
||||
* src/test/resources/log4j2.properties
|
||||
*/
|
76
hadoop-hdds/common/src/test/resources/log4j2.properties
Normal file
76
hadoop-hdds/common/src/test/resources/log4j2.properties
Normal file
@ -0,0 +1,76 @@
|
||||
#
|
||||
# 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.
|
||||
#
|
||||
name=PropertiesConfig
|
||||
|
||||
# Checks for config change periodically and reloads
|
||||
monitorInterval=5
|
||||
|
||||
filter=read, write
|
||||
# filter.read.onMatch = DENY avoids logging all READ events
|
||||
# filter.read.onMatch = ACCEPT permits logging all READ events
|
||||
# The above two settings ignore the log levels in configuration
|
||||
# filter.read.onMatch = NEUTRAL permits logging of only those READ events
|
||||
# which are attempted at log level equal or greater than log level specified
|
||||
# in the configuration
|
||||
filter.read.type = MarkerFilter
|
||||
filter.read.marker = READ
|
||||
filter.read.onMatch = DENY
|
||||
filter.read.onMismatch = NEUTRAL
|
||||
|
||||
# filter.write.onMatch = DENY avoids logging all WRITE events
|
||||
# filter.write.onMatch = ACCEPT permits logging all WRITE events
|
||||
# The above two settings ignore the log levels in configuration
|
||||
# filter.write.onMatch = NEUTRAL permits logging of only those WRITE events
|
||||
# which are attempted at log level equal or greater than log level specified
|
||||
# in the configuration
|
||||
filter.write.type = MarkerFilter
|
||||
filter.write.marker = WRITE
|
||||
filter.write.onMatch = NEUTRAL
|
||||
filter.write.onMismatch = NEUTRAL
|
||||
|
||||
# Log Levels are organized from most specific to least:
|
||||
# OFF (most specific, no logging)
|
||||
# FATAL (most specific, little data)
|
||||
# ERROR
|
||||
# WARN
|
||||
# INFO
|
||||
# DEBUG
|
||||
# TRACE (least specific, a lot of data)
|
||||
# ALL (least specific, all data)
|
||||
|
||||
appenders = console, audit
|
||||
appender.console.type = Console
|
||||
appender.console.name = STDOUT
|
||||
appender.console.layout.type = PatternLayout
|
||||
appender.console.layout.pattern = [%-5level] %c{1} - %msg%n
|
||||
|
||||
appender.audit.type = File
|
||||
appender.audit.name = AUDITLOG
|
||||
appender.audit.fileName=audit.log
|
||||
appender.audit.layout.type=PatternLayout
|
||||
appender.audit.layout.pattern= [%-5level] %c{1} - %msg%n
|
||||
|
||||
loggers=audit
|
||||
logger.audit.type=AsyncLogger
|
||||
logger.audit.name=OMAudit
|
||||
logger.audit.level = INFO
|
||||
logger.audit.appenderRefs = audit
|
||||
logger.audit.appenderRef.file.ref = AUDITLOG
|
||||
|
||||
rootLogger.level = INFO
|
||||
rootLogger.appenderRefs = stdout
|
||||
rootLogger.appenderRef.stdout.ref = STDOUT
|
@ -258,17 +258,6 @@ public static long getDeadNodeInterval(Configuration conf) {
|
||||
return deadNodeIntervalMs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the maximum number of heartbeat to process per loop of the process
|
||||
* thread.
|
||||
* @param conf Configuration
|
||||
* @return - int -- Number of HBs to process
|
||||
*/
|
||||
public static int getMaxHBToProcessPerLoop(Configuration conf) {
|
||||
return conf.getInt(ScmConfigKeys.OZONE_SCM_MAX_HB_COUNT_TO_PROCESS,
|
||||
ScmConfigKeys.OZONE_SCM_MAX_HB_COUNT_TO_PROCESS_DEFAULT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Timeout value for the RPC from Datanode to SCM, primarily used for
|
||||
* Heartbeats and container reports.
|
||||
|
@ -34,7 +34,6 @@
|
||||
import org.apache.hadoop.ozone.container.common.helpers
|
||||
.DeletedContainerBlocksSummary;
|
||||
import org.apache.hadoop.ozone.container.common.interfaces.Container;
|
||||
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
|
||||
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
|
||||
import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
|
||||
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
|
||||
@ -180,6 +179,13 @@ private void deleteKeyValueContainerBlocks(
|
||||
containerData.getMetadataPath());
|
||||
}
|
||||
|
||||
if (delTX.getTxID() < containerData.getDeleteTransactionId()) {
|
||||
LOG.debug(String.format("Ignoring delete blocks for containerId: %d."
|
||||
+ " Outdated delete transactionId %d < %d", containerId,
|
||||
delTX.getTxID(), containerData.getDeleteTransactionId()));
|
||||
return;
|
||||
}
|
||||
|
||||
int newDeletionBlocks = 0;
|
||||
MetadataStore containerDB = KeyUtils.getDB(containerData, conf);
|
||||
for (Long blk : delTX.getLocalIDList()) {
|
||||
@ -187,10 +193,20 @@ private void deleteKeyValueContainerBlocks(
|
||||
byte[] blkBytes = Longs.toByteArray(blk);
|
||||
byte[] blkInfo = containerDB.get(blkBytes);
|
||||
if (blkInfo != null) {
|
||||
byte[] deletingKeyBytes =
|
||||
DFSUtil.string2Bytes(OzoneConsts.DELETING_KEY_PREFIX + blk);
|
||||
byte[] deletedKeyBytes =
|
||||
DFSUtil.string2Bytes(OzoneConsts.DELETED_KEY_PREFIX + blk);
|
||||
if (containerDB.get(deletingKeyBytes) != null
|
||||
|| containerDB.get(deletedKeyBytes) != null) {
|
||||
LOG.debug(String.format(
|
||||
"Ignoring delete for block %d in container %d."
|
||||
+ " Entry already added.", blk, containerId));
|
||||
continue;
|
||||
}
|
||||
// Found the block in container db,
|
||||
// use an atomic update to change its state to deleting.
|
||||
batch.put(DFSUtil.string2Bytes(OzoneConsts.DELETING_KEY_PREFIX + blk),
|
||||
blkInfo);
|
||||
batch.put(deletingKeyBytes, blkInfo);
|
||||
batch.delete(blkBytes);
|
||||
try {
|
||||
containerDB.writeBatch(batch);
|
||||
@ -208,11 +224,13 @@ private void deleteKeyValueContainerBlocks(
|
||||
LOG.debug("Block {} not found or already under deletion in"
|
||||
+ " container {}, skip deleting it.", blk, containerId);
|
||||
}
|
||||
containerDB.put(DFSUtil.string2Bytes(
|
||||
OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX + containerId),
|
||||
Longs.toByteArray(delTX.getTxID()));
|
||||
}
|
||||
|
||||
containerDB.put(DFSUtil.string2Bytes(
|
||||
OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX + delTX.getContainerID()),
|
||||
Longs.toByteArray(delTX.getTxID()));
|
||||
containerData
|
||||
.updateDeleteTransactionId(delTX.getTxID());
|
||||
// update pending deletion blocks count in in-memory container status
|
||||
containerData.incrPendingDeletionBlocks(newDeletionBlocks);
|
||||
}
|
||||
|
@ -59,10 +59,8 @@ RegisteredCommand register(DatanodeDetails datanodeDetails,
|
||||
/**
|
||||
* Send heartbeat to indicate the datanode is alive and doing well.
|
||||
* @param datanodeDetails - Datanode ID.
|
||||
* @param nodeReport - node report.
|
||||
* @return SCMheartbeat response list
|
||||
*/
|
||||
List<SCMCommand> sendHeartbeat(DatanodeDetails datanodeDetails,
|
||||
NodeReportProto nodeReport);
|
||||
List<SCMCommand> processHeartbeat(DatanodeDetails datanodeDetails);
|
||||
|
||||
}
|
||||
|
@ -270,7 +270,7 @@
|
||||
$http.get("conf?cmd=getOzoneTags")
|
||||
.then(function(response) {
|
||||
ctrl.tags = response.data;
|
||||
var excludedTags = ['CBLOCK', 'KSM', 'SCM'];
|
||||
var excludedTags = ['CBLOCK', 'OM', 'SCM'];
|
||||
for (var i = 0; i < excludedTags.length; i++) {
|
||||
var idx = ctrl.tags.indexOf(excludedTags[i]);
|
||||
// Remove CBLOCK related properties
|
||||
@ -302,7 +302,7 @@
|
||||
}
|
||||
|
||||
ctrl.loadAll = function() {
|
||||
$http.get("conf?cmd=getPropertyByTag&tags=KSM,SCM," + ctrl.tags)
|
||||
$http.get("conf?cmd=getPropertyByTag&tags=OM,SCM," + ctrl.tags)
|
||||
.then(function(response) {
|
||||
|
||||
ctrl.convertToArray(response.data);
|
||||
|
@ -27,8 +27,8 @@
|
||||
ng-click="$ctrl.switchto('All')">All
|
||||
</a>
|
||||
<a class="btn"
|
||||
ng-class="$ctrl.allSelected('KSM') ? 'btn-primary' :'btn-secondary'"
|
||||
ng-click="$ctrl.switchto('KSM')">KSM</a>
|
||||
ng-class="$ctrl.allSelected('OM') ? 'btn-primary' :'btn-secondary'"
|
||||
ng-click="$ctrl.switchto('OM')">OM</a>
|
||||
<a class="btn"
|
||||
ng-class="$ctrl.allSelected('SCM') ? 'btn-primary' :'btn-secondary'"
|
||||
ng-click="$ctrl.switchto('SCM')">SCM</a>
|
||||
|
@ -16,10 +16,12 @@
|
||||
*/
|
||||
package org.apache.hadoop.hdds.scm.block;
|
||||
|
||||
import java.util.UUID;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||
import org.apache.hadoop.hdds.scm.container.Mapping;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
||||
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
||||
@ -156,13 +158,13 @@ private void preAllocateContainers(int count, ReplicationType type,
|
||||
lock.lock();
|
||||
try {
|
||||
for (int i = 0; i < count; i++) {
|
||||
ContainerInfo containerInfo = null;
|
||||
ContainerWithPipeline containerWithPipeline = null;
|
||||
try {
|
||||
// TODO: Fix this later when Ratis is made the Default.
|
||||
containerInfo = containerManager.allocateContainer(type, factor,
|
||||
containerWithPipeline = containerManager.allocateContainer(type, factor,
|
||||
owner);
|
||||
|
||||
if (containerInfo == null) {
|
||||
if (containerWithPipeline == null) {
|
||||
LOG.warn("Unable to allocate container.");
|
||||
continue;
|
||||
}
|
||||
@ -231,30 +233,27 @@ public AllocatedBlock allocateBlock(final long size,
|
||||
can use different kind of policies.
|
||||
*/
|
||||
|
||||
ContainerInfo containerInfo;
|
||||
ContainerWithPipeline containerWithPipeline;
|
||||
|
||||
// Look for ALLOCATED container that matches all other parameters.
|
||||
containerInfo =
|
||||
containerManager
|
||||
.getStateManager()
|
||||
.getMatchingContainer(
|
||||
size, owner, type, factor, HddsProtos.LifeCycleState
|
||||
.ALLOCATED);
|
||||
if (containerInfo != null) {
|
||||
containerManager.updateContainerState(containerInfo.getContainerID(),
|
||||
containerWithPipeline = containerManager
|
||||
.getMatchingContainerWithPipeline(size, owner, type, factor,
|
||||
HddsProtos.LifeCycleState.ALLOCATED);
|
||||
if (containerWithPipeline != null) {
|
||||
containerManager.updateContainerState(
|
||||
containerWithPipeline.getContainerInfo().getContainerID(),
|
||||
HddsProtos.LifeCycleEvent.CREATE);
|
||||
return newBlock(containerInfo, HddsProtos.LifeCycleState.ALLOCATED);
|
||||
return newBlock(containerWithPipeline,
|
||||
HddsProtos.LifeCycleState.ALLOCATED);
|
||||
}
|
||||
|
||||
// Since we found no allocated containers that match our criteria, let us
|
||||
// look for OPEN containers that match the criteria.
|
||||
containerInfo =
|
||||
containerManager
|
||||
.getStateManager()
|
||||
.getMatchingContainer(size, owner, type, factor, HddsProtos
|
||||
.LifeCycleState.OPEN);
|
||||
if (containerInfo != null) {
|
||||
return newBlock(containerInfo, HddsProtos.LifeCycleState.OPEN);
|
||||
containerWithPipeline = containerManager
|
||||
.getMatchingContainerWithPipeline(size, owner, type, factor,
|
||||
HddsProtos.LifeCycleState.OPEN);
|
||||
if (containerWithPipeline != null) {
|
||||
return newBlock(containerWithPipeline, HddsProtos.LifeCycleState.OPEN);
|
||||
}
|
||||
|
||||
// We found neither ALLOCATED or OPEN Containers. This generally means
|
||||
@ -264,16 +263,15 @@ public AllocatedBlock allocateBlock(final long size,
|
||||
preAllocateContainers(containerProvisionBatchSize, type, factor, owner);
|
||||
|
||||
// Since we just allocated a set of containers this should work
|
||||
containerInfo =
|
||||
containerManager
|
||||
.getStateManager()
|
||||
.getMatchingContainer(
|
||||
size, owner, type, factor, HddsProtos.LifeCycleState
|
||||
.ALLOCATED);
|
||||
if (containerInfo != null) {
|
||||
containerManager.updateContainerState(containerInfo.getContainerID(),
|
||||
containerWithPipeline = containerManager
|
||||
.getMatchingContainerWithPipeline(size, owner, type, factor,
|
||||
HddsProtos.LifeCycleState.ALLOCATED);
|
||||
if (containerWithPipeline != null) {
|
||||
containerManager.updateContainerState(
|
||||
containerWithPipeline.getContainerInfo().getContainerID(),
|
||||
HddsProtos.LifeCycleEvent.CREATE);
|
||||
return newBlock(containerInfo, HddsProtos.LifeCycleState.ALLOCATED);
|
||||
return newBlock(containerWithPipeline,
|
||||
HddsProtos.LifeCycleState.ALLOCATED);
|
||||
}
|
||||
|
||||
// we have tried all strategies we know and but somehow we are not able
|
||||
@ -290,18 +288,28 @@ public AllocatedBlock allocateBlock(final long size,
|
||||
}
|
||||
}
|
||||
|
||||
private String getChannelName(ReplicationType type) {
|
||||
switch (type) {
|
||||
case RATIS:
|
||||
return "RA" + UUID.randomUUID().toString().substring(3);
|
||||
case STAND_ALONE:
|
||||
return "SA" + UUID.randomUUID().toString().substring(3);
|
||||
default:
|
||||
return "RA" + UUID.randomUUID().toString().substring(3);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* newBlock - returns a new block assigned to a container.
|
||||
*
|
||||
* @param containerInfo - Container Info.
|
||||
* @param containerWithPipeline - Container Info.
|
||||
* @param state - Current state of the container.
|
||||
* @return AllocatedBlock
|
||||
*/
|
||||
private AllocatedBlock newBlock(
|
||||
ContainerInfo containerInfo, HddsProtos.LifeCycleState state)
|
||||
throws IOException {
|
||||
|
||||
if (containerInfo.getPipeline().getMachines().size() == 0) {
|
||||
private AllocatedBlock newBlock(ContainerWithPipeline containerWithPipeline,
|
||||
HddsProtos.LifeCycleState state) throws IOException {
|
||||
ContainerInfo containerInfo = containerWithPipeline.getContainerInfo();
|
||||
if (containerWithPipeline.getPipeline().getDatanodes().size() == 0) {
|
||||
LOG.error("Pipeline Machine count is zero.");
|
||||
return null;
|
||||
}
|
||||
@ -317,7 +325,7 @@ private AllocatedBlock newBlock(
|
||||
AllocatedBlock.Builder abb =
|
||||
new AllocatedBlock.Builder()
|
||||
.setBlockID(new BlockID(containerID, localID))
|
||||
.setPipeline(containerInfo.getPipeline())
|
||||
.setPipeline(containerWithPipeline.getPipeline())
|
||||
.setShouldCreateContainer(createContainer);
|
||||
LOG.trace("New block allocated : {} Container ID: {}", localID,
|
||||
containerID);
|
||||
|
@ -18,7 +18,6 @@
|
||||
|
||||
import com.google.common.collect.ArrayListMultimap;
|
||||
import org.apache.hadoop.hdds.scm.container.Mapping;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.proto
|
||||
.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
|
||||
@ -29,6 +28,7 @@
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||
|
||||
/**
|
||||
* A wrapper class to hold info about datanode and all deleted block
|
||||
@ -54,21 +54,22 @@ public class DatanodeDeletedBlockTransactions {
|
||||
}
|
||||
|
||||
public void addTransaction(DeletedBlocksTransaction tx) throws IOException {
|
||||
ContainerInfo info = null;
|
||||
Pipeline pipeline = null;
|
||||
try {
|
||||
info = mappingService.getContainer(tx.getContainerID());
|
||||
pipeline = mappingService.getContainerWithPipeline(tx.getContainerID())
|
||||
.getPipeline();
|
||||
} catch (IOException e) {
|
||||
SCMBlockDeletingService.LOG.warn("Got container info error.", e);
|
||||
}
|
||||
|
||||
if (info == null) {
|
||||
if (pipeline == null) {
|
||||
SCMBlockDeletingService.LOG.warn(
|
||||
"Container {} not found, continue to process next",
|
||||
tx.getContainerID());
|
||||
return;
|
||||
}
|
||||
|
||||
for (DatanodeDetails dd : info.getPipeline().getMachines()) {
|
||||
for (DatanodeDetails dd : pipeline.getMachines()) {
|
||||
UUID dnID = dd.getUuid();
|
||||
if (transactions.containsKey(dnID)) {
|
||||
List<DeletedBlocksTransaction> txs = transactions.get(dnID);
|
||||
|
@ -28,7 +28,7 @@
|
||||
/**
|
||||
* The DeletedBlockLog is a persisted log in SCM to keep tracking
|
||||
* container blocks which are under deletion. It maintains info
|
||||
* about under-deletion container blocks that notified by KSM,
|
||||
* about under-deletion container blocks that notified by OM,
|
||||
* and the state how it is processed.
|
||||
*/
|
||||
public interface DeletedBlockLog extends Closeable {
|
||||
|
@ -16,21 +16,22 @@
|
||||
*/
|
||||
package org.apache.hadoop.hdds.scm.container;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
||||
import org.apache.hadoop.hdds.server.events.EventHandler;
|
||||
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
||||
import org.apache.hadoop.hdds.server.events.TypedEvent;
|
||||
import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* In case of a node failure, volume failure, volume out of spapce, node
|
||||
* out of space etc, CLOSE_CONTAINER_EVENT will be triggered.
|
||||
* CloseContainerEventHandler is the handler for CLOSE_CONTAINER_EVENT.
|
||||
* out of space etc, CLOSE_CONTAINER will be triggered.
|
||||
* CloseContainerEventHandler is the handler for CLOSE_CONTAINER.
|
||||
* When a close container event is fired, a close command for the container
|
||||
* should be sent to all the datanodes in the pipeline and containerStateManager
|
||||
* needs to update the container state to Closing.
|
||||
@ -40,8 +41,6 @@ public class CloseContainerEventHandler implements EventHandler<ContainerID> {
|
||||
public static final Logger LOG =
|
||||
LoggerFactory.getLogger(CloseContainerEventHandler.class);
|
||||
|
||||
public static final TypedEvent<ContainerID> CLOSE_CONTAINER_EVENT =
|
||||
new TypedEvent<>(ContainerID.class);
|
||||
|
||||
private final Mapping containerManager;
|
||||
|
||||
@ -54,22 +53,34 @@ public void onMessage(ContainerID containerID, EventPublisher publisher) {
|
||||
|
||||
LOG.info("Close container Event triggered for container : {}",
|
||||
containerID.getId());
|
||||
ContainerStateManager stateManager = containerManager.getStateManager();
|
||||
ContainerInfo info = stateManager.getContainer(containerID);
|
||||
if (info == null) {
|
||||
LOG.info("Container with id : {} does not exist", containerID.getId());
|
||||
ContainerWithPipeline containerWithPipeline = null;
|
||||
ContainerInfo info;
|
||||
try {
|
||||
containerWithPipeline =
|
||||
containerManager.getContainerWithPipeline(containerID.getId());
|
||||
info = containerWithPipeline.getContainerInfo();
|
||||
if (info == null) {
|
||||
LOG.info("Failed to update the container state. Container with id : {} "
|
||||
+ "does not exist", containerID.getId());
|
||||
return;
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.info("Failed to update the container state. Container with id : {} "
|
||||
+ "does not exist", containerID.getId());
|
||||
return;
|
||||
}
|
||||
|
||||
if (info.getState() == HddsProtos.LifeCycleState.OPEN) {
|
||||
for (DatanodeDetails datanode : info.getPipeline().getMachines()) {
|
||||
for (DatanodeDetails datanode :
|
||||
containerWithPipeline.getPipeline().getMachines()) {
|
||||
containerManager.getNodeManager().addDatanodeCommand(datanode.getUuid(),
|
||||
new CloseContainerCommand(containerID.getId(),
|
||||
info.getPipeline().getType()));
|
||||
info.getReplicationType()));
|
||||
}
|
||||
try {
|
||||
// Finalize event will make sure the state of the container transitions
|
||||
// from OPEN to CLOSING in containerStateManager.
|
||||
stateManager
|
||||
containerManager.getStateManager()
|
||||
.updateContainerState(info, HddsProtos.LifeCycleEvent.FINALIZE);
|
||||
} catch (SCMException ex) {
|
||||
LOG.error("Failed to update the container state for container : {}"
|
||||
|
@ -21,6 +21,10 @@
|
||||
import com.google.common.primitives.Longs;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.SCMContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||
import org.apache.hadoop.hdds.scm.container.closer.ContainerCloser;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
@ -166,6 +170,44 @@ public ContainerInfo getContainer(final long containerID) throws
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the ContainerInfo from the container ID.
|
||||
*
|
||||
* @param containerID - ID of container.
|
||||
* @return - ContainerWithPipeline such as creation state and the pipeline.
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public ContainerWithPipeline getContainerWithPipeline(long containerID)
|
||||
throws IOException {
|
||||
ContainerInfo contInfo;
|
||||
lock.lock();
|
||||
try {
|
||||
byte[] containerBytes = containerStore.get(
|
||||
Longs.toByteArray(containerID));
|
||||
if (containerBytes == null) {
|
||||
throw new SCMException(
|
||||
"Specified key does not exist. key : " + containerID,
|
||||
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
|
||||
}
|
||||
HddsProtos.SCMContainerInfo temp = HddsProtos.SCMContainerInfo.PARSER
|
||||
.parseFrom(containerBytes);
|
||||
contInfo = ContainerInfo.fromProtobuf(temp);
|
||||
Pipeline pipeline = pipelineSelector
|
||||
.getPipeline(contInfo.getPipelineName(),
|
||||
contInfo.getReplicationType());
|
||||
|
||||
if(pipeline == null) {
|
||||
pipeline = pipelineSelector
|
||||
.getReplicationPipeline(contInfo.getReplicationType(),
|
||||
contInfo.getReplicationFactor());
|
||||
}
|
||||
return new ContainerWithPipeline(contInfo, pipeline);
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@ -208,13 +250,15 @@ public List<ContainerInfo> listContainer(long startContainerID,
|
||||
* @throws IOException - Exception
|
||||
*/
|
||||
@Override
|
||||
public ContainerInfo allocateContainer(
|
||||
public ContainerWithPipeline allocateContainer(
|
||||
ReplicationType type,
|
||||
ReplicationFactor replicationFactor,
|
||||
String owner)
|
||||
throws IOException {
|
||||
|
||||
ContainerInfo containerInfo;
|
||||
ContainerWithPipeline containerWithPipeline;
|
||||
|
||||
if (!nodeManager.isOutOfChillMode()) {
|
||||
throw new SCMException(
|
||||
"Unable to create container while in chill mode",
|
||||
@ -223,9 +267,9 @@ public ContainerInfo allocateContainer(
|
||||
|
||||
lock.lock();
|
||||
try {
|
||||
containerInfo =
|
||||
containerStateManager.allocateContainer(
|
||||
containerWithPipeline = containerStateManager.allocateContainer(
|
||||
pipelineSelector, type, replicationFactor, owner);
|
||||
containerInfo = containerWithPipeline.getContainerInfo();
|
||||
|
||||
byte[] containerIDBytes = Longs.toByteArray(
|
||||
containerInfo.getContainerID());
|
||||
@ -234,7 +278,7 @@ public ContainerInfo allocateContainer(
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
return containerInfo;
|
||||
return containerWithPipeline;
|
||||
}
|
||||
|
||||
/**
|
||||
@ -380,6 +424,35 @@ public ContainerStateManager getStateManager() {
|
||||
return containerStateManager;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a container matching the attributes specified.
|
||||
*
|
||||
* @param size - Space needed in the Container.
|
||||
* @param owner - Owner of the container - A specific nameservice.
|
||||
* @param type - Replication Type {StandAlone, Ratis}
|
||||
* @param factor - Replication Factor {ONE, THREE}
|
||||
* @param state - State of the Container-- {Open, Allocated etc.}
|
||||
* @return ContainerInfo, null if there is no match found.
|
||||
*/
|
||||
public ContainerWithPipeline getMatchingContainerWithPipeline(final long size,
|
||||
String owner, ReplicationType type, ReplicationFactor factor,
|
||||
LifeCycleState state) throws IOException {
|
||||
ContainerInfo containerInfo = getStateManager()
|
||||
.getMatchingContainer(size, owner, type, factor, state);
|
||||
if (containerInfo == null) {
|
||||
return null;
|
||||
}
|
||||
Pipeline pipeline = pipelineSelector
|
||||
.getPipeline(containerInfo.getPipelineName(),
|
||||
containerInfo.getReplicationType());
|
||||
if (pipeline == null) {
|
||||
pipelineSelector
|
||||
.getReplicationPipeline(containerInfo.getReplicationType(),
|
||||
containerInfo.getReplicationFactor());
|
||||
}
|
||||
return new ContainerWithPipeline(containerInfo, pipeline);
|
||||
}
|
||||
|
||||
/**
|
||||
* Process container report from Datanode.
|
||||
* <p>
|
||||
@ -415,7 +488,7 @@ public void processContainerReports(DatanodeDetails datanodeDetails,
|
||||
HddsProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes);
|
||||
|
||||
HddsProtos.SCMContainerInfo newState =
|
||||
reconcileState(datanodeState, knownState);
|
||||
reconcileState(datanodeState, knownState, datanodeDetails);
|
||||
|
||||
// FIX ME: This can be optimized, we write twice to memory, where a
|
||||
// single write would work well.
|
||||
@ -425,8 +498,14 @@ public void processContainerReports(DatanodeDetails datanodeDetails,
|
||||
containerStore.put(dbKey, newState.toByteArray());
|
||||
|
||||
// If the container is closed, then state is already written to SCM
|
||||
Pipeline pipeline = pipelineSelector.getPipeline(newState.getPipelineName(), newState.getReplicationType());
|
||||
if(pipeline == null) {
|
||||
pipeline = pipelineSelector
|
||||
.getReplicationPipeline(newState.getReplicationType(),
|
||||
newState.getReplicationFactor());
|
||||
}
|
||||
// DB.TODO: So can we can write only once to DB.
|
||||
if (closeContainerIfNeeded(newState)) {
|
||||
if (closeContainerIfNeeded(newState, pipeline)) {
|
||||
LOG.info("Closing the Container: {}", newState.getContainerID());
|
||||
}
|
||||
} else {
|
||||
@ -447,15 +526,22 @@ public void processContainerReports(DatanodeDetails datanodeDetails,
|
||||
*
|
||||
* @param datanodeState - State from the Datanode.
|
||||
* @param knownState - State inside SCM.
|
||||
* @param dnDetails
|
||||
* @return new SCM State for this container.
|
||||
*/
|
||||
private HddsProtos.SCMContainerInfo reconcileState(
|
||||
StorageContainerDatanodeProtocolProtos.ContainerInfo datanodeState,
|
||||
HddsProtos.SCMContainerInfo knownState) {
|
||||
SCMContainerInfo knownState, DatanodeDetails dnDetails) {
|
||||
HddsProtos.SCMContainerInfo.Builder builder =
|
||||
HddsProtos.SCMContainerInfo.newBuilder();
|
||||
builder.setContainerID(knownState.getContainerID());
|
||||
builder.setPipeline(knownState.getPipeline());
|
||||
builder.setContainerID(knownState.getContainerID())
|
||||
.setPipelineName(knownState.getPipelineName())
|
||||
.setReplicationType(knownState.getReplicationType())
|
||||
.setReplicationFactor(knownState.getReplicationFactor());
|
||||
|
||||
// TODO: If current state doesn't have this DN in list of DataNodes with replica
|
||||
// then add it in list of replicas.
|
||||
|
||||
// If used size is greater than allocated size, we will be updating
|
||||
// allocated size with used size. This update is done as a fallback
|
||||
// mechanism in case SCM crashes without properly updating allocated
|
||||
@ -464,13 +550,13 @@ private HddsProtos.SCMContainerInfo reconcileState(
|
||||
long usedSize = datanodeState.getUsed();
|
||||
long allocated = knownState.getAllocatedBytes() > usedSize ?
|
||||
knownState.getAllocatedBytes() : usedSize;
|
||||
builder.setAllocatedBytes(allocated);
|
||||
builder.setUsedBytes(usedSize);
|
||||
builder.setNumberOfKeys(datanodeState.getKeyCount());
|
||||
builder.setState(knownState.getState());
|
||||
builder.setStateEnterTime(knownState.getStateEnterTime());
|
||||
builder.setContainerID(knownState.getContainerID());
|
||||
builder.setDeleteTransactionId(knownState.getDeleteTransactionId());
|
||||
builder.setAllocatedBytes(allocated)
|
||||
.setUsedBytes(usedSize)
|
||||
.setNumberOfKeys(datanodeState.getKeyCount())
|
||||
.setState(knownState.getState())
|
||||
.setStateEnterTime(knownState.getStateEnterTime())
|
||||
.setContainerID(knownState.getContainerID())
|
||||
.setDeleteTransactionId(knownState.getDeleteTransactionId());
|
||||
if (knownState.getOwner() != null) {
|
||||
builder.setOwner(knownState.getOwner());
|
||||
}
|
||||
@ -485,9 +571,11 @@ private HddsProtos.SCMContainerInfo reconcileState(
|
||||
* one protobuf in one file and another definition in another file.
|
||||
*
|
||||
* @param newState - This is the state we maintain in SCM.
|
||||
* @param pipeline
|
||||
* @throws IOException
|
||||
*/
|
||||
private boolean closeContainerIfNeeded(HddsProtos.SCMContainerInfo newState)
|
||||
private boolean closeContainerIfNeeded(SCMContainerInfo newState,
|
||||
Pipeline pipeline)
|
||||
throws IOException {
|
||||
float containerUsedPercentage = 1.0f *
|
||||
newState.getUsedBytes() / this.size;
|
||||
@ -498,7 +586,7 @@ private boolean closeContainerIfNeeded(HddsProtos.SCMContainerInfo newState)
|
||||
// We will call closer till get to the closed state.
|
||||
// That is SCM will make this call repeatedly until we reach the closed
|
||||
// state.
|
||||
closer.close(newState);
|
||||
closer.close(newState, pipeline);
|
||||
|
||||
if (shouldClose(scmInfo)) {
|
||||
// This event moves the Container from Open to Closing State, this is
|
||||
@ -598,10 +686,12 @@ public void flushContainerInfo() throws IOException {
|
||||
.setAllocatedBytes(info.getAllocatedBytes())
|
||||
.setNumberOfKeys(oldInfo.getNumberOfKeys())
|
||||
.setOwner(oldInfo.getOwner())
|
||||
.setPipeline(oldInfo.getPipeline())
|
||||
.setPipelineName(oldInfo.getPipelineName())
|
||||
.setState(oldInfo.getState())
|
||||
.setUsedBytes(oldInfo.getUsedBytes())
|
||||
.setDeleteTransactionId(oldInfo.getDeleteTransactionId())
|
||||
.setReplicationFactor(oldInfo.getReplicationFactor())
|
||||
.setReplicationType(oldInfo.getReplicationType())
|
||||
.build();
|
||||
containerStore.put(dbKey, newInfo.getProtobuf().toByteArray());
|
||||
} else {
|
||||
|
@ -20,6 +20,7 @@
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||
import org.apache.hadoop.hdds.scm.container.states.ContainerState;
|
||||
@ -279,10 +280,10 @@ private void initializeStateMachine() {
|
||||
* @param selector -- Pipeline selector class.
|
||||
* @param type -- Replication type.
|
||||
* @param replicationFactor - Replication replicationFactor.
|
||||
* @return Container Info.
|
||||
* @return ContainerWithPipeline
|
||||
* @throws IOException on Failure.
|
||||
*/
|
||||
public ContainerInfo allocateContainer(PipelineSelector selector, HddsProtos
|
||||
public ContainerWithPipeline allocateContainer(PipelineSelector selector, HddsProtos
|
||||
.ReplicationType type, HddsProtos.ReplicationFactor replicationFactor,
|
||||
String owner) throws IOException {
|
||||
|
||||
@ -295,7 +296,7 @@ public ContainerInfo allocateContainer(PipelineSelector selector, HddsProtos
|
||||
|
||||
ContainerInfo containerInfo = new ContainerInfo.Builder()
|
||||
.setState(HddsProtos.LifeCycleState.ALLOCATED)
|
||||
.setPipeline(pipeline)
|
||||
.setPipelineName(pipeline.getPipelineName())
|
||||
// This is bytes allocated for blocks inside container, not the
|
||||
// container size
|
||||
.setAllocatedBytes(0)
|
||||
@ -305,11 +306,13 @@ public ContainerInfo allocateContainer(PipelineSelector selector, HddsProtos
|
||||
.setOwner(owner)
|
||||
.setContainerID(containerCount.incrementAndGet())
|
||||
.setDeleteTransactionId(0)
|
||||
.setReplicationFactor(replicationFactor)
|
||||
.setReplicationType(pipeline.getType())
|
||||
.build();
|
||||
Preconditions.checkNotNull(containerInfo);
|
||||
containers.addContainer(containerInfo);
|
||||
LOG.trace("New container allocated: {}", containerInfo);
|
||||
return containerInfo;
|
||||
return new ContainerWithPipeline(containerInfo, pipeline);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -432,8 +435,8 @@ private ContainerInfo findContainerWithSpace(long size,
|
||||
containerInfo.updateLastUsedTime();
|
||||
|
||||
ContainerState key = new ContainerState(owner,
|
||||
containerInfo.getPipeline().getType(),
|
||||
containerInfo.getPipeline().getFactor());
|
||||
containerInfo.getReplicationType(),
|
||||
containerInfo.getReplicationFactor());
|
||||
lastUsedMap.put(key, containerInfo.containerID());
|
||||
return containerInfo;
|
||||
}
|
||||
@ -457,6 +460,20 @@ public NavigableSet<ContainerID> getMatchingContainerIDs(
|
||||
factor, type);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the containerInfo with pipeline for the given container id.
|
||||
* @param selector -- Pipeline selector class.
|
||||
* @param containerID id of the container
|
||||
* @return ContainerInfo containerInfo
|
||||
* @throws IOException
|
||||
*/
|
||||
public ContainerWithPipeline getContainer(PipelineSelector selector,
|
||||
ContainerID containerID) throws IOException {
|
||||
ContainerInfo info = containers.getContainerInfo(containerID.getId());
|
||||
Pipeline pipeline = selector.getPipeline(info.getPipelineName(), info.getReplicationType());
|
||||
return new ContainerWithPipeline(info, pipeline);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the containerInfo for the given container id.
|
||||
* @param containerID id of the container
|
||||
@ -466,6 +483,7 @@ public NavigableSet<ContainerID> getMatchingContainerIDs(
|
||||
public ContainerInfo getContainer(ContainerID containerID) {
|
||||
return containers.getContainerInfo(containerID.getId());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
}
|
||||
|
@ -17,6 +17,10 @@
|
||||
package org.apache.hadoop.hdds.scm.container;
|
||||
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||
import org.apache.hadoop.hdds.protocol.proto
|
||||
@ -42,6 +46,16 @@ public interface Mapping extends Closeable {
|
||||
*/
|
||||
ContainerInfo getContainer(long containerID) throws IOException;
|
||||
|
||||
/**
|
||||
* Returns the ContainerInfo from the container ID.
|
||||
*
|
||||
* @param containerID - ID of container.
|
||||
* @return - ContainerWithPipeline such as creation state and the pipeline.
|
||||
* @throws IOException
|
||||
*/
|
||||
ContainerWithPipeline getContainerWithPipeline(long containerID)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Returns containers under certain conditions.
|
||||
* Search container IDs from start ID(exclusive),
|
||||
@ -65,10 +79,10 @@ List<ContainerInfo> listContainer(long startContainerID, int count)
|
||||
*
|
||||
* @param replicationFactor - replication factor of the container.
|
||||
* @param owner
|
||||
* @return - Container Info.
|
||||
* @return - ContainerWithPipeline.
|
||||
* @throws IOException
|
||||
*/
|
||||
ContainerInfo allocateContainer(HddsProtos.ReplicationType type,
|
||||
ContainerWithPipeline allocateContainer(HddsProtos.ReplicationType type,
|
||||
HddsProtos.ReplicationFactor replicationFactor, String owner)
|
||||
throws IOException;
|
||||
|
||||
@ -120,4 +134,12 @@ void updateDeleteTransactionId(Map<Long, Long> deleteTransactionMap)
|
||||
* @return NodeManager
|
||||
*/
|
||||
NodeManager getNodeManager();
|
||||
|
||||
/**
|
||||
* Returns the ContainerWithPipeline.
|
||||
* @return NodeManager
|
||||
*/
|
||||
public ContainerWithPipeline getMatchingContainerWithPipeline(final long size,
|
||||
String owner, ReplicationType type, ReplicationFactor factor,
|
||||
LifeCycleState state) throws IOException;
|
||||
}
|
||||
|
@ -22,6 +22,8 @@
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.SCMContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||
@ -90,8 +92,10 @@ public static int getCleanupWaterMark() {
|
||||
* lives.
|
||||
*
|
||||
* @param info - ContainerInfo.
|
||||
* @param pipeline
|
||||
*/
|
||||
public void close(HddsProtos.SCMContainerInfo info) {
|
||||
public void close(SCMContainerInfo info,
|
||||
Pipeline pipeline) {
|
||||
|
||||
if (commandIssued.containsKey(info.getContainerID())) {
|
||||
// We check if we issued a close command in last 3 * reportInterval secs.
|
||||
@ -126,13 +130,10 @@ public void close(HddsProtos.SCMContainerInfo info) {
|
||||
// this queue can be emptied by a datanode after a close report is send
|
||||
// to SCM. In that case also, data node will ignore this command.
|
||||
|
||||
HddsProtos.Pipeline pipeline = info.getPipeline();
|
||||
for (HddsProtos.DatanodeDetailsProto datanodeDetails :
|
||||
pipeline.getMembersList()) {
|
||||
nodeManager.addDatanodeCommand(
|
||||
DatanodeDetails.getFromProtoBuf(datanodeDetails).getUuid(),
|
||||
for (DatanodeDetails datanodeDetails : pipeline.getMachines()) {
|
||||
nodeManager.addDatanodeCommand(datanodeDetails.getUuid(),
|
||||
new CloseContainerCommand(info.getContainerID(),
|
||||
pipeline.getType()));
|
||||
info.getReplicationType()));
|
||||
}
|
||||
if (!commandIssued.containsKey(info.getContainerID())) {
|
||||
commandIssued.put(info.getContainerID(),
|
||||
|
@ -53,9 +53,9 @@
|
||||
* client to able to write to it.
|
||||
* <p>
|
||||
* 2. Owners - Each instance of Name service, for example, Namenode of HDFS or
|
||||
* Key Space Manager (KSM) of Ozone or CBlockServer -- is an owner. It is
|
||||
* possible to have many KSMs for a Ozone cluster and only one SCM. But SCM
|
||||
* keeps the data from each KSM in separate bucket, never mixing them. To
|
||||
* Ozone Manager (OM) of Ozone or CBlockServer -- is an owner. It is
|
||||
* possible to have many OMs for a Ozone cluster and only one SCM. But SCM
|
||||
* keeps the data from each OM in separate bucket, never mixing them. To
|
||||
* write data, often we have to find all open containers for a specific owner.
|
||||
* <p>
|
||||
* 3. ReplicationType - The clients are allowed to specify what kind of
|
||||
@ -116,7 +116,8 @@ public ContainerStateMap() {
|
||||
public void addContainer(ContainerInfo info)
|
||||
throws SCMException {
|
||||
Preconditions.checkNotNull(info, "Container Info cannot be null");
|
||||
Preconditions.checkNotNull(info.getPipeline(), "Pipeline cannot be null");
|
||||
Preconditions.checkArgument(info.getReplicationFactor().getNumber() > 0,
|
||||
"ExpectedReplicaCount should be greater than 0");
|
||||
|
||||
try (AutoCloseableLock lock = autoLock.acquire()) {
|
||||
ContainerID id = ContainerID.valueof(info.getContainerID());
|
||||
@ -129,8 +130,8 @@ public void addContainer(ContainerInfo info)
|
||||
|
||||
lifeCycleStateMap.insert(info.getState(), id);
|
||||
ownerMap.insert(info.getOwner(), id);
|
||||
factorMap.insert(info.getPipeline().getFactor(), id);
|
||||
typeMap.insert(info.getPipeline().getType(), id);
|
||||
factorMap.insert(info.getReplicationFactor(), id);
|
||||
typeMap.insert(info.getReplicationType(), id);
|
||||
LOG.trace("Created container with {} successfully.", id);
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,80 @@
|
||||
/*
|
||||
* 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.events;
|
||||
|
||||
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.ContainerReportFromDatanode;
|
||||
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.NodeReportFromDatanode;
|
||||
|
||||
import org.apache.hadoop.hdds.server.events.Event;
|
||||
import org.apache.hadoop.hdds.server.events.TypedEvent;
|
||||
import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode;
|
||||
|
||||
/**
|
||||
* Class that acts as the namespace for all SCM Events.
|
||||
*/
|
||||
public final class SCMEvents {
|
||||
|
||||
/**
|
||||
* NodeReports are sent out by Datanodes. This report is
|
||||
* received by SCMDatanodeHeartbeatDispatcher and NodeReport Event is
|
||||
* generated.
|
||||
*/
|
||||
public static final TypedEvent<NodeReportFromDatanode> NODE_REPORT =
|
||||
new TypedEvent<>(NodeReportFromDatanode.class, "Node_Report");
|
||||
/**
|
||||
* ContainerReports are send out by Datanodes. This report
|
||||
* is received by SCMDatanodeHeartbeatDispatcher and Container_Report Event
|
||||
* i generated.
|
||||
*/
|
||||
public static final TypedEvent<ContainerReportFromDatanode> CONTAINER_REPORT =
|
||||
new TypedEvent<>(ContainerReportFromDatanode.class, "Container_Report");
|
||||
|
||||
/**
|
||||
* When ever a command for the Datanode needs to be issued by any component
|
||||
* inside SCM, a Datanode_Command event is generated. NodeManager listens
|
||||
* to these events and dispatches them to Datanode for further processing.
|
||||
*/
|
||||
public static final Event<CommandForDatanode> DATANODE_COMMAND =
|
||||
new TypedEvent<>(CommandForDatanode.class, "Datanode_Command");
|
||||
|
||||
/**
|
||||
* A Close Container Event can be triggered under many condition.
|
||||
* Some of them are:
|
||||
* 1. A Container is full, then we stop writing further information to
|
||||
* that container. DN's let SCM know that current state and sends a
|
||||
* informational message that allows SCM to close the container.
|
||||
*
|
||||
* 2. If a pipeline is open; for example Ratis; if a single node fails,
|
||||
* we will proactively close these containers.
|
||||
*
|
||||
* Once a command is dispatched to DN, we will also listen to updates from
|
||||
* the datanode which lets us know that this command completed or timed out.
|
||||
*/
|
||||
public static final TypedEvent<ContainerID> CLOSE_CONTAINER =
|
||||
new TypedEvent<>(ContainerID.class, "Close_Container");
|
||||
|
||||
/**
|
||||
* Private Ctor. Never Constructed.
|
||||
*/
|
||||
private SCMEvents() {
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,23 @@
|
||||
/*
|
||||
* 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.
|
||||
*
|
||||
*/
|
||||
/**
|
||||
* Events Package contains all the Events used by SCM internally to
|
||||
* communicate between different sub-systems that make up SCM.
|
||||
*/
|
||||
package org.apache.hadoop.hdds.scm.events;
|
@ -33,7 +33,7 @@
|
||||
/**
|
||||
* Command Queue is queue of commands for the datanode.
|
||||
* <p>
|
||||
* Node manager, container Manager and key space managers can queue commands for
|
||||
* Node manager, container Manager and Ozone managers can queue commands for
|
||||
* datanodes into this queue. These commands will be send in the order in which
|
||||
* there where queued.
|
||||
*/
|
||||
|
@ -0,0 +1,109 @@
|
||||
/**
|
||||
* 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.node;
|
||||
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.proto
|
||||
.StorageContainerDatanodeProtocolProtos.StorageReportProto;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.concurrent.locks.ReadWriteLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
/**
|
||||
* This class extends the primary identifier of a Datanode with ephemeral
|
||||
* state, eg last reported time, usage information etc.
|
||||
*/
|
||||
public class DatanodeInfo extends DatanodeDetails {
|
||||
|
||||
private final ReadWriteLock lock;
|
||||
|
||||
private volatile long lastHeartbeatTime;
|
||||
private long lastStatsUpdatedTime;
|
||||
|
||||
// If required we can dissect StorageReportProto and store the raw data
|
||||
private List<StorageReportProto> storageReports;
|
||||
|
||||
/**
|
||||
* Constructs DatanodeInfo from DatanodeDetails.
|
||||
*
|
||||
* @param datanodeDetails Details about the datanode
|
||||
*/
|
||||
public DatanodeInfo(DatanodeDetails datanodeDetails) {
|
||||
super(datanodeDetails);
|
||||
lock = new ReentrantReadWriteLock();
|
||||
lastHeartbeatTime = Time.monotonicNow();
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the last heartbeat time with current time.
|
||||
*/
|
||||
public void updateLastHeartbeatTime() {
|
||||
try {
|
||||
lock.writeLock().lock();
|
||||
lastHeartbeatTime = Time.monotonicNow();
|
||||
} finally {
|
||||
lock.writeLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the last heartbeat time.
|
||||
*
|
||||
* @return last heartbeat time.
|
||||
*/
|
||||
public long getLastHeartbeatTime() {
|
||||
try {
|
||||
lock.readLock().lock();
|
||||
return lastHeartbeatTime;
|
||||
} finally {
|
||||
lock.readLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the datanode storage reports.
|
||||
*
|
||||
* @param reports list of storage report
|
||||
*/
|
||||
public void updateStorageReports(List<StorageReportProto> reports) {
|
||||
try {
|
||||
lock.writeLock().lock();
|
||||
lastStatsUpdatedTime = Time.monotonicNow();
|
||||
storageReports = reports;
|
||||
} finally {
|
||||
lock.writeLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the storage reports associated with this datanode.
|
||||
*
|
||||
* @return list of storage report
|
||||
*/
|
||||
public List<StorageReportProto> getStorageReports() {
|
||||
try {
|
||||
lock.readLock().lock();
|
||||
return storageReports;
|
||||
} finally {
|
||||
lock.readLock().unlock();
|
||||
}
|
||||
}
|
||||
}
|
@ -1,98 +0,0 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hdds.scm.node;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.proto
|
||||
.StorageContainerDatanodeProtocolProtos.NodeReportProto;
|
||||
|
||||
import static org.apache.hadoop.util.Time.monotonicNow;
|
||||
|
||||
/**
|
||||
* This class represents the item in SCM heartbeat queue.
|
||||
*/
|
||||
public class HeartbeatQueueItem {
|
||||
private DatanodeDetails datanodeDetails;
|
||||
private long recvTimestamp;
|
||||
private NodeReportProto nodeReport;
|
||||
|
||||
/**
|
||||
*
|
||||
* @param datanodeDetails - datanode ID of the heartbeat.
|
||||
* @param recvTimestamp - heartbeat receive timestamp.
|
||||
* @param nodeReport - node report associated with the heartbeat if any.
|
||||
*/
|
||||
HeartbeatQueueItem(DatanodeDetails datanodeDetails, long recvTimestamp,
|
||||
NodeReportProto nodeReport) {
|
||||
this.datanodeDetails = datanodeDetails;
|
||||
this.recvTimestamp = recvTimestamp;
|
||||
this.nodeReport = nodeReport;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return datanode ID.
|
||||
*/
|
||||
public DatanodeDetails getDatanodeDetails() {
|
||||
return datanodeDetails;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return node report.
|
||||
*/
|
||||
public NodeReportProto getNodeReport() {
|
||||
return nodeReport;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return heartbeat receive timestamp.
|
||||
*/
|
||||
public long getRecvTimestamp() {
|
||||
return recvTimestamp;
|
||||
}
|
||||
|
||||
/**
|
||||
* Builder for HeartbeatQueueItem.
|
||||
*/
|
||||
public static class Builder {
|
||||
private DatanodeDetails datanodeDetails;
|
||||
private NodeReportProto nodeReport;
|
||||
private long recvTimestamp = monotonicNow();
|
||||
|
||||
public Builder setDatanodeDetails(DatanodeDetails dnDetails) {
|
||||
this.datanodeDetails = dnDetails;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setNodeReport(NodeReportProto report) {
|
||||
this.nodeReport = report;
|
||||
return this;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public Builder setRecvTimestamp(long recvTime) {
|
||||
this.recvTimestamp = recvTime;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HeartbeatQueueItem build() {
|
||||
return new HeartbeatQueueItem(datanodeDetails, recvTimestamp, nodeReport);
|
||||
}
|
||||
}
|
||||
}
|
@ -17,10 +17,9 @@
|
||||
*/
|
||||
package org.apache.hadoop.hdds.scm.node;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
|
||||
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
|
||||
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
|
||||
import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
|
||||
import org.apache.hadoop.ozone.protocol.StorageContainerNodeProtocol;
|
||||
@ -54,14 +53,14 @@
|
||||
* list, by calling removeNode. We will throw away this nodes info soon.
|
||||
*/
|
||||
public interface NodeManager extends StorageContainerNodeProtocol,
|
||||
NodeManagerMXBean, Closeable, Runnable {
|
||||
NodeManagerMXBean, Closeable {
|
||||
/**
|
||||
* Removes a data node from the management of this Node Manager.
|
||||
*
|
||||
* @param node - DataNode.
|
||||
* @throws UnregisteredNodeException
|
||||
* @throws NodeNotFoundException
|
||||
*/
|
||||
void removeNode(DatanodeDetails node) throws UnregisteredNodeException;
|
||||
void removeNode(DatanodeDetails node) throws NodeNotFoundException;
|
||||
|
||||
/**
|
||||
* Gets all Live Datanodes that is currently communicating with SCM.
|
||||
@ -123,13 +122,6 @@ public interface NodeManager extends StorageContainerNodeProtocol,
|
||||
*/
|
||||
SCMNodeMetric getNodeStat(DatanodeDetails datanodeDetails);
|
||||
|
||||
/**
|
||||
* Wait for the heartbeat is processed by NodeManager.
|
||||
* @return true if heartbeat has been processed.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
boolean waitForHeartbeatProcessed();
|
||||
|
||||
/**
|
||||
* Returns the node state of a specific node.
|
||||
* @param datanodeDetails DatanodeDetails
|
||||
|
@ -0,0 +1,575 @@
|
||||
/**
|
||||
* 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.node;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
|
||||
import org.apache.hadoop.hdds.scm.HddsServerUtil;
|
||||
import org.apache.hadoop.hdds.scm.node.states.NodeAlreadyExistsException;
|
||||
import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
|
||||
import org.apache.hadoop.hdds.scm.node.states.NodeStateMap;
|
||||
import org.apache.hadoop.ozone.common.statemachine
|
||||
.InvalidStateTransitionException;
|
||||
import org.apache.hadoop.ozone.common.statemachine.StateMachine;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
||||
.OZONE_SCM_DEADNODE_INTERVAL;
|
||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
||||
.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL;
|
||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
||||
.OZONE_SCM_STALENODE_INTERVAL;
|
||||
|
||||
/**
|
||||
* NodeStateManager maintains the state of all the datanodes in the cluster. All
|
||||
* the node state change should happen only via NodeStateManager. It also
|
||||
* runs a heartbeat thread which periodically updates the node state.
|
||||
* <p>
|
||||
* The getNode(byState) functions make copy of node maps and then creates a list
|
||||
* based on that. It should be assumed that these get functions always report
|
||||
* *stale* information. For example, getting the deadNodeCount followed by
|
||||
* getNodes(DEAD) could very well produce totally different count. Also
|
||||
* getNodeCount(HEALTHY) + getNodeCount(DEAD) + getNodeCode(STALE), is not
|
||||
* guaranteed to add up to the total nodes that we know off. Please treat all
|
||||
* get functions in this file as a snap-shot of information that is inconsistent
|
||||
* as soon as you read it.
|
||||
*/
|
||||
public class NodeStateManager implements Runnable, Closeable {
|
||||
|
||||
/**
|
||||
* Node's life cycle events.
|
||||
*/
|
||||
private enum NodeLifeCycleEvent {
|
||||
TIMEOUT, RESTORE, RESURRECT, DECOMMISSION, DECOMMISSIONED
|
||||
}
|
||||
|
||||
private static final Logger LOG = LoggerFactory
|
||||
.getLogger(NodeStateManager.class);
|
||||
|
||||
/**
|
||||
* StateMachine for node lifecycle.
|
||||
*/
|
||||
private final StateMachine<NodeState, NodeLifeCycleEvent> stateMachine;
|
||||
/**
|
||||
* This is the map which maintains the current state of all datanodes.
|
||||
*/
|
||||
private final NodeStateMap nodeStateMap;
|
||||
/**
|
||||
* ExecutorService used for scheduling heartbeat processing thread.
|
||||
*/
|
||||
private final ScheduledExecutorService executorService;
|
||||
/**
|
||||
* The frequency in which we have run the heartbeat processing thread.
|
||||
*/
|
||||
private final long heartbeatCheckerIntervalMs;
|
||||
/**
|
||||
* The timeout value which will be used for marking a datanode as stale.
|
||||
*/
|
||||
private final long staleNodeIntervalMs;
|
||||
/**
|
||||
* The timeout value which will be used for marking a datanode as dead.
|
||||
*/
|
||||
private final long deadNodeIntervalMs;
|
||||
|
||||
/**
|
||||
* Constructs a NodeStateManager instance with the given configuration.
|
||||
*
|
||||
* @param conf Configuration
|
||||
*/
|
||||
public NodeStateManager(Configuration conf) {
|
||||
nodeStateMap = new NodeStateMap();
|
||||
Set<NodeState> finalStates = new HashSet<>();
|
||||
finalStates.add(NodeState.DECOMMISSIONED);
|
||||
this.stateMachine = new StateMachine<>(NodeState.HEALTHY, finalStates);
|
||||
initializeStateMachine();
|
||||
heartbeatCheckerIntervalMs = HddsServerUtil
|
||||
.getScmheartbeatCheckerInterval(conf);
|
||||
staleNodeIntervalMs = HddsServerUtil.getStaleNodeInterval(conf);
|
||||
deadNodeIntervalMs = HddsServerUtil.getDeadNodeInterval(conf);
|
||||
Preconditions.checkState(heartbeatCheckerIntervalMs > 0,
|
||||
OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL + " should be greater than 0.");
|
||||
Preconditions.checkState(staleNodeIntervalMs < deadNodeIntervalMs,
|
||||
OZONE_SCM_STALENODE_INTERVAL + " should be less than" +
|
||||
OZONE_SCM_DEADNODE_INTERVAL);
|
||||
executorService = HadoopExecutors.newScheduledThreadPool(1,
|
||||
new ThreadFactoryBuilder().setDaemon(true)
|
||||
.setNameFormat("SCM Heartbeat Processing Thread - %d").build());
|
||||
executorService.schedule(this, heartbeatCheckerIntervalMs,
|
||||
TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
/*
|
||||
*
|
||||
* Node and State Transition Mapping:
|
||||
*
|
||||
* State: HEALTHY -------------------> STALE
|
||||
* Event: TIMEOUT
|
||||
*
|
||||
* State: STALE -------------------> DEAD
|
||||
* Event: TIMEOUT
|
||||
*
|
||||
* State: STALE -------------------> HEALTHY
|
||||
* Event: RESTORE
|
||||
*
|
||||
* State: DEAD -------------------> HEALTHY
|
||||
* Event: RESURRECT
|
||||
*
|
||||
* State: HEALTHY -------------------> DECOMMISSIONING
|
||||
* Event: DECOMMISSION
|
||||
*
|
||||
* State: STALE -------------------> DECOMMISSIONING
|
||||
* Event: DECOMMISSION
|
||||
*
|
||||
* State: DEAD -------------------> DECOMMISSIONING
|
||||
* Event: DECOMMISSION
|
||||
*
|
||||
* State: DECOMMISSIONING -------------------> DECOMMISSIONED
|
||||
* Event: DECOMMISSIONED
|
||||
*
|
||||
* Node State Flow
|
||||
*
|
||||
* +--------------------------------------------------------+
|
||||
* | (RESURRECT) |
|
||||
* | +--------------------------+ |
|
||||
* | | (RESTORE) | |
|
||||
* | | | |
|
||||
* V V | |
|
||||
* [HEALTHY]------------------->[STALE]------------------->[DEAD]
|
||||
* | (TIMEOUT) | (TIMEOUT) |
|
||||
* | | |
|
||||
* | | |
|
||||
* | | |
|
||||
* | | |
|
||||
* | (DECOMMISSION) | (DECOMMISSION) | (DECOMMISSION)
|
||||
* | V |
|
||||
* +------------------->[DECOMMISSIONING]<----------------+
|
||||
* |
|
||||
* | (DECOMMISSIONED)
|
||||
* |
|
||||
* V
|
||||
* [DECOMMISSIONED]
|
||||
*
|
||||
*/
|
||||
|
||||
/**
|
||||
* Initializes the lifecycle of node state machine.
|
||||
*/
|
||||
private void initializeStateMachine() {
|
||||
stateMachine.addTransition(
|
||||
NodeState.HEALTHY, NodeState.STALE, NodeLifeCycleEvent.TIMEOUT);
|
||||
stateMachine.addTransition(
|
||||
NodeState.STALE, NodeState.DEAD, NodeLifeCycleEvent.TIMEOUT);
|
||||
stateMachine.addTransition(
|
||||
NodeState.STALE, NodeState.HEALTHY, NodeLifeCycleEvent.RESTORE);
|
||||
stateMachine.addTransition(
|
||||
NodeState.DEAD, NodeState.HEALTHY, NodeLifeCycleEvent.RESURRECT);
|
||||
stateMachine.addTransition(
|
||||
NodeState.HEALTHY, NodeState.DECOMMISSIONING,
|
||||
NodeLifeCycleEvent.DECOMMISSION);
|
||||
stateMachine.addTransition(
|
||||
NodeState.STALE, NodeState.DECOMMISSIONING,
|
||||
NodeLifeCycleEvent.DECOMMISSION);
|
||||
stateMachine.addTransition(
|
||||
NodeState.DEAD, NodeState.DECOMMISSIONING,
|
||||
NodeLifeCycleEvent.DECOMMISSION);
|
||||
stateMachine.addTransition(
|
||||
NodeState.DECOMMISSIONING, NodeState.DECOMMISSIONED,
|
||||
NodeLifeCycleEvent.DECOMMISSIONED);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a new node to the state manager.
|
||||
*
|
||||
* @param datanodeDetails DatanodeDetails
|
||||
*
|
||||
* @throws NodeAlreadyExistsException if the node is already present
|
||||
*/
|
||||
public void addNode(DatanodeDetails datanodeDetails)
|
||||
throws NodeAlreadyExistsException {
|
||||
nodeStateMap.addNode(datanodeDetails, stateMachine.getInitialState());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get information about the node.
|
||||
*
|
||||
* @param datanodeDetails DatanodeDetails
|
||||
*
|
||||
* @return DatanodeInfo
|
||||
*
|
||||
* @throws NodeNotFoundException if the node is not present
|
||||
*/
|
||||
public DatanodeInfo getNode(DatanodeDetails datanodeDetails)
|
||||
throws NodeNotFoundException {
|
||||
return nodeStateMap.getNodeInfo(datanodeDetails.getUuid());
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the last heartbeat time of the node.
|
||||
*
|
||||
* @throws NodeNotFoundException if the node is not present
|
||||
*/
|
||||
public void updateLastHeartbeatTime(DatanodeDetails datanodeDetails)
|
||||
throws NodeNotFoundException {
|
||||
nodeStateMap.getNodeInfo(datanodeDetails.getUuid())
|
||||
.updateLastHeartbeatTime();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the current state of the node.
|
||||
*
|
||||
* @param datanodeDetails DatanodeDetails
|
||||
*
|
||||
* @return NodeState
|
||||
*
|
||||
* @throws NodeNotFoundException if the node is not present
|
||||
*/
|
||||
public NodeState getNodeState(DatanodeDetails datanodeDetails)
|
||||
throws NodeNotFoundException {
|
||||
return nodeStateMap.getNodeState(datanodeDetails.getUuid());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all the node which are in healthy state.
|
||||
*
|
||||
* @return list of healthy nodes
|
||||
*/
|
||||
public List<DatanodeDetails> getHealthyNodes() {
|
||||
return getNodes(NodeState.HEALTHY);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all the node which are in stale state.
|
||||
*
|
||||
* @return list of stale nodes
|
||||
*/
|
||||
public List<DatanodeDetails> getStaleNodes() {
|
||||
return getNodes(NodeState.STALE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all the node which are in dead state.
|
||||
*
|
||||
* @return list of dead nodes
|
||||
*/
|
||||
public List<DatanodeDetails> getDeadNodes() {
|
||||
return getNodes(NodeState.DEAD);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all the node which are in the specified state.
|
||||
*
|
||||
* @param state NodeState
|
||||
*
|
||||
* @return list of nodes
|
||||
*/
|
||||
public List<DatanodeDetails> getNodes(NodeState state) {
|
||||
List<DatanodeDetails> nodes = new LinkedList<>();
|
||||
nodeStateMap.getNodes(state).forEach(
|
||||
uuid -> {
|
||||
try {
|
||||
nodes.add(nodeStateMap.getNodeDetails(uuid));
|
||||
} catch (NodeNotFoundException e) {
|
||||
// This should not happen unless someone else other than
|
||||
// NodeStateManager is directly modifying NodeStateMap and removed
|
||||
// the node entry after we got the list of UUIDs.
|
||||
LOG.error("Inconsistent NodeStateMap! " + nodeStateMap);
|
||||
}
|
||||
});
|
||||
return nodes;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all the nodes which have registered to NodeStateManager.
|
||||
*
|
||||
* @return all the managed nodes
|
||||
*/
|
||||
public List<DatanodeDetails> getAllNodes() {
|
||||
List<DatanodeDetails> nodes = new LinkedList<>();
|
||||
nodeStateMap.getAllNodes().forEach(
|
||||
uuid -> {
|
||||
try {
|
||||
nodes.add(nodeStateMap.getNodeDetails(uuid));
|
||||
} catch (NodeNotFoundException e) {
|
||||
// This should not happen unless someone else other than
|
||||
// NodeStateManager is directly modifying NodeStateMap and removed
|
||||
// the node entry after we got the list of UUIDs.
|
||||
LOG.error("Inconsistent NodeStateMap! " + nodeStateMap);
|
||||
}
|
||||
});
|
||||
return nodes;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the count of healthy nodes.
|
||||
*
|
||||
* @return healthy node count
|
||||
*/
|
||||
public int getHealthyNodeCount() {
|
||||
return getNodeCount(NodeState.HEALTHY);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the count of stale nodes.
|
||||
*
|
||||
* @return stale node count
|
||||
*/
|
||||
public int getStaleNodeCount() {
|
||||
return getNodeCount(NodeState.STALE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the count of dead nodes.
|
||||
*
|
||||
* @return dead node count
|
||||
*/
|
||||
public int getDeadNodeCount() {
|
||||
return getNodeCount(NodeState.DEAD);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the count of nodes in specified state.
|
||||
*
|
||||
* @param state NodeState
|
||||
*
|
||||
* @return node count
|
||||
*/
|
||||
public int getNodeCount(NodeState state) {
|
||||
return nodeStateMap.getNodeCount(state);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the count of all nodes managed by NodeStateManager.
|
||||
*
|
||||
* @return node count
|
||||
*/
|
||||
public int getTotalNodeCount() {
|
||||
return nodeStateMap.getTotalNodeCount();
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes a node from NodeStateManager.
|
||||
*
|
||||
* @param datanodeDetails DatanodeDetails
|
||||
*
|
||||
* @throws NodeNotFoundException if the node is not present
|
||||
*/
|
||||
public void removeNode(DatanodeDetails datanodeDetails)
|
||||
throws NodeNotFoundException {
|
||||
nodeStateMap.removeNode(datanodeDetails.getUuid());
|
||||
}
|
||||
|
||||
/**
|
||||
* Move Stale or Dead node to healthy if we got a heartbeat from them.
|
||||
* Move healthy nodes to stale nodes if it is needed.
|
||||
* Move Stales node to dead if needed.
|
||||
*
|
||||
* @see Thread#run()
|
||||
*/
|
||||
@Override
|
||||
public void run() {
|
||||
|
||||
/*
|
||||
*
|
||||
* staleNodeDeadline healthyNodeDeadline
|
||||
* | |
|
||||
* Dead | Stale | Healthy
|
||||
* Node | Node | Node
|
||||
* Window | Window | Window
|
||||
* ----------------+----------------------------------+------------------->
|
||||
* >>-->> time-line >>-->>
|
||||
*
|
||||
* Here is the logic of computing the health of a node.
|
||||
*
|
||||
* 1. We get the current time and look back that the time
|
||||
* when we got a heartbeat from a node.
|
||||
*
|
||||
* 2. If the last heartbeat was within the window of healthy node we mark
|
||||
* it as healthy.
|
||||
*
|
||||
* 3. If the last HB Time stamp is longer and falls within the window of
|
||||
* Stale Node time, we will mark it as Stale.
|
||||
*
|
||||
* 4. If the last HB time is older than the Stale Window, then the node is
|
||||
* marked as dead.
|
||||
*
|
||||
* The Processing starts from current time and looks backwards in time.
|
||||
*/
|
||||
long processingStartTime = Time.monotonicNow();
|
||||
// After this time node is considered to be stale.
|
||||
long healthyNodeDeadline = processingStartTime - staleNodeIntervalMs;
|
||||
// After this time node is considered to be dead.
|
||||
long staleNodeDeadline = processingStartTime - deadNodeIntervalMs;
|
||||
|
||||
Predicate<Long> healthyNodeCondition =
|
||||
(lastHbTime) -> lastHbTime >= healthyNodeDeadline;
|
||||
// staleNodeCondition is superset of stale and dead node
|
||||
Predicate<Long> staleNodeCondition =
|
||||
(lastHbTime) -> lastHbTime < healthyNodeDeadline;
|
||||
Predicate<Long> deadNodeCondition =
|
||||
(lastHbTime) -> lastHbTime < staleNodeDeadline;
|
||||
try {
|
||||
for (NodeState state : NodeState.values()) {
|
||||
List<UUID> nodes = nodeStateMap.getNodes(state);
|
||||
for (UUID id : nodes) {
|
||||
DatanodeInfo node = nodeStateMap.getNodeInfo(id);
|
||||
switch (state) {
|
||||
case HEALTHY:
|
||||
// Move the node to STALE if the last heartbeat time is less than
|
||||
// configured stale-node interval.
|
||||
updateNodeState(node, staleNodeCondition, state,
|
||||
NodeLifeCycleEvent.TIMEOUT);
|
||||
break;
|
||||
case STALE:
|
||||
// Move the node to DEAD if the last heartbeat time is less than
|
||||
// configured dead-node interval.
|
||||
updateNodeState(node, deadNodeCondition, state,
|
||||
NodeLifeCycleEvent.TIMEOUT);
|
||||
// Restore the node if we have received heartbeat before configured
|
||||
// stale-node interval.
|
||||
updateNodeState(node, healthyNodeCondition, state,
|
||||
NodeLifeCycleEvent.RESTORE);
|
||||
break;
|
||||
case DEAD:
|
||||
// Resurrect the node if we have received heartbeat before
|
||||
// configured stale-node interval.
|
||||
updateNodeState(node, healthyNodeCondition, state,
|
||||
NodeLifeCycleEvent.RESURRECT);
|
||||
break;
|
||||
// We don't do anything for DECOMMISSIONING and DECOMMISSIONED in
|
||||
// heartbeat processing.
|
||||
case DECOMMISSIONING:
|
||||
case DECOMMISSIONED:
|
||||
default:
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (NodeNotFoundException e) {
|
||||
// This should not happen unless someone else other than
|
||||
// NodeStateManager is directly modifying NodeStateMap and removed
|
||||
// the node entry after we got the list of UUIDs.
|
||||
LOG.error("Inconsistent NodeStateMap! " + nodeStateMap);
|
||||
}
|
||||
long processingEndTime = Time.monotonicNow();
|
||||
//If we have taken too much time for HB processing, log that information.
|
||||
if ((processingEndTime - processingStartTime) >
|
||||
heartbeatCheckerIntervalMs) {
|
||||
LOG.error("Total time spend processing datanode HB's is greater than " +
|
||||
"configured values for datanode heartbeats. Please adjust the" +
|
||||
" heartbeat configs. Time Spend on HB processing: {} seconds " +
|
||||
"Datanode heartbeat Interval: {} seconds.",
|
||||
TimeUnit.MILLISECONDS
|
||||
.toSeconds(processingEndTime - processingStartTime),
|
||||
heartbeatCheckerIntervalMs);
|
||||
}
|
||||
|
||||
// we purposefully make this non-deterministic. Instead of using a
|
||||
// scheduleAtFixedFrequency we will just go to sleep
|
||||
// and wake up at the next rendezvous point, which is currentTime +
|
||||
// heartbeatCheckerIntervalMs. This leads to the issue that we are now
|
||||
// heart beating not at a fixed cadence, but clock tick + time taken to
|
||||
// work.
|
||||
//
|
||||
// This time taken to work can skew the heartbeat processor thread.
|
||||
// The reason why we don't care is because of the following reasons.
|
||||
//
|
||||
// 1. checkerInterval is general many magnitudes faster than datanode HB
|
||||
// frequency.
|
||||
//
|
||||
// 2. if we have too much nodes, the SCM would be doing only HB
|
||||
// processing, this could lead to SCM's CPU starvation. With this
|
||||
// approach we always guarantee that HB thread sleeps for a little while.
|
||||
//
|
||||
// 3. It is possible that we will never finish processing the HB's in the
|
||||
// thread. But that means we have a mis-configured system. We will warn
|
||||
// the users by logging that information.
|
||||
//
|
||||
// 4. And the most important reason, heartbeats are not blocked even if
|
||||
// this thread does not run, they will go into the processing queue.
|
||||
|
||||
if (!Thread.currentThread().isInterrupted() &&
|
||||
!executorService.isShutdown()) {
|
||||
executorService.schedule(this, heartbeatCheckerIntervalMs,
|
||||
TimeUnit.MILLISECONDS);
|
||||
} else {
|
||||
LOG.info("Current Thread is interrupted, shutting down HB processing " +
|
||||
"thread for Node Manager.");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the node state if the condition satisfies.
|
||||
*
|
||||
* @param node DatanodeInfo
|
||||
* @param condition condition to check
|
||||
* @param state current state of node
|
||||
* @param lifeCycleEvent NodeLifeCycleEvent to be applied if condition
|
||||
* matches
|
||||
*
|
||||
* @throws NodeNotFoundException if the node is not present
|
||||
*/
|
||||
private void updateNodeState(DatanodeInfo node, Predicate<Long> condition,
|
||||
NodeState state, NodeLifeCycleEvent lifeCycleEvent)
|
||||
throws NodeNotFoundException {
|
||||
try {
|
||||
if (condition.test(node.getLastHeartbeatTime())) {
|
||||
NodeState newState = stateMachine.getNextState(state, lifeCycleEvent);
|
||||
nodeStateMap.updateNodeState(node.getUuid(), state, newState);
|
||||
}
|
||||
} catch (InvalidStateTransitionException e) {
|
||||
LOG.warn("Invalid state transition of node {}." +
|
||||
" Current state: {}, life cycle event: {}",
|
||||
node, state, lifeCycleEvent);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
executorService.shutdown();
|
||||
try {
|
||||
if (!executorService.awaitTermination(5, TimeUnit.SECONDS)) {
|
||||
executorService.shutdownNow();
|
||||
}
|
||||
|
||||
if (!executorService.awaitTermination(5, TimeUnit.SECONDS)) {
|
||||
LOG.error("Unable to shutdown NodeStateManager properly.");
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
executorService.shutdownNow();
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
}
|
@ -19,17 +19,14 @@
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import org.apache.hadoop.hdds.scm.HddsServerUtil;
|
||||
import org.apache.hadoop.hdds.scm.node.states.NodeAlreadyExistsException;
|
||||
import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
|
||||
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
||||
import org.apache.hadoop.hdds.scm.VersionInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
|
||||
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
|
||||
import org.apache.hadoop.hdds.server.events.Event;
|
||||
import org.apache.hadoop.hdds.server.events.EventHandler;
|
||||
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
||||
import org.apache.hadoop.hdds.server.events.TypedEvent;
|
||||
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
|
||||
@ -51,8 +48,6 @@
|
||||
import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
|
||||
import org.apache.hadoop.ozone.protocol.commands.ReregisterCommand;
|
||||
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
@ -64,39 +59,15 @@
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Queue;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DEAD;
|
||||
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState
|
||||
.HEALTHY;
|
||||
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState
|
||||
.INVALID;
|
||||
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.STALE;
|
||||
import static org.apache.hadoop.util.Time.monotonicNow;
|
||||
|
||||
/**
|
||||
* Maintains information about the Datanodes on SCM side.
|
||||
* <p>
|
||||
* Heartbeats under SCM is very simple compared to HDFS heartbeatManager.
|
||||
* <p>
|
||||
* Here we maintain 3 maps, and we propagate a node from healthyNodesMap to
|
||||
* staleNodesMap to deadNodesMap. This moving of a node from one map to another
|
||||
* is controlled by 4 configuration variables. These variables define how many
|
||||
* heartbeats must go missing for the node to move from one map to another.
|
||||
* <p>
|
||||
* Each heartbeat that SCMNodeManager receives is put into heartbeatQueue. The
|
||||
* worker thread wakes up and grabs that heartbeat from the queue. The worker
|
||||
* thread will lookup the healthynodes map and set the timestamp if the entry
|
||||
* is there. if not it will look up stale and deadnodes map.
|
||||
* <p>
|
||||
* The getNode(byState) functions make copy of node maps and then creates a list
|
||||
* based on that. It should be assumed that these get functions always report
|
||||
* *stale* information. For example, getting the deadNodeCount followed by
|
||||
@ -114,33 +85,18 @@ public class SCMNodeManager
|
||||
static final Logger LOG =
|
||||
LoggerFactory.getLogger(SCMNodeManager.class);
|
||||
|
||||
/**
|
||||
* Key = NodeID, value = timestamp.
|
||||
*/
|
||||
private final ConcurrentHashMap<UUID, Long> healthyNodes;
|
||||
private final ConcurrentHashMap<UUID, Long> staleNodes;
|
||||
private final ConcurrentHashMap<UUID, Long> deadNodes;
|
||||
private final Queue<HeartbeatQueueItem> heartbeatQueue;
|
||||
private final ConcurrentHashMap<UUID, DatanodeDetails> nodes;
|
||||
|
||||
private final NodeStateManager nodeStateManager;
|
||||
// Individual live node stats
|
||||
// TODO: NodeStat should be moved to NodeStatemanager (NodeStateMap)
|
||||
private final ConcurrentHashMap<UUID, SCMNodeStat> nodeStats;
|
||||
// Should we maintain aggregated stats? If this is not frequently used, we
|
||||
// can always calculate it from nodeStats whenever required.
|
||||
// Aggregated node stats
|
||||
private SCMNodeStat scmStat;
|
||||
// TODO: expose nodeStats and scmStat as metrics
|
||||
private final AtomicInteger healthyNodeCount;
|
||||
private final AtomicInteger staleNodeCount;
|
||||
private final AtomicInteger deadNodeCount;
|
||||
private final AtomicInteger totalNodes;
|
||||
private long staleNodeIntervalMs;
|
||||
private final long deadNodeIntervalMs;
|
||||
private final long heartbeatCheckerIntervalMs;
|
||||
private final long datanodeHBIntervalSeconds;
|
||||
private final ScheduledExecutorService executorService;
|
||||
private long lastHBcheckStart;
|
||||
private long lastHBcheckFinished = 0;
|
||||
private long lastHBProcessedCount;
|
||||
// Should we create ChillModeManager and extract all the chill mode logic
|
||||
// to a new class?
|
||||
private int chillModeNodeCount;
|
||||
private final int maxHBToProcessPerLoop;
|
||||
private final String clusterID;
|
||||
private final VersionInfo version;
|
||||
/**
|
||||
@ -161,55 +117,26 @@ public class SCMNodeManager
|
||||
// Node pool manager.
|
||||
private final StorageContainerManager scmManager;
|
||||
|
||||
public static final Event<CommandForDatanode> DATANODE_COMMAND =
|
||||
new TypedEvent<>(CommandForDatanode.class, "DATANODE_COMMAND");
|
||||
|
||||
|
||||
/**
|
||||
* Constructs SCM machine Manager.
|
||||
*/
|
||||
public SCMNodeManager(OzoneConfiguration conf, String clusterID,
|
||||
StorageContainerManager scmManager) throws IOException {
|
||||
heartbeatQueue = new ConcurrentLinkedQueue<>();
|
||||
healthyNodes = new ConcurrentHashMap<>();
|
||||
deadNodes = new ConcurrentHashMap<>();
|
||||
staleNodes = new ConcurrentHashMap<>();
|
||||
nodes = new ConcurrentHashMap<>();
|
||||
nodeStats = new ConcurrentHashMap<>();
|
||||
scmStat = new SCMNodeStat();
|
||||
|
||||
healthyNodeCount = new AtomicInteger(0);
|
||||
staleNodeCount = new AtomicInteger(0);
|
||||
deadNodeCount = new AtomicInteger(0);
|
||||
totalNodes = new AtomicInteger(0);
|
||||
this.nodeStateManager = new NodeStateManager(conf);
|
||||
this.nodeStats = new ConcurrentHashMap<>();
|
||||
this.scmStat = new SCMNodeStat();
|
||||
this.clusterID = clusterID;
|
||||
this.version = VersionInfo.getLatestVersion();
|
||||
commandQueue = new CommandQueue();
|
||||
|
||||
this.commandQueue = new CommandQueue();
|
||||
// TODO: Support this value as a Percentage of known machines.
|
||||
chillModeNodeCount = 1;
|
||||
|
||||
staleNodeIntervalMs = HddsServerUtil.getStaleNodeInterval(conf);
|
||||
deadNodeIntervalMs = HddsServerUtil.getDeadNodeInterval(conf);
|
||||
heartbeatCheckerIntervalMs =
|
||||
HddsServerUtil.getScmheartbeatCheckerInterval(conf);
|
||||
datanodeHBIntervalSeconds = HddsServerUtil.getScmHeartbeatInterval(conf);
|
||||
maxHBToProcessPerLoop = HddsServerUtil.getMaxHBToProcessPerLoop(conf);
|
||||
|
||||
executorService = HadoopExecutors.newScheduledThreadPool(1,
|
||||
new ThreadFactoryBuilder().setDaemon(true)
|
||||
.setNameFormat("SCM Heartbeat Processing Thread - %d").build());
|
||||
|
||||
LOG.info("Entering startup chill mode.");
|
||||
this.chillModeNodeCount = 1;
|
||||
this.inStartupChillMode = new AtomicBoolean(true);
|
||||
this.inManualChillMode = new AtomicBoolean(false);
|
||||
|
||||
Preconditions.checkState(heartbeatCheckerIntervalMs > 0);
|
||||
executorService.schedule(this, heartbeatCheckerIntervalMs,
|
||||
TimeUnit.MILLISECONDS);
|
||||
|
||||
registerMXBean();
|
||||
|
||||
this.scmManager = scmManager;
|
||||
LOG.info("Entering startup chill mode.");
|
||||
registerMXBean();
|
||||
}
|
||||
|
||||
private void registerMXBean() {
|
||||
@ -228,12 +155,11 @@ private void unregisterMXBean() {
|
||||
* Removes a data node from the management of this Node Manager.
|
||||
*
|
||||
* @param node - DataNode.
|
||||
* @throws UnregisteredNodeException
|
||||
* @throws NodeNotFoundException
|
||||
*/
|
||||
@Override
|
||||
public void removeNode(DatanodeDetails node) {
|
||||
// TODO : Fix me when adding the SCM CLI.
|
||||
|
||||
public void removeNode(DatanodeDetails node) throws NodeNotFoundException {
|
||||
nodeStateManager.removeNode(node);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -245,31 +171,8 @@ public void removeNode(DatanodeDetails node) {
|
||||
* @return List of Datanodes that are known to SCM in the requested state.
|
||||
*/
|
||||
@Override
|
||||
public List<DatanodeDetails> getNodes(NodeState nodestate)
|
||||
throws IllegalArgumentException {
|
||||
Map<UUID, Long> set;
|
||||
switch (nodestate) {
|
||||
case HEALTHY:
|
||||
synchronized (this) {
|
||||
set = Collections.unmodifiableMap(new HashMap<>(healthyNodes));
|
||||
}
|
||||
break;
|
||||
case STALE:
|
||||
synchronized (this) {
|
||||
set = Collections.unmodifiableMap(new HashMap<>(staleNodes));
|
||||
}
|
||||
break;
|
||||
case DEAD:
|
||||
synchronized (this) {
|
||||
set = Collections.unmodifiableMap(new HashMap<>(deadNodes));
|
||||
}
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("Unknown node state requested.");
|
||||
}
|
||||
|
||||
return set.entrySet().stream().map(entry -> nodes.get(entry.getKey()))
|
||||
.collect(Collectors.toList());
|
||||
public List<DatanodeDetails> getNodes(NodeState nodestate) {
|
||||
return nodeStateManager.getNodes(nodestate);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -279,12 +182,7 @@ public List<DatanodeDetails> getNodes(NodeState nodestate)
|
||||
*/
|
||||
@Override
|
||||
public List<DatanodeDetails> getAllNodes() {
|
||||
Map<UUID, DatanodeDetails> set;
|
||||
synchronized (this) {
|
||||
set = Collections.unmodifiableMap(new HashMap<>(nodes));
|
||||
}
|
||||
return set.entrySet().stream().map(entry -> nodes.get(entry.getKey()))
|
||||
.collect(Collectors.toList());
|
||||
return nodeStateManager.getAllNodes();
|
||||
}
|
||||
|
||||
/**
|
||||
@ -316,14 +214,16 @@ public String getChillModeStatus() {
|
||||
if (inStartupChillMode.get()) {
|
||||
return "Still in chill mode, waiting on nodes to report in." +
|
||||
String.format(" %d nodes reported, minimal %d nodes required.",
|
||||
totalNodes.get(), getMinimumChillModeNodes());
|
||||
nodeStateManager.getTotalNodeCount(), getMinimumChillModeNodes());
|
||||
}
|
||||
if (inManualChillMode.get()) {
|
||||
return "Out of startup chill mode, but in manual chill mode." +
|
||||
String.format(" %d nodes have reported in.", totalNodes.get());
|
||||
String.format(" %d nodes have reported in.",
|
||||
nodeStateManager.getTotalNodeCount());
|
||||
}
|
||||
return "Out of chill mode." +
|
||||
String.format(" %d nodes have reported in.", totalNodes.get());
|
||||
String.format(" %d nodes have reported in.",
|
||||
nodeStateManager.getTotalNodeCount());
|
||||
}
|
||||
|
||||
/**
|
||||
@ -377,33 +277,7 @@ public boolean isOutOfChillMode() {
|
||||
*/
|
||||
@Override
|
||||
public int getNodeCount(NodeState nodestate) {
|
||||
switch (nodestate) {
|
||||
case HEALTHY:
|
||||
return healthyNodeCount.get();
|
||||
case STALE:
|
||||
return staleNodeCount.get();
|
||||
case DEAD:
|
||||
return deadNodeCount.get();
|
||||
case INVALID:
|
||||
// This is unknown due to the fact that some nodes can be in
|
||||
// transit between the other states. Returning a count for that is not
|
||||
// possible. The fact that we have such state is to deal with the fact
|
||||
// that this information might not be consistent always.
|
||||
return 0;
|
||||
default:
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Used for testing.
|
||||
*
|
||||
* @return true if the HB check is done.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@Override
|
||||
public boolean waitForHeartbeatProcessed() {
|
||||
return lastHBcheckFinished != 0;
|
||||
return nodeStateManager.getNodeCount(nodestate);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -414,236 +288,14 @@ public boolean waitForHeartbeatProcessed() {
|
||||
*/
|
||||
@Override
|
||||
public NodeState getNodeState(DatanodeDetails datanodeDetails) {
|
||||
// There is a subtle race condition here, hence we also support
|
||||
// the NODEState.UNKNOWN. It is possible that just before we check the
|
||||
// healthyNodes, we have removed the node from the healthy list but stil
|
||||
// not added it to Stale Nodes list.
|
||||
// We can fix that by adding the node to stale list before we remove, but
|
||||
// then the node is in 2 states to avoid this race condition. Instead we
|
||||
// just deal with the possibilty of getting a state called unknown.
|
||||
|
||||
UUID id = datanodeDetails.getUuid();
|
||||
if(healthyNodes.containsKey(id)) {
|
||||
return HEALTHY;
|
||||
}
|
||||
|
||||
if(staleNodes.containsKey(id)) {
|
||||
return STALE;
|
||||
}
|
||||
|
||||
if(deadNodes.containsKey(id)) {
|
||||
return DEAD;
|
||||
}
|
||||
|
||||
return INVALID;
|
||||
}
|
||||
|
||||
/**
|
||||
* This is the real worker thread that processes the HB queue. We do the
|
||||
* following things in this thread.
|
||||
* <p>
|
||||
* Process the Heartbeats that are in the HB Queue. Move Stale or Dead node to
|
||||
* healthy if we got a heartbeat from them. Move Stales Node to dead node
|
||||
* table if it is needed. Move healthy nodes to stale nodes if it is needed.
|
||||
* <p>
|
||||
* if it is a new node, we call register node and add it to the list of nodes.
|
||||
* This will be replaced when we support registration of a node in SCM.
|
||||
*
|
||||
* @see Thread#run()
|
||||
*/
|
||||
@Override
|
||||
public void run() {
|
||||
lastHBcheckStart = monotonicNow();
|
||||
lastHBProcessedCount = 0;
|
||||
|
||||
// Process the whole queue.
|
||||
while (!heartbeatQueue.isEmpty() &&
|
||||
(lastHBProcessedCount < maxHBToProcessPerLoop)) {
|
||||
HeartbeatQueueItem hbItem = heartbeatQueue.poll();
|
||||
synchronized (this) {
|
||||
handleHeartbeat(hbItem);
|
||||
}
|
||||
// we are shutting down or something give up processing the rest of
|
||||
// HBs. This will terminate the HB processing thread.
|
||||
if (Thread.currentThread().isInterrupted()) {
|
||||
LOG.info("Current Thread is isInterrupted, shutting down HB " +
|
||||
"processing thread for Node Manager.");
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
if (lastHBProcessedCount >= maxHBToProcessPerLoop) {
|
||||
LOG.error("SCM is being flooded by heartbeats. Not able to keep up with" +
|
||||
" the heartbeat counts. Processed {} heartbeats. Breaking out of" +
|
||||
" loop. Leaving rest to be processed later. ", lastHBProcessedCount);
|
||||
}
|
||||
|
||||
// Iterate over the Stale nodes and decide if we need to move any node to
|
||||
// dead State.
|
||||
long currentTime = monotonicNow();
|
||||
for (Map.Entry<UUID, Long> entry : staleNodes.entrySet()) {
|
||||
if (currentTime - entry.getValue() > deadNodeIntervalMs) {
|
||||
synchronized (this) {
|
||||
moveStaleNodeToDead(entry);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Iterate over the healthy nodes and decide if we need to move any node to
|
||||
// Stale State.
|
||||
currentTime = monotonicNow();
|
||||
for (Map.Entry<UUID, Long> entry : healthyNodes.entrySet()) {
|
||||
if (currentTime - entry.getValue() > staleNodeIntervalMs) {
|
||||
synchronized (this) {
|
||||
moveHealthyNodeToStale(entry);
|
||||
}
|
||||
}
|
||||
}
|
||||
lastHBcheckFinished = monotonicNow();
|
||||
|
||||
monitorHBProcessingTime();
|
||||
|
||||
// we purposefully make this non-deterministic. Instead of using a
|
||||
// scheduleAtFixedFrequency we will just go to sleep
|
||||
// and wake up at the next rendezvous point, which is currentTime +
|
||||
// heartbeatCheckerIntervalMs. This leads to the issue that we are now
|
||||
// heart beating not at a fixed cadence, but clock tick + time taken to
|
||||
// work.
|
||||
//
|
||||
// This time taken to work can skew the heartbeat processor thread.
|
||||
// The reason why we don't care is because of the following reasons.
|
||||
//
|
||||
// 1. checkerInterval is general many magnitudes faster than datanode HB
|
||||
// frequency.
|
||||
//
|
||||
// 2. if we have too much nodes, the SCM would be doing only HB
|
||||
// processing, this could lead to SCM's CPU starvation. With this
|
||||
// approach we always guarantee that HB thread sleeps for a little while.
|
||||
//
|
||||
// 3. It is possible that we will never finish processing the HB's in the
|
||||
// thread. But that means we have a mis-configured system. We will warn
|
||||
// the users by logging that information.
|
||||
//
|
||||
// 4. And the most important reason, heartbeats are not blocked even if
|
||||
// this thread does not run, they will go into the processing queue.
|
||||
|
||||
if (!Thread.currentThread().isInterrupted() &&
|
||||
!executorService.isShutdown()) {
|
||||
executorService.schedule(this, heartbeatCheckerIntervalMs, TimeUnit
|
||||
.MILLISECONDS);
|
||||
} else {
|
||||
LOG.info("Current Thread is interrupted, shutting down HB processing " +
|
||||
"thread for Node Manager.");
|
||||
try {
|
||||
return nodeStateManager.getNodeState(datanodeDetails);
|
||||
} catch (NodeNotFoundException e) {
|
||||
// TODO: should we throw NodeNotFoundException?
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* If we have taken too much time for HB processing, log that information.
|
||||
*/
|
||||
private void monitorHBProcessingTime() {
|
||||
if (TimeUnit.MILLISECONDS.toSeconds(lastHBcheckFinished -
|
||||
lastHBcheckStart) > datanodeHBIntervalSeconds) {
|
||||
LOG.error("Total time spend processing datanode HB's is greater than " +
|
||||
"configured values for datanode heartbeats. Please adjust the" +
|
||||
" heartbeat configs. Time Spend on HB processing: {} seconds " +
|
||||
"Datanode heartbeat Interval: {} seconds , heartbeats " +
|
||||
"processed: {}",
|
||||
TimeUnit.MILLISECONDS
|
||||
.toSeconds(lastHBcheckFinished - lastHBcheckStart),
|
||||
datanodeHBIntervalSeconds, lastHBProcessedCount);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Moves a Healthy node to a Stale node state.
|
||||
*
|
||||
* @param entry - Map Entry
|
||||
*/
|
||||
private void moveHealthyNodeToStale(Map.Entry<UUID, Long> entry) {
|
||||
LOG.trace("Moving healthy node to stale: {}", entry.getKey());
|
||||
healthyNodes.remove(entry.getKey());
|
||||
healthyNodeCount.decrementAndGet();
|
||||
staleNodes.put(entry.getKey(), entry.getValue());
|
||||
staleNodeCount.incrementAndGet();
|
||||
|
||||
if (scmManager != null) {
|
||||
// remove stale node's container report
|
||||
scmManager.removeContainerReport(entry.getKey().toString());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Moves a Stale node to a dead node state.
|
||||
*
|
||||
* @param entry - Map Entry
|
||||
*/
|
||||
private void moveStaleNodeToDead(Map.Entry<UUID, Long> entry) {
|
||||
LOG.trace("Moving stale node to dead: {}", entry.getKey());
|
||||
staleNodes.remove(entry.getKey());
|
||||
staleNodeCount.decrementAndGet();
|
||||
deadNodes.put(entry.getKey(), entry.getValue());
|
||||
deadNodeCount.incrementAndGet();
|
||||
|
||||
// Update SCM node stats
|
||||
SCMNodeStat deadNodeStat = nodeStats.get(entry.getKey());
|
||||
scmStat.subtract(deadNodeStat);
|
||||
nodeStats.remove(entry.getKey());
|
||||
}
|
||||
|
||||
/**
|
||||
* Handles a single heartbeat from a datanode.
|
||||
*
|
||||
* @param hbItem - heartbeat item from a datanode.
|
||||
*/
|
||||
private void handleHeartbeat(HeartbeatQueueItem hbItem) {
|
||||
lastHBProcessedCount++;
|
||||
|
||||
DatanodeDetails datanodeDetails = hbItem.getDatanodeDetails();
|
||||
UUID datanodeUuid = datanodeDetails.getUuid();
|
||||
NodeReportProto nodeReport = hbItem.getNodeReport();
|
||||
long recvTimestamp = hbItem.getRecvTimestamp();
|
||||
long processTimestamp = Time.monotonicNow();
|
||||
if (LOG.isTraceEnabled()) {
|
||||
//TODO: add average queue time of heartbeat request as metrics
|
||||
LOG.trace("Processing Heartbeat from datanode {}: queueing time {}",
|
||||
datanodeUuid, processTimestamp - recvTimestamp);
|
||||
}
|
||||
|
||||
// If this node is already in the list of known and healthy nodes
|
||||
// just set the last timestamp and return.
|
||||
if (healthyNodes.containsKey(datanodeUuid)) {
|
||||
healthyNodes.put(datanodeUuid, processTimestamp);
|
||||
updateNodeStat(datanodeUuid, nodeReport);
|
||||
return;
|
||||
}
|
||||
|
||||
// A stale node has heartbeat us we need to remove the node from stale
|
||||
// list and move to healthy list.
|
||||
if (staleNodes.containsKey(datanodeUuid)) {
|
||||
staleNodes.remove(datanodeUuid);
|
||||
healthyNodes.put(datanodeUuid, processTimestamp);
|
||||
healthyNodeCount.incrementAndGet();
|
||||
staleNodeCount.decrementAndGet();
|
||||
updateNodeStat(datanodeUuid, nodeReport);
|
||||
return;
|
||||
}
|
||||
|
||||
// A dead node has heartbeat us, we need to remove that node from dead
|
||||
// node list and move it to the healthy list.
|
||||
if (deadNodes.containsKey(datanodeUuid)) {
|
||||
deadNodes.remove(datanodeUuid);
|
||||
healthyNodes.put(datanodeUuid, processTimestamp);
|
||||
deadNodeCount.decrementAndGet();
|
||||
healthyNodeCount.incrementAndGet();
|
||||
updateNodeStat(datanodeUuid, nodeReport);
|
||||
return;
|
||||
}
|
||||
|
||||
LOG.warn("SCM receive heartbeat from unregistered datanode {}",
|
||||
datanodeUuid);
|
||||
this.commandQueue.addCommand(datanodeUuid,
|
||||
new ReregisterCommand());
|
||||
}
|
||||
|
||||
private void updateNodeStat(UUID dnId, NodeReportProto nodeReport) {
|
||||
SCMNodeStat stat = nodeStats.get(dnId);
|
||||
@ -680,24 +332,6 @@ private void updateNodeStat(UUID dnId, NodeReportProto nodeReport) {
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
unregisterMXBean();
|
||||
executorService.shutdown();
|
||||
try {
|
||||
if (!executorService.awaitTermination(5, TimeUnit.SECONDS)) {
|
||||
executorService.shutdownNow();
|
||||
}
|
||||
|
||||
if (!executorService.awaitTermination(5, TimeUnit.SECONDS)) {
|
||||
LOG.error("Unable to shutdown NodeManager properly.");
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
executorService.shutdownNow();
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
long getLastHBProcessedCount() {
|
||||
return lastHBProcessedCount;
|
||||
}
|
||||
|
||||
/**
|
||||
@ -743,27 +377,22 @@ public RegisteredCommand register(
|
||||
datanodeDetails.setHostName(hostname);
|
||||
datanodeDetails.setIpAddress(ip);
|
||||
}
|
||||
RegisteredCommand responseCommand = verifyDatanodeUUID(datanodeDetails);
|
||||
if (responseCommand != null) {
|
||||
return responseCommand;
|
||||
}
|
||||
UUID dnId = datanodeDetails.getUuid();
|
||||
nodes.put(dnId, datanodeDetails);
|
||||
totalNodes.incrementAndGet();
|
||||
healthyNodes.put(dnId, monotonicNow());
|
||||
healthyNodeCount.incrementAndGet();
|
||||
nodeStats.put(dnId, new SCMNodeStat());
|
||||
|
||||
if(inStartupChillMode.get() &&
|
||||
totalNodes.get() >= getMinimumChillModeNodes()) {
|
||||
inStartupChillMode.getAndSet(false);
|
||||
LOG.info("Leaving startup chill mode.");
|
||||
try {
|
||||
nodeStateManager.addNode(datanodeDetails);
|
||||
nodeStats.put(dnId, new SCMNodeStat());
|
||||
if(inStartupChillMode.get() &&
|
||||
nodeStateManager.getTotalNodeCount() >= getMinimumChillModeNodes()) {
|
||||
inStartupChillMode.getAndSet(false);
|
||||
LOG.info("Leaving startup chill mode.");
|
||||
}
|
||||
// Updating Node Report, as registration is successful
|
||||
updateNodeStat(datanodeDetails.getUuid(), nodeReport);
|
||||
LOG.info("Data node with ID: {} Registered.", datanodeDetails.getUuid());
|
||||
} catch (NodeAlreadyExistsException e) {
|
||||
LOG.trace("Datanode is already registered. Datanode: {}",
|
||||
datanodeDetails.toString());
|
||||
}
|
||||
|
||||
// Updating Node Report, as registration is successful
|
||||
updateNodeStat(datanodeDetails.getUuid(), nodeReport);
|
||||
LOG.info("Data node with ID: {} Registered.",
|
||||
datanodeDetails.getUuid());
|
||||
RegisteredCommand.Builder builder =
|
||||
RegisteredCommand.newBuilder().setErrorCode(ErrorCode.success)
|
||||
.setDatanodeUUID(datanodeDetails.getUuidString())
|
||||
@ -774,46 +403,25 @@ public RegisteredCommand register(
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Verifies the datanode does not have a valid UUID already.
|
||||
*
|
||||
* @param datanodeDetails - Datanode Details.
|
||||
* @return SCMCommand
|
||||
*/
|
||||
private RegisteredCommand verifyDatanodeUUID(
|
||||
DatanodeDetails datanodeDetails) {
|
||||
if (datanodeDetails.getUuid() != null &&
|
||||
nodes.containsKey(datanodeDetails.getUuid())) {
|
||||
LOG.trace("Datanode is already registered. Datanode: {}",
|
||||
datanodeDetails.toString());
|
||||
return RegisteredCommand.newBuilder()
|
||||
.setErrorCode(ErrorCode.success)
|
||||
.setClusterID(this.clusterID)
|
||||
.setDatanodeUUID(datanodeDetails.getUuidString())
|
||||
.build();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Send heartbeat to indicate the datanode is alive and doing well.
|
||||
*
|
||||
* @param datanodeDetails - DatanodeDetailsProto.
|
||||
* @param nodeReport - node report.
|
||||
* @return SCMheartbeat response.
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public List<SCMCommand> sendHeartbeat(
|
||||
DatanodeDetails datanodeDetails, NodeReportProto nodeReport) {
|
||||
|
||||
public List<SCMCommand> processHeartbeat(DatanodeDetails datanodeDetails) {
|
||||
Preconditions.checkNotNull(datanodeDetails, "Heartbeat is missing " +
|
||||
"DatanodeDetails.");
|
||||
heartbeatQueue.add(
|
||||
new HeartbeatQueueItem.Builder()
|
||||
.setDatanodeDetails(datanodeDetails)
|
||||
.setNodeReport(nodeReport)
|
||||
.build());
|
||||
try {
|
||||
nodeStateManager.updateLastHeartbeatTime(datanodeDetails);
|
||||
} catch (NodeNotFoundException e) {
|
||||
LOG.warn("SCM receive heartbeat from unregistered datanode {}",
|
||||
datanodeDetails);
|
||||
commandQueue.addCommand(datanodeDetails.getUuid(),
|
||||
new ReregisterCommand());
|
||||
}
|
||||
return commandQueue.getCommand(datanodeDetails.getUuid());
|
||||
}
|
||||
|
||||
@ -859,11 +467,6 @@ public void addDatanodeCommand(UUID dnId, SCMCommand command) {
|
||||
this.commandQueue.addCommand(dnId, command);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void setStaleNodeIntervalMs(long interval) {
|
||||
this.staleNodeIntervalMs = interval;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onMessage(CommandForDatanode commandForDatanode,
|
||||
EventPublisher publisher) {
|
||||
|
@ -0,0 +1,45 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with this
|
||||
* work for additional information regarding copyright ownership. The ASF
|
||||
* licenses this file to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
* <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.node.states;
|
||||
|
||||
/**
|
||||
* This exception represents that there is already a node added to NodeStateMap
|
||||
* with same UUID.
|
||||
*/
|
||||
public class NodeAlreadyExistsException extends NodeException {
|
||||
|
||||
/**
|
||||
* Constructs an {@code NodeAlreadyExistsException} with {@code null}
|
||||
* as its error detail message.
|
||||
*/
|
||||
public NodeAlreadyExistsException() {
|
||||
super();
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs an {@code NodeAlreadyExistsException} with the specified
|
||||
* detail message.
|
||||
*
|
||||
* @param message
|
||||
* The detail message (which is saved for later retrieval
|
||||
* by the {@link #getMessage()} method)
|
||||
*/
|
||||
public NodeAlreadyExistsException(String message) {
|
||||
super(message);
|
||||
}
|
||||
}
|
@ -0,0 +1,44 @@
|
||||
/**
|
||||
* 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.node.states;
|
||||
|
||||
/**
|
||||
* This exception represents all node related exceptions in NodeStateMap.
|
||||
*/
|
||||
public class NodeException extends Exception {
|
||||
|
||||
/**
|
||||
* Constructs an {@code NodeException} with {@code null}
|
||||
* as its error detail message.
|
||||
*/
|
||||
public NodeException() {
|
||||
super();
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs an {@code NodeException} with the specified
|
||||
* detail message.
|
||||
*
|
||||
* @param message
|
||||
* The detail message (which is saved for later retrieval
|
||||
* by the {@link #getMessage()} method)
|
||||
*/
|
||||
public NodeException(String message) {
|
||||
super(message);
|
||||
}
|
||||
}
|
@ -0,0 +1,49 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with this
|
||||
* work for additional information regarding copyright ownership. The ASF
|
||||
* licenses this file to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
* <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.node.states;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* This exception represents that the node that is being accessed does not
|
||||
* exist in NodeStateMap.
|
||||
*/
|
||||
public class NodeNotFoundException extends NodeException {
|
||||
|
||||
|
||||
/**
|
||||
* Constructs an {@code NodeNotFoundException} with {@code null}
|
||||
* as its error detail message.
|
||||
*/
|
||||
public NodeNotFoundException() {
|
||||
super();
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs an {@code NodeNotFoundException} with the specified
|
||||
* detail message.
|
||||
*
|
||||
* @param message
|
||||
* The detail message (which is saved for later retrieval
|
||||
* by the {@link #getMessage()} method)
|
||||
*/
|
||||
public NodeNotFoundException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,281 @@
|
||||
/**
|
||||
* 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.node.states;
|
||||
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
|
||||
import org.apache.hadoop.hdds.scm.node.DatanodeInfo;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.locks.ReadWriteLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
/**
|
||||
* Maintains the state of datanodes in SCM. This class should only be used by
|
||||
* NodeStateManager to maintain the state. If anyone wants to change the
|
||||
* state of a node they should call NodeStateManager, do not directly use
|
||||
* this class.
|
||||
*/
|
||||
public class NodeStateMap {
|
||||
|
||||
/**
|
||||
* Node id to node info map.
|
||||
*/
|
||||
private final ConcurrentHashMap<UUID, DatanodeInfo> nodeMap;
|
||||
/**
|
||||
* Represents the current state of node.
|
||||
*/
|
||||
private final ConcurrentHashMap<NodeState, Set<UUID>> stateMap;
|
||||
private final ReadWriteLock lock;
|
||||
|
||||
/**
|
||||
* Creates a new instance of NodeStateMap with no nodes.
|
||||
*/
|
||||
public NodeStateMap() {
|
||||
lock = new ReentrantReadWriteLock();
|
||||
nodeMap = new ConcurrentHashMap<>();
|
||||
stateMap = new ConcurrentHashMap<>();
|
||||
initStateMap();
|
||||
}
|
||||
|
||||
/**
|
||||
* Initializes the state map with available states.
|
||||
*/
|
||||
private void initStateMap() {
|
||||
for (NodeState state : NodeState.values()) {
|
||||
stateMap.put(state, new HashSet<>());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a node to NodeStateMap.
|
||||
*
|
||||
* @param datanodeDetails DatanodeDetails
|
||||
* @param nodeState initial NodeState
|
||||
*
|
||||
* @throws NodeAlreadyExistsException if the node already exist
|
||||
*/
|
||||
public void addNode(DatanodeDetails datanodeDetails, NodeState nodeState)
|
||||
throws NodeAlreadyExistsException {
|
||||
lock.writeLock().lock();
|
||||
try {
|
||||
UUID id = datanodeDetails.getUuid();
|
||||
if (nodeMap.containsKey(id)) {
|
||||
throw new NodeAlreadyExistsException("Node UUID: " + id);
|
||||
}
|
||||
nodeMap.put(id, new DatanodeInfo(datanodeDetails));
|
||||
stateMap.get(nodeState).add(id);
|
||||
} finally {
|
||||
lock.writeLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the node state.
|
||||
*
|
||||
* @param nodeId Node Id
|
||||
* @param currentState current state
|
||||
* @param newState new state
|
||||
*
|
||||
* @throws NodeNotFoundException if the node is not present
|
||||
*/
|
||||
public void updateNodeState(UUID nodeId, NodeState currentState,
|
||||
NodeState newState)throws NodeNotFoundException {
|
||||
lock.writeLock().lock();
|
||||
try {
|
||||
if (stateMap.get(currentState).remove(nodeId)) {
|
||||
stateMap.get(newState).add(nodeId);
|
||||
} else {
|
||||
throw new NodeNotFoundException("Node UUID: " + nodeId +
|
||||
", not found in state: " + currentState);
|
||||
}
|
||||
} finally {
|
||||
lock.writeLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns DatanodeDetails for the given node id.
|
||||
*
|
||||
* @param uuid Node Id
|
||||
*
|
||||
* @return DatanodeDetails of the node
|
||||
*
|
||||
* @throws NodeNotFoundException if the node is not present
|
||||
*/
|
||||
public DatanodeDetails getNodeDetails(UUID uuid)
|
||||
throws NodeNotFoundException {
|
||||
return getNodeInfo(uuid);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns DatanodeInfo for the given node id.
|
||||
*
|
||||
* @param uuid Node Id
|
||||
*
|
||||
* @return DatanodeInfo of the node
|
||||
*
|
||||
* @throws NodeNotFoundException if the node is not present
|
||||
*/
|
||||
public DatanodeInfo getNodeInfo(UUID uuid) throws NodeNotFoundException {
|
||||
lock.readLock().lock();
|
||||
try {
|
||||
if (nodeMap.containsKey(uuid)) {
|
||||
return nodeMap.get(uuid);
|
||||
}
|
||||
throw new NodeNotFoundException("Node UUID: " + uuid);
|
||||
} finally {
|
||||
lock.readLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Returns the list of node ids which are in the specified state.
|
||||
*
|
||||
* @param state NodeState
|
||||
*
|
||||
* @return list of node ids
|
||||
*/
|
||||
public List<UUID> getNodes(NodeState state) {
|
||||
lock.readLock().lock();
|
||||
try {
|
||||
return new LinkedList<>(stateMap.get(state));
|
||||
} finally {
|
||||
lock.readLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the list of all the node ids.
|
||||
*
|
||||
* @return list of all the node ids
|
||||
*/
|
||||
public List<UUID> getAllNodes() {
|
||||
lock.readLock().lock();
|
||||
try {
|
||||
return new LinkedList<>(nodeMap.keySet());
|
||||
} finally {
|
||||
lock.readLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the count of nodes in the specified state.
|
||||
*
|
||||
* @param state NodeState
|
||||
*
|
||||
* @return Number of nodes in the specified state
|
||||
*/
|
||||
public int getNodeCount(NodeState state) {
|
||||
lock.readLock().lock();
|
||||
try {
|
||||
return stateMap.get(state).size();
|
||||
} finally {
|
||||
lock.readLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the total node count.
|
||||
*
|
||||
* @return node count
|
||||
*/
|
||||
public int getTotalNodeCount() {
|
||||
lock.readLock().lock();
|
||||
try {
|
||||
return nodeMap.size();
|
||||
} finally {
|
||||
lock.readLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the current state of the node.
|
||||
*
|
||||
* @param uuid node id
|
||||
*
|
||||
* @return NodeState
|
||||
*
|
||||
* @throws NodeNotFoundException if the node is not found
|
||||
*/
|
||||
public NodeState getNodeState(UUID uuid) throws NodeNotFoundException {
|
||||
lock.readLock().lock();
|
||||
try {
|
||||
for (Map.Entry<NodeState, Set<UUID>> entry : stateMap.entrySet()) {
|
||||
if (entry.getValue().contains(uuid)) {
|
||||
return entry.getKey();
|
||||
}
|
||||
}
|
||||
throw new NodeNotFoundException("Node UUID: " + uuid);
|
||||
} finally {
|
||||
lock.readLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes the node from NodeStateMap.
|
||||
*
|
||||
* @param uuid node id
|
||||
*
|
||||
* @throws NodeNotFoundException if the node is not found
|
||||
*/
|
||||
public void removeNode(UUID uuid) throws NodeNotFoundException {
|
||||
lock.writeLock().lock();
|
||||
try {
|
||||
if (nodeMap.containsKey(uuid)) {
|
||||
for (Map.Entry<NodeState, Set<UUID>> entry : stateMap.entrySet()) {
|
||||
if(entry.getValue().remove(uuid)) {
|
||||
break;
|
||||
}
|
||||
nodeMap.remove(uuid);
|
||||
}
|
||||
throw new NodeNotFoundException("Node UUID: " + uuid);
|
||||
}
|
||||
} finally {
|
||||
lock.writeLock().unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Since we don't hold a global lock while constructing this string,
|
||||
* the result might be inconsistent. If someone has changed the state of node
|
||||
* while we are constructing the string, the result will be inconsistent.
|
||||
* This should only be used for logging. We should not parse this string and
|
||||
* use it for any critical calculations.
|
||||
*
|
||||
* @return current state of NodeStateMap
|
||||
*/
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append("Total number of nodes: ").append(getTotalNodeCount());
|
||||
for (NodeState state : NodeState.values()) {
|
||||
builder.append("Number of nodes in ").append(state).append(" state: ")
|
||||
.append(getNodeCount(state));
|
||||
}
|
||||
return builder.toString();
|
||||
}
|
||||
}
|
@ -16,6 +16,9 @@
|
||||
*/
|
||||
package org.apache.hadoop.hdds.scm.pipelines;
|
||||
|
||||
import java.util.LinkedList;
|
||||
import java.util.Map;
|
||||
import java.util.WeakHashMap;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
||||
@ -25,7 +28,6 @@
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
@ -36,11 +38,13 @@ public abstract class PipelineManager {
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(PipelineManager.class);
|
||||
private final List<Pipeline> activePipelines;
|
||||
private final Map<String, Pipeline> activePipelineMap;
|
||||
private final AtomicInteger pipelineIndex;
|
||||
|
||||
public PipelineManager() {
|
||||
activePipelines = new LinkedList<>();
|
||||
pipelineIndex = new AtomicInteger(0);
|
||||
activePipelineMap = new WeakHashMap<>();
|
||||
}
|
||||
|
||||
/**
|
||||
@ -76,6 +80,7 @@ public synchronized final Pipeline getPipeline(
|
||||
"replicationType:{} replicationFactor:{}",
|
||||
pipeline.getPipelineName(), replicationType, replicationFactor);
|
||||
activePipelines.add(pipeline);
|
||||
activePipelineMap.put(pipeline.getPipelineName(), pipeline);
|
||||
} else {
|
||||
pipeline =
|
||||
findOpenPipeline(replicationType, replicationFactor);
|
||||
@ -94,6 +99,26 @@ public synchronized final Pipeline getPipeline(
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This function to get pipeline with given pipeline name.
|
||||
*
|
||||
* @param pipelineName
|
||||
* @return a Pipeline.
|
||||
*/
|
||||
public synchronized final Pipeline getPipeline(String pipelineName) {
|
||||
Pipeline pipeline = null;
|
||||
|
||||
// 1. Check if pipeline channel already exists
|
||||
if (activePipelineMap.containsKey(pipelineName)) {
|
||||
pipeline = activePipelineMap.get(pipelineName);
|
||||
LOG.debug("Returning pipeline for pipelineName:{}", pipelineName);
|
||||
return pipeline;
|
||||
} else {
|
||||
LOG.debug("Unable to find pipeline for pipelineName:{}", pipelineName);
|
||||
}
|
||||
return pipeline;
|
||||
}
|
||||
|
||||
protected int getReplicationCount(ReplicationFactor factor) {
|
||||
switch (factor) {
|
||||
case ONE:
|
||||
|
@ -19,6 +19,7 @@
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||
import org.apache.hadoop.hdds.scm.container.placement.algorithms
|
||||
.ContainerPlacementPolicy;
|
||||
@ -176,6 +177,21 @@ public Pipeline getReplicationPipeline(ReplicationType replicationType,
|
||||
getPipeline(replicationFactor, replicationType);
|
||||
}
|
||||
|
||||
/**
|
||||
* This function to return pipeline for given pipeline name and replication
|
||||
* type.
|
||||
*/
|
||||
public Pipeline getPipeline(String pipelineName,
|
||||
ReplicationType replicationType) throws IOException {
|
||||
if (pipelineName == null) {
|
||||
return null;
|
||||
}
|
||||
PipelineManager manager = getPipelineManager(replicationType);
|
||||
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
|
||||
LOG.debug("Getting replication pipeline forReplicationType {} :" +
|
||||
" pipelineName:{}", replicationType, pipelineName);
|
||||
return manager.getPipeline(pipelineName);
|
||||
}
|
||||
/**
|
||||
* Creates a pipeline from a specified set of Nodes.
|
||||
*/
|
||||
|
@ -19,6 +19,7 @@
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdds.scm.XceiverClientRatis;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||
import org.apache.hadoop.hdds.scm.container.placement.algorithms
|
||||
.ContainerPlacementPolicy;
|
||||
|
@ -17,6 +17,7 @@
|
||||
package org.apache.hadoop.hdds.scm.pipelines.standalone;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||
import org.apache.hadoop.hdds.scm.container.placement.algorithms
|
||||
.ContainerPlacementPolicy;
|
||||
|
@ -152,7 +152,7 @@ public AllocatedBlock allocateBlock(long size, HddsProtos.ReplicationType
|
||||
@Override
|
||||
public List<DeleteBlockGroupResult> deleteKeyBlocks(
|
||||
List<BlockGroup> keyBlocksInfoList) throws IOException {
|
||||
LOG.info("SCM is informed by KSM to delete {} blocks", keyBlocksInfoList
|
||||
LOG.info("SCM is informed by OM to delete {} blocks", keyBlocksInfoList
|
||||
.size());
|
||||
List<DeleteBlockGroupResult> results = new ArrayList<>();
|
||||
for (BlockGroup keyBlocks : keyBlocksInfoList) {
|
||||
|
@ -31,6 +31,7 @@
|
||||
.StorageContainerLocationProtocolProtos;
|
||||
import org.apache.hadoop.hdds.scm.HddsServerUtil;
|
||||
import org.apache.hadoop.hdds.scm.ScmInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
|
||||
@ -46,7 +47,7 @@
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.EnumSet;
|
||||
import java.util.ArrayList;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
@ -145,11 +146,12 @@ public String getRpcRemoteUsername() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public ContainerInfo allocateContainer(HddsProtos.ReplicationType
|
||||
public ContainerWithPipeline allocateContainer(HddsProtos.ReplicationType
|
||||
replicationType, HddsProtos.ReplicationFactor factor,
|
||||
String owner) throws IOException {
|
||||
String remoteUser = getRpcRemoteUsername();
|
||||
getScm().checkAdminAccess(remoteUser);
|
||||
|
||||
return scm.getScmContainerManager()
|
||||
.allocateContainer(replicationType, factor, owner);
|
||||
}
|
||||
@ -162,6 +164,14 @@ public ContainerInfo getContainer(long containerID) throws IOException {
|
||||
.getContainer(containerID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ContainerWithPipeline getContainerWithPipeline(long containerID) throws IOException {
|
||||
String remoteUser = getRpcRemoteUsername();
|
||||
getScm().checkAdminAccess(remoteUser);
|
||||
return scm.getScmContainerManager()
|
||||
.getContainerWithPipeline(containerID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ContainerInfo> listContainer(long startContainerID,
|
||||
int count) throws IOException {
|
||||
@ -178,27 +188,21 @@ public void deleteContainer(long containerID) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public HddsProtos.NodePool queryNode(EnumSet<HddsProtos.NodeState>
|
||||
nodeStatuses, HddsProtos.QueryScope queryScope, String poolName) throws
|
||||
public List<HddsProtos.Node> queryNode(HddsProtos.NodeState state,
|
||||
HddsProtos.QueryScope queryScope, String poolName) throws
|
||||
IOException {
|
||||
|
||||
if (queryScope == HddsProtos.QueryScope.POOL) {
|
||||
throw new IllegalArgumentException("Not Supported yet");
|
||||
}
|
||||
|
||||
List<DatanodeDetails> datanodes = queryNode(nodeStatuses);
|
||||
HddsProtos.NodePool.Builder poolBuilder = HddsProtos.NodePool.newBuilder();
|
||||
List<HddsProtos.Node> result = new ArrayList<>();
|
||||
queryNode(state).forEach(node -> result.add(HddsProtos.Node.newBuilder()
|
||||
.setNodeID(node.getProtoBufMessage())
|
||||
.addNodeStates(state)
|
||||
.build()));
|
||||
|
||||
for (DatanodeDetails datanode : datanodes) {
|
||||
HddsProtos.Node node =
|
||||
HddsProtos.Node.newBuilder()
|
||||
.setNodeID(datanode.getProtoBufMessage())
|
||||
.addAllNodeStates(nodeStatuses)
|
||||
.build();
|
||||
poolBuilder.addNodes(node);
|
||||
}
|
||||
|
||||
return poolBuilder.build();
|
||||
return result;
|
||||
|
||||
}
|
||||
|
||||
@ -248,7 +252,7 @@ public Pipeline createReplicationPipeline(HddsProtos.ReplicationType type,
|
||||
throws IOException {
|
||||
// TODO: will be addressed in future patch.
|
||||
// This is needed only for debugging purposes to make sure cluster is
|
||||
// working correctly.
|
||||
// working correctly.
|
||||
return null;
|
||||
}
|
||||
|
||||
@ -272,35 +276,12 @@ public ScmInfo getScmInfo() throws IOException {
|
||||
* operation between the
|
||||
* operators.
|
||||
*
|
||||
* @param nodeStatuses - A set of NodeStates.
|
||||
* @param state - NodeStates.
|
||||
* @return List of Datanodes.
|
||||
*/
|
||||
public List<DatanodeDetails> queryNode(EnumSet<HddsProtos.NodeState>
|
||||
nodeStatuses) {
|
||||
Preconditions.checkNotNull(nodeStatuses, "Node Query set cannot be null");
|
||||
Preconditions.checkState(nodeStatuses.size() > 0, "No valid arguments " +
|
||||
"in the query set");
|
||||
List<DatanodeDetails> resultList = new LinkedList<>();
|
||||
Set<DatanodeDetails> currentSet = new TreeSet<>();
|
||||
|
||||
for (HddsProtos.NodeState nodeState : nodeStatuses) {
|
||||
Set<DatanodeDetails> nextSet = queryNodeState(nodeState);
|
||||
if ((nextSet == null) || (nextSet.size() == 0)) {
|
||||
// Right now we only support AND operation. So intersect with
|
||||
// any empty set is null.
|
||||
return resultList;
|
||||
}
|
||||
// First time we have to add all the elements, next time we have to
|
||||
// do an intersection operation on the set.
|
||||
if (currentSet.size() == 0) {
|
||||
currentSet.addAll(nextSet);
|
||||
} else {
|
||||
currentSet.retainAll(nextSet);
|
||||
}
|
||||
}
|
||||
|
||||
resultList.addAll(currentSet);
|
||||
return resultList;
|
||||
public List<DatanodeDetails> queryNode(HddsProtos.NodeState state) {
|
||||
Preconditions.checkNotNull(state, "Node Query set cannot be null");
|
||||
return new LinkedList<>(queryNodeState(state));
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@ -315,11 +296,6 @@ public StorageContainerManager getScm() {
|
||||
* @return Set of Datanodes that match the NodeState.
|
||||
*/
|
||||
private Set<DatanodeDetails> queryNodeState(HddsProtos.NodeState nodeState) {
|
||||
if (nodeState == HddsProtos.NodeState.RAFT_MEMBER || nodeState ==
|
||||
HddsProtos.NodeState
|
||||
.FREE_NODE) {
|
||||
throw new IllegalStateException("Not implemented yet");
|
||||
}
|
||||
Set<DatanodeDetails> returnSet = new TreeSet<>();
|
||||
List<DatanodeDetails> tmp = scm.getScmNodeManager().getNodes(nodeState);
|
||||
if ((tmp != null) && (tmp.size() > 0)) {
|
||||
|
@ -25,12 +25,14 @@
|
||||
import org.apache.hadoop.hdds.protocol.proto
|
||||
.StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto;
|
||||
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
||||
import org.apache.hadoop.hdds.server.events.TypedEvent;
|
||||
|
||||
import com.google.protobuf.GeneratedMessage;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import static org.apache.hadoop.hdds.scm.events.SCMEvents.CONTAINER_REPORT;
|
||||
import static org.apache.hadoop.hdds.scm.events.SCMEvents.NODE_REPORT;
|
||||
|
||||
/**
|
||||
* This class is responsible for dispatching heartbeat from datanode to
|
||||
* appropriate EventHandler at SCM.
|
||||
@ -42,11 +44,6 @@ public final class SCMDatanodeHeartbeatDispatcher {
|
||||
|
||||
private EventPublisher eventPublisher;
|
||||
|
||||
public static final TypedEvent<NodeReportFromDatanode> NODE_REPORT =
|
||||
new TypedEvent<>(NodeReportFromDatanode.class);
|
||||
|
||||
public static final TypedEvent<ContainerReportFromDatanode> CONTAINER_REPORT =
|
||||
new TypedEvent<ContainerReportFromDatanode>(ContainerReportFromDatanode.class);
|
||||
|
||||
public SCMDatanodeHeartbeatDispatcher(EventPublisher eventPublisher) {
|
||||
this.eventPublisher = eventPublisher;
|
||||
@ -61,14 +58,16 @@ public SCMDatanodeHeartbeatDispatcher(EventPublisher eventPublisher) {
|
||||
public void dispatch(SCMHeartbeatRequestProto heartbeat) {
|
||||
DatanodeDetails datanodeDetails =
|
||||
DatanodeDetails.getFromProtoBuf(heartbeat.getDatanodeDetails());
|
||||
|
||||
// should we dispatch heartbeat through eventPublisher?
|
||||
if (heartbeat.hasNodeReport()) {
|
||||
LOG.debug("Dispatching Node Report.");
|
||||
eventPublisher.fireEvent(NODE_REPORT,
|
||||
new NodeReportFromDatanode(datanodeDetails,
|
||||
heartbeat.getNodeReport()));
|
||||
}
|
||||
|
||||
if (heartbeat.hasContainerReport()) {
|
||||
LOG.debug("Dispatching Container Report.");
|
||||
eventPublisher.fireEvent(CONTAINER_REPORT,
|
||||
new ContainerReportFromDatanode(datanodeDetails,
|
||||
heartbeat.getContainerReport()));
|
||||
|
@ -223,7 +223,7 @@ public SCMHeartbeatResponseProto sendHeartbeat(
|
||||
.getFromProtoBuf(heartbeat.getDatanodeDetails());
|
||||
NodeReportProto nodeReport = heartbeat.getNodeReport();
|
||||
List<SCMCommand> commands =
|
||||
scm.getScmNodeManager().sendHeartbeat(datanodeDetails, nodeReport);
|
||||
scm.getScmNodeManager().processHeartbeat(datanodeDetails);
|
||||
List<SCMCommandProto> cmdResponses = new LinkedList<>();
|
||||
for (SCMCommand cmd : commands) {
|
||||
cmdResponses.add(getCommandResponse(cmd));
|
||||
|
@ -70,6 +70,8 @@
|
||||
|
||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DB_CACHE_SIZE_DEFAULT;
|
||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DB_CACHE_SIZE_MB;
|
||||
|
||||
import static org.apache.hadoop.hdds.scm.events.SCMEvents.DATANODE_COMMAND;
|
||||
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED;
|
||||
import static org.apache.hadoop.util.ExitUtil.terminate;
|
||||
|
||||
@ -164,9 +166,10 @@ private StorageContainerManager(OzoneConfiguration conf) throws IOException {
|
||||
}
|
||||
EventQueue eventQueue = new EventQueue();
|
||||
|
||||
SCMNodeManager nm = new SCMNodeManager(conf, scmStorage.getClusterID(), this);
|
||||
SCMNodeManager nm =
|
||||
new SCMNodeManager(conf, scmStorage.getClusterID(), this);
|
||||
scmNodeManager = nm;
|
||||
eventQueue.addHandler(SCMNodeManager.DATANODE_COMMAND, nm);
|
||||
eventQueue.addHandler(DATANODE_COMMAND, nm);
|
||||
|
||||
scmContainerManager = new ContainerMapping(conf, getScmNodeManager(),
|
||||
cacheSize);
|
||||
|
@ -22,6 +22,7 @@
|
||||
import org.apache.hadoop.hdds.scm.container.ContainerMapping;
|
||||
import org.apache.hadoop.hdds.scm.container.Mapping;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||
import org.apache.hadoop.hdfs.DFSUtil;
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
@ -362,10 +363,16 @@ private void mockContainerInfo(Mapping mappingService, long containerID,
|
||||
pipeline.addMember(dd);
|
||||
|
||||
ContainerInfo.Builder builder = new ContainerInfo.Builder();
|
||||
builder.setPipeline(pipeline);
|
||||
builder.setPipelineName(pipeline.getPipelineName())
|
||||
.setReplicationType(pipeline.getType())
|
||||
.setReplicationFactor(pipeline.getFactor());
|
||||
|
||||
ContainerInfo conatinerInfo = builder.build();
|
||||
Mockito.doReturn(conatinerInfo).when(mappingService)
|
||||
ContainerInfo containerInfo = builder.build();
|
||||
ContainerWithPipeline containerWithPipeline = new ContainerWithPipeline(
|
||||
containerInfo, pipeline);
|
||||
Mockito.doReturn(containerInfo).when(mappingService)
|
||||
.getContainer(containerID);
|
||||
Mockito.doReturn(containerWithPipeline).when(mappingService)
|
||||
.getContainerWithPipeline(containerID);
|
||||
}
|
||||
}
|
||||
|
@ -19,13 +19,11 @@
|
||||
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
|
||||
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
|
||||
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
||||
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
|
||||
import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||
import org.apache.hadoop.hdds.protocol.proto
|
||||
.StorageContainerDatanodeProtocolProtos.NodeReportProto;
|
||||
import org.apache.hadoop.hdds.protocol.proto
|
||||
.StorageContainerDatanodeProtocolProtos.StorageReportProto;
|
||||
import org.apache.hadoop.hdds.protocol.proto
|
||||
.StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
|
||||
import org.apache.hadoop.ozone.OzoneConsts;
|
||||
@ -33,7 +31,6 @@
|
||||
import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
|
||||
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
|
||||
import org.assertj.core.util.Preconditions;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
@ -130,11 +127,11 @@ public void setChillmode(boolean chillmode) {
|
||||
* Removes a data node from the management of this Node Manager.
|
||||
*
|
||||
* @param node - DataNode.
|
||||
* @throws UnregisteredNodeException
|
||||
* @throws NodeNotFoundException
|
||||
*/
|
||||
@Override
|
||||
public void removeNode(DatanodeDetails node)
|
||||
throws UnregisteredNodeException {
|
||||
throws NodeNotFoundException {
|
||||
|
||||
}
|
||||
|
||||
@ -272,16 +269,6 @@ public SCMNodeMetric getNodeStat(DatanodeDetails datanodeDetails) {
|
||||
return new SCMNodeMetric(nodeMetricMap.get(datanodeDetails.getUuid()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Used for testing.
|
||||
*
|
||||
* @return true if the HB check is done.
|
||||
*/
|
||||
@Override
|
||||
public boolean waitForHeartbeatProcessed() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the node state of a specific node.
|
||||
*
|
||||
@ -334,21 +321,6 @@ public void close() throws IOException {
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* When an object implementing interface <code>Runnable</code> is used to
|
||||
* create a thread, starting the thread causes the object's <code>run</code>
|
||||
* method to be called in that separately executing thread.
|
||||
* <p>
|
||||
* The general contract of the method <code>run</code> is that it may take any
|
||||
* action whatsoever.
|
||||
*
|
||||
* @see Thread#run()
|
||||
*/
|
||||
@Override
|
||||
public void run() {
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the version info from SCM.
|
||||
*
|
||||
@ -379,32 +351,10 @@ public RegisteredCommand register(DatanodeDetails datanodeDetails,
|
||||
* Send heartbeat to indicate the datanode is alive and doing well.
|
||||
*
|
||||
* @param datanodeDetails - Datanode ID.
|
||||
* @param nodeReport - node report.
|
||||
* @return SCMheartbeat response list
|
||||
*/
|
||||
@Override
|
||||
public List<SCMCommand> sendHeartbeat(DatanodeDetails datanodeDetails,
|
||||
NodeReportProto nodeReport) {
|
||||
if ((datanodeDetails != null) && (nodeReport != null) && (nodeReport
|
||||
.getStorageReportCount() > 0)) {
|
||||
SCMNodeStat stat = this.nodeMetricMap.get(datanodeDetails.getUuid());
|
||||
|
||||
long totalCapacity = 0L;
|
||||
long totalRemaining = 0L;
|
||||
long totalScmUsed = 0L;
|
||||
List<StorageReportProto> storageReports = nodeReport
|
||||
.getStorageReportList();
|
||||
for (StorageReportProto report : storageReports) {
|
||||
totalCapacity += report.getCapacity();
|
||||
totalRemaining += report.getRemaining();
|
||||
totalScmUsed += report.getScmUsed();
|
||||
}
|
||||
aggregateStat.subtract(stat);
|
||||
stat.set(totalCapacity, totalScmUsed, totalRemaining);
|
||||
aggregateStat.add(stat);
|
||||
nodeMetricMap.put(datanodeDetails.getUuid(), stat);
|
||||
|
||||
}
|
||||
public List<SCMCommand> processHeartbeat(DatanodeDetails datanodeDetails) {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -17,11 +17,13 @@
|
||||
|
||||
package org.apache.hadoop.hdds.scm.container;
|
||||
|
||||
import org.apache.commons.lang3.RandomUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers
|
||||
.ContainerWithPipeline;
|
||||
import org.apache.hadoop.hdds.server.events.EventQueue;
|
||||
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
|
||||
@ -33,12 +35,12 @@
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Random;
|
||||
|
||||
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent.CREATE;
|
||||
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent.CREATED;
|
||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT;
|
||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB;
|
||||
import static org.apache.hadoop.hdds.scm.events.SCMEvents.CLOSE_CONTAINER;
|
||||
|
||||
/**
|
||||
* Tests the closeContainerEventHandler class.
|
||||
@ -65,7 +67,7 @@ public static void setUp() throws Exception {
|
||||
nodeManager = new MockNodeManager(true, 10);
|
||||
mapping = new ContainerMapping(configuration, nodeManager, 128);
|
||||
eventQueue = new EventQueue();
|
||||
eventQueue.addHandler(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT,
|
||||
eventQueue.addHandler(CLOSE_CONTAINER,
|
||||
new CloseContainerEventHandler(mapping));
|
||||
}
|
||||
|
||||
@ -81,8 +83,8 @@ public static void tearDown() throws Exception {
|
||||
public void testIfCloseContainerEventHadnlerInvoked() {
|
||||
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
|
||||
.captureLogs(CloseContainerEventHandler.LOG);
|
||||
eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT,
|
||||
new ContainerID(Math.abs(new Random().nextLong())));
|
||||
eventQueue.fireEvent(CLOSE_CONTAINER,
|
||||
new ContainerID(Math.abs(RandomUtils.nextInt())));
|
||||
eventQueue.processAll(1000);
|
||||
Assert.assertTrue(logCapturer.getOutput()
|
||||
.contains("Close container Event triggered for container"));
|
||||
@ -90,14 +92,14 @@ public void testIfCloseContainerEventHadnlerInvoked() {
|
||||
|
||||
@Test
|
||||
public void testCloseContainerEventWithInvalidContainer() {
|
||||
long id = Math.abs(new Random().nextLong());
|
||||
long id = Math.abs(RandomUtils.nextInt());
|
||||
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
|
||||
.captureLogs(CloseContainerEventHandler.LOG);
|
||||
eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT,
|
||||
eventQueue.fireEvent(CLOSE_CONTAINER,
|
||||
new ContainerID(id));
|
||||
eventQueue.processAll(1000);
|
||||
Assert.assertTrue(logCapturer.getOutput()
|
||||
.contains("Container with id : " + id + " does not exist"));
|
||||
.contains("Failed to update the container state"));
|
||||
}
|
||||
|
||||
@Test
|
||||
@ -105,13 +107,14 @@ public void testCloseContainerEventWithValidContainers() throws IOException {
|
||||
|
||||
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
|
||||
.captureLogs(CloseContainerEventHandler.LOG);
|
||||
ContainerInfo info = mapping
|
||||
ContainerWithPipeline containerWithPipeline = mapping
|
||||
.allocateContainer(HddsProtos.ReplicationType.STAND_ALONE,
|
||||
HddsProtos.ReplicationFactor.ONE, "ozone");
|
||||
ContainerID id = new ContainerID(info.getContainerID());
|
||||
DatanodeDetails datanode = info.getPipeline().getLeader();
|
||||
ContainerID id = new ContainerID(
|
||||
containerWithPipeline.getContainerInfo().getContainerID());
|
||||
DatanodeDetails datanode = containerWithPipeline.getPipeline().getLeader();
|
||||
int closeCount = nodeManager.getCommandCount(datanode);
|
||||
eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT, id);
|
||||
eventQueue.fireEvent(CLOSE_CONTAINER, id);
|
||||
eventQueue.processAll(1000);
|
||||
// At this point of time, the allocated container is not in open
|
||||
// state, so firing close container event should not queue CLOSE
|
||||
@ -124,10 +127,12 @@ public void testCloseContainerEventWithValidContainers() throws IOException {
|
||||
//Execute these state transitions so that we can close the container.
|
||||
mapping.updateContainerState(id.getId(), CREATE);
|
||||
mapping.updateContainerState(id.getId(), CREATED);
|
||||
eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT,
|
||||
new ContainerID(info.getContainerID()));
|
||||
eventQueue.fireEvent(CLOSE_CONTAINER,
|
||||
new ContainerID(
|
||||
containerWithPipeline.getContainerInfo().getContainerID()));
|
||||
eventQueue.processAll(1000);
|
||||
Assert.assertEquals(closeCount + 1, nodeManager.getCommandCount(datanode));
|
||||
Assert.assertEquals(closeCount + 1,
|
||||
nodeManager.getCommandCount(datanode));
|
||||
Assert.assertEquals(HddsProtos.LifeCycleState.CLOSING,
|
||||
mapping.getStateManager().getContainer(id).getState());
|
||||
}
|
||||
@ -137,20 +142,23 @@ public void testCloseContainerEventWithRatis() throws IOException {
|
||||
|
||||
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
|
||||
.captureLogs(CloseContainerEventHandler.LOG);
|
||||
ContainerInfo info = mapping
|
||||
ContainerWithPipeline containerWithPipeline = mapping
|
||||
.allocateContainer(HddsProtos.ReplicationType.RATIS,
|
||||
HddsProtos.ReplicationFactor.THREE, "ozone");
|
||||
ContainerID id = new ContainerID(info.getContainerID());
|
||||
ContainerID id = new ContainerID(
|
||||
containerWithPipeline.getContainerInfo().getContainerID());
|
||||
int[] closeCount = new int[3];
|
||||
eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT, id);
|
||||
eventQueue.fireEvent(CLOSE_CONTAINER, id);
|
||||
eventQueue.processAll(1000);
|
||||
int i = 0;
|
||||
for (DatanodeDetails details : info.getPipeline().getMachines()) {
|
||||
for (DatanodeDetails details : containerWithPipeline.getPipeline()
|
||||
.getMachines()) {
|
||||
closeCount[i] = nodeManager.getCommandCount(details);
|
||||
i++;
|
||||
}
|
||||
i = 0;
|
||||
for (DatanodeDetails details : info.getPipeline().getMachines()) {
|
||||
for (DatanodeDetails details : containerWithPipeline.getPipeline()
|
||||
.getMachines()) {
|
||||
Assert.assertEquals(closeCount[i], nodeManager.getCommandCount(details));
|
||||
i++;
|
||||
}
|
||||
@ -161,12 +169,12 @@ public void testCloseContainerEventWithRatis() throws IOException {
|
||||
//Execute these state transitions so that we can close the container.
|
||||
mapping.updateContainerState(id.getId(), CREATE);
|
||||
mapping.updateContainerState(id.getId(), CREATED);
|
||||
eventQueue.fireEvent(CloseContainerEventHandler.CLOSE_CONTAINER_EVENT,
|
||||
new ContainerID(info.getContainerID()));
|
||||
eventQueue.fireEvent(CLOSE_CONTAINER, id);
|
||||
eventQueue.processAll(1000);
|
||||
i = 0;
|
||||
// Make sure close is queued for each datanode on the pipeline
|
||||
for (DatanodeDetails details : info.getPipeline().getMachines()) {
|
||||
for (DatanodeDetails details : containerWithPipeline.getPipeline()
|
||||
.getMachines()) {
|
||||
Assert.assertEquals(closeCount[i] + 1,
|
||||
nodeManager.getCommandCount(details));
|
||||
Assert.assertEquals(HddsProtos.LifeCycleState.CLOSING,
|
||||
|
@ -22,6 +22,7 @@
|
||||
import org.apache.hadoop.hdds.scm.TestUtils;
|
||||
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||
@ -103,7 +104,7 @@ public void clearChillMode() {
|
||||
|
||||
@Test
|
||||
public void testallocateContainer() throws Exception {
|
||||
ContainerInfo containerInfo = mapping.allocateContainer(
|
||||
ContainerWithPipeline containerInfo = mapping.allocateContainer(
|
||||
xceiverClientManager.getType(),
|
||||
xceiverClientManager.getFactor(),
|
||||
containerOwner);
|
||||
@ -120,7 +121,7 @@ public void testallocateContainerDistributesAllocation() throws Exception {
|
||||
*/
|
||||
Set<UUID> pipelineList = new TreeSet<>();
|
||||
for (int x = 0; x < 30; x++) {
|
||||
ContainerInfo containerInfo = mapping.allocateContainer(
|
||||
ContainerWithPipeline containerInfo = mapping.allocateContainer(
|
||||
xceiverClientManager.getType(),
|
||||
xceiverClientManager.getFactor(),
|
||||
containerOwner);
|
||||
@ -135,14 +136,13 @@ public void testallocateContainerDistributesAllocation() throws Exception {
|
||||
|
||||
@Test
|
||||
public void testGetContainer() throws IOException {
|
||||
ContainerInfo containerInfo = mapping.allocateContainer(
|
||||
ContainerWithPipeline containerInfo = mapping.allocateContainer(
|
||||
xceiverClientManager.getType(),
|
||||
xceiverClientManager.getFactor(),
|
||||
containerOwner);
|
||||
Pipeline pipeline = containerInfo.getPipeline();
|
||||
Assert.assertNotNull(pipeline);
|
||||
Pipeline newPipeline = mapping.getContainer(
|
||||
containerInfo.getContainerID()).getPipeline();
|
||||
Pipeline newPipeline = containerInfo.getPipeline();
|
||||
Assert.assertEquals(pipeline.getLeader().getUuid(),
|
||||
newPipeline.getLeader().getUuid());
|
||||
}
|
||||
@ -165,12 +165,12 @@ public void testChillModeAllocateContainerFails() throws IOException {
|
||||
public void testContainerCreationLeaseTimeout() throws IOException,
|
||||
InterruptedException {
|
||||
nodeManager.setChillmode(false);
|
||||
ContainerInfo containerInfo = mapping.allocateContainer(
|
||||
ContainerWithPipeline containerInfo = mapping.allocateContainer(
|
||||
xceiverClientManager.getType(),
|
||||
xceiverClientManager.getFactor(),
|
||||
containerOwner);
|
||||
mapping.updateContainerState(containerInfo.getContainerID(),
|
||||
HddsProtos.LifeCycleEvent.CREATE);
|
||||
mapping.updateContainerState(containerInfo.getContainerInfo()
|
||||
.getContainerID(), HddsProtos.LifeCycleEvent.CREATE);
|
||||
Thread.sleep(TIMEOUT + 1000);
|
||||
|
||||
NavigableSet<ContainerID> deleteContainers = mapping.getStateManager()
|
||||
@ -179,12 +179,14 @@ public void testContainerCreationLeaseTimeout() throws IOException,
|
||||
xceiverClientManager.getType(),
|
||||
xceiverClientManager.getFactor(),
|
||||
HddsProtos.LifeCycleState.DELETING);
|
||||
Assert.assertTrue(deleteContainers.contains(containerInfo.containerID()));
|
||||
Assert.assertTrue(deleteContainers
|
||||
.contains(containerInfo.getContainerInfo().containerID()));
|
||||
|
||||
thrown.expect(IOException.class);
|
||||
thrown.expectMessage("Lease Exception");
|
||||
mapping.updateContainerState(containerInfo.getContainerID(),
|
||||
HddsProtos.LifeCycleEvent.CREATED);
|
||||
mapping
|
||||
.updateContainerState(containerInfo.getContainerInfo().getContainerID(),
|
||||
HddsProtos.LifeCycleEvent.CREATED);
|
||||
}
|
||||
|
||||
@Test
|
||||
@ -294,10 +296,11 @@ public void testCloseContainer() throws IOException {
|
||||
private ContainerInfo createContainer()
|
||||
throws IOException {
|
||||
nodeManager.setChillmode(false);
|
||||
ContainerInfo containerInfo = mapping.allocateContainer(
|
||||
ContainerWithPipeline containerWithPipeline = mapping.allocateContainer(
|
||||
xceiverClientManager.getType(),
|
||||
xceiverClientManager.getFactor(),
|
||||
containerOwner);
|
||||
ContainerInfo containerInfo = containerWithPipeline.getContainerInfo();
|
||||
mapping.updateContainerState(containerInfo.getContainerID(),
|
||||
HddsProtos.LifeCycleEvent.CREATE);
|
||||
mapping.updateContainerState(containerInfo.getContainerID(),
|
||||
|
@ -24,6 +24,7 @@
|
||||
import org.apache.hadoop.hdds.scm.container.ContainerMapping;
|
||||
import org.apache.hadoop.hdds.scm.container.MockNodeManager;
|
||||
import org.apache.hadoop.hdds.scm.container.TestContainerMapping;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||
@ -91,9 +92,10 @@ public static void tearDown() throws Exception {
|
||||
|
||||
@Test
|
||||
public void testClose() throws IOException {
|
||||
ContainerInfo info = mapping.allocateContainer(
|
||||
ContainerWithPipeline containerWithPipeline = mapping.allocateContainer(
|
||||
HddsProtos.ReplicationType.STAND_ALONE,
|
||||
HddsProtos.ReplicationFactor.ONE, "ozone");
|
||||
ContainerInfo info = containerWithPipeline.getContainerInfo();
|
||||
|
||||
//Execute these state transitions so that we can close the container.
|
||||
mapping.updateContainerState(info.getContainerID(), CREATE);
|
||||
@ -101,7 +103,7 @@ public void testClose() throws IOException {
|
||||
long currentCount = mapping.getCloser().getCloseCount();
|
||||
long runCount = mapping.getCloser().getThreadRunCount();
|
||||
|
||||
DatanodeDetails datanode = info.getPipeline().getLeader();
|
||||
DatanodeDetails datanode = containerWithPipeline.getPipeline().getLeader();
|
||||
// Send a container report with used set to 1 GB. This should not close.
|
||||
sendContainerReport(info, 1 * GIGABYTE);
|
||||
|
||||
@ -138,9 +140,10 @@ public void testRepeatedClose() throws IOException,
|
||||
configuration.setTimeDuration(OZONE_CONTAINER_REPORT_INTERVAL, 1,
|
||||
TimeUnit.SECONDS);
|
||||
|
||||
ContainerInfo info = mapping.allocateContainer(
|
||||
ContainerWithPipeline containerWithPipeline = mapping.allocateContainer(
|
||||
HddsProtos.ReplicationType.STAND_ALONE,
|
||||
HddsProtos.ReplicationFactor.ONE, "ozone");
|
||||
ContainerInfo info = containerWithPipeline.getContainerInfo();
|
||||
|
||||
//Execute these state transitions so that we can close the container.
|
||||
mapping.updateContainerState(info.getContainerID(), CREATE);
|
||||
@ -148,10 +151,10 @@ public void testRepeatedClose() throws IOException,
|
||||
long currentCount = mapping.getCloser().getCloseCount();
|
||||
long runCount = mapping.getCloser().getThreadRunCount();
|
||||
|
||||
DatanodeDetails datanodeDetails = containerWithPipeline.getPipeline()
|
||||
.getLeader();
|
||||
|
||||
DatanodeDetails datanodeDetails = info.getPipeline().getLeader();
|
||||
|
||||
// Send this command twice and assert we have only one command in the queue.
|
||||
// Send this command twice and assert we have only one command in queue.
|
||||
sendContainerReport(info, 5 * GIGABYTE);
|
||||
sendContainerReport(info, 5 * GIGABYTE);
|
||||
|
||||
@ -183,9 +186,10 @@ public void testCleanupThreadRuns() throws IOException,
|
||||
long runCount = mapping.getCloser().getThreadRunCount();
|
||||
|
||||
for (int x = 0; x < ContainerCloser.getCleanupWaterMark() + 10; x++) {
|
||||
ContainerInfo info = mapping.allocateContainer(
|
||||
ContainerWithPipeline containerWithPipeline = mapping.allocateContainer(
|
||||
HddsProtos.ReplicationType.STAND_ALONE,
|
||||
HddsProtos.ReplicationFactor.ONE, "ozone");
|
||||
ContainerInfo info = containerWithPipeline.getContainerInfo();
|
||||
mapping.updateContainerState(info.getContainerID(), CREATE);
|
||||
mapping.updateContainerState(info.getContainerID(), CREATED);
|
||||
sendContainerReport(info, 5 * GIGABYTE);
|
||||
|
@ -25,7 +25,7 @@
|
||||
import org.apache.hadoop.hdds.scm.TestUtils;
|
||||
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
||||
import org.apache.hadoop.hdds.scm.container.ContainerMapping;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||
import org.apache.hadoop.hdds.scm.container.placement.algorithms
|
||||
.ContainerPlacementPolicy;
|
||||
import org.apache.hadoop.hdds.scm.container.placement.algorithms
|
||||
@ -36,8 +36,8 @@
|
||||
.StorageContainerDatanodeProtocolProtos.StorageReportProto;
|
||||
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||
import org.apache.hadoop.ozone.OzoneConsts;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.hadoop.test.PathUtils;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
@ -109,6 +109,7 @@ ContainerMapping createContainerManager(Configuration config,
|
||||
* @throws TimeoutException
|
||||
*/
|
||||
@Test
|
||||
@Ignore
|
||||
public void testContainerPlacementCapacity() throws IOException,
|
||||
InterruptedException, TimeoutException {
|
||||
OzoneConfiguration conf = getConf();
|
||||
@ -135,12 +136,11 @@ public void testContainerPlacementCapacity() throws IOException,
|
||||
String path = testDir.getAbsolutePath() + "/" + id;
|
||||
List<StorageReportProto> reports = TestUtils
|
||||
.createStorageReport(capacity, used, remaining, path, null, id, 1);
|
||||
nodeManager.sendHeartbeat(datanodeDetails,
|
||||
TestUtils.createNodeReport(reports));
|
||||
nodeManager.processHeartbeat(datanodeDetails);
|
||||
}
|
||||
|
||||
GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(),
|
||||
100, 4 * 1000);
|
||||
//TODO: wait for heartbeat to be processed
|
||||
Thread.sleep(4 * 1000);
|
||||
assertEquals(nodeCount, nodeManager.getNodeCount(HEALTHY));
|
||||
assertEquals(capacity * nodeCount,
|
||||
(long) nodeManager.getStats().getCapacity().get());
|
||||
@ -151,11 +151,11 @@ public void testContainerPlacementCapacity() throws IOException,
|
||||
|
||||
assertTrue(nodeManager.isOutOfChillMode());
|
||||
|
||||
ContainerInfo containerInfo = containerManager.allocateContainer(
|
||||
ContainerWithPipeline containerWithPipeline = containerManager.allocateContainer(
|
||||
xceiverClientManager.getType(),
|
||||
xceiverClientManager.getFactor(), "OZONE");
|
||||
assertEquals(xceiverClientManager.getFactor().getNumber(),
|
||||
containerInfo.getPipeline().getMachines().size());
|
||||
containerWithPipeline.getPipeline().getMachines().size());
|
||||
} finally {
|
||||
IOUtils.closeQuietly(containerManager);
|
||||
IOUtils.closeQuietly(nodeManager);
|
||||
|
@ -41,6 +41,7 @@
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
@ -61,14 +62,13 @@
|
||||
.OZONE_SCM_HEARTBEAT_INTERVAL;
|
||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
||||
.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL;
|
||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
||||
.OZONE_SCM_MAX_HB_COUNT_TO_PROCESS;
|
||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
|
||||
.OZONE_SCM_STALENODE_INTERVAL;
|
||||
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DEAD;
|
||||
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState
|
||||
.HEALTHY;
|
||||
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.STALE;
|
||||
import static org.apache.hadoop.hdds.scm.events.SCMEvents.DATANODE_COMMAND;
|
||||
import static org.hamcrest.CoreMatchers.containsString;
|
||||
import static org.hamcrest.core.StringStartsWith.startsWith;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
@ -148,14 +148,11 @@ public void testScmHeartbeat() throws IOException,
|
||||
for (int x = 0; x < nodeManager.getMinimumChillModeNodes(); x++) {
|
||||
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(
|
||||
nodeManager);
|
||||
nodeManager.sendHeartbeat(datanodeDetails,
|
||||
null);
|
||||
nodeManager.processHeartbeat(datanodeDetails);
|
||||
}
|
||||
|
||||
// Wait for 4 seconds max.
|
||||
GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(),
|
||||
100, 4 * 1000);
|
||||
|
||||
//TODO: wait for heartbeat to be processed
|
||||
Thread.sleep(4 * 1000);
|
||||
assertTrue("Heartbeat thread should have picked up the" +
|
||||
"scheduled heartbeats and transitioned out of chill mode.",
|
||||
nodeManager.isOutOfChillMode());
|
||||
@ -174,8 +171,8 @@ public void testScmNoHeartbeats() throws IOException,
|
||||
InterruptedException, TimeoutException {
|
||||
|
||||
try (SCMNodeManager nodeManager = createNodeManager(getConf())) {
|
||||
GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(),
|
||||
100, 4 * 1000);
|
||||
//TODO: wait for heartbeat to be processed
|
||||
Thread.sleep(4 * 1000);
|
||||
assertFalse("No heartbeats, Node manager should have been in" +
|
||||
" chill mode.", nodeManager.isOutOfChillMode());
|
||||
}
|
||||
@ -195,10 +192,9 @@ public void testScmNotEnoughHeartbeats() throws IOException,
|
||||
|
||||
// Need 100 nodes to come out of chill mode, only one node is sending HB.
|
||||
nodeManager.setMinimumChillModeNodes(100);
|
||||
nodeManager.sendHeartbeat(TestUtils.getDatanodeDetails(nodeManager),
|
||||
null);
|
||||
GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(),
|
||||
100, 4 * 1000);
|
||||
nodeManager.processHeartbeat(TestUtils.getDatanodeDetails(nodeManager));
|
||||
//TODO: wait for heartbeat to be processed
|
||||
Thread.sleep(4 * 1000);
|
||||
assertFalse("Not enough heartbeat, Node manager should have" +
|
||||
"been in chillmode.", nodeManager.isOutOfChillMode());
|
||||
}
|
||||
@ -223,12 +219,11 @@ public void testScmSameNodeHeartbeats() throws IOException,
|
||||
|
||||
// Send 10 heartbeat from same node, and assert we never leave chill mode.
|
||||
for (int x = 0; x < 10; x++) {
|
||||
nodeManager.sendHeartbeat(datanodeDetails,
|
||||
null);
|
||||
nodeManager.processHeartbeat(datanodeDetails);
|
||||
}
|
||||
|
||||
GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(),
|
||||
100, 4 * 1000);
|
||||
//TODO: wait for heartbeat to be processed
|
||||
Thread.sleep(4 * 1000);
|
||||
assertFalse("Not enough nodes have send heartbeat to node" +
|
||||
"manager.", nodeManager.isOutOfChillMode());
|
||||
}
|
||||
@ -254,14 +249,12 @@ public void testScmShutdown() throws IOException, InterruptedException,
|
||||
nodeManager.close();
|
||||
|
||||
// These should never be processed.
|
||||
nodeManager.sendHeartbeat(datanodeDetails,
|
||||
null);
|
||||
nodeManager.processHeartbeat(datanodeDetails);
|
||||
|
||||
// Let us just wait for 2 seconds to prove that HBs are not processed.
|
||||
Thread.sleep(2 * 1000);
|
||||
|
||||
assertEquals("Assert new HBs were never processed", 0,
|
||||
nodeManager.getLastHBProcessedCount());
|
||||
//TODO: add assertion
|
||||
}
|
||||
|
||||
/**
|
||||
@ -283,8 +276,7 @@ public void testScmHeartbeatAfterRestart() throws Exception {
|
||||
try (SCMNodeManager nodemanager = createNodeManager(conf)) {
|
||||
nodemanager.register(datanodeDetails,
|
||||
TestUtils.createNodeReport(reports));
|
||||
List<SCMCommand> command = nodemanager.sendHeartbeat(
|
||||
datanodeDetails, null);
|
||||
List<SCMCommand> command = nodemanager.processHeartbeat(datanodeDetails);
|
||||
Assert.assertTrue(nodemanager.getAllNodes().contains(datanodeDetails));
|
||||
Assert.assertTrue("On regular HB calls, SCM responses a "
|
||||
+ "datanode with an empty command list", command.isEmpty());
|
||||
@ -302,8 +294,7 @@ public void testScmHeartbeatAfterRestart() throws Exception {
|
||||
GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
||||
@Override public Boolean get() {
|
||||
List<SCMCommand> command =
|
||||
nodemanager.sendHeartbeat(datanodeDetails,
|
||||
null);
|
||||
nodemanager.processHeartbeat(datanodeDetails);
|
||||
return command.size() == 1 && command.get(0).getType()
|
||||
.equals(SCMCommandProto.Type.reregisterCommand);
|
||||
}
|
||||
@ -334,11 +325,10 @@ public void testScmHealthyNodeCount() throws IOException,
|
||||
for (int x = 0; x < count; x++) {
|
||||
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(
|
||||
nodeManager);
|
||||
nodeManager.sendHeartbeat(datanodeDetails,
|
||||
null);
|
||||
nodeManager.processHeartbeat(datanodeDetails);
|
||||
}
|
||||
GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(),
|
||||
100, 4 * 1000);
|
||||
//TODO: wait for heartbeat to be processed
|
||||
Thread.sleep(4 * 1000);
|
||||
assertEquals(count, nodeManager.getNodeCount(HEALTHY));
|
||||
}
|
||||
}
|
||||
@ -426,19 +416,18 @@ public void testScmDetectStaleAndDeadNode() throws IOException,
|
||||
DatanodeDetails staleNode = TestUtils.getDatanodeDetails(nodeManager);
|
||||
|
||||
// Heartbeat once
|
||||
nodeManager.sendHeartbeat(staleNode,
|
||||
null);
|
||||
nodeManager.processHeartbeat(staleNode);
|
||||
|
||||
// Heartbeat all other nodes.
|
||||
for (DatanodeDetails dn : nodeList) {
|
||||
nodeManager.sendHeartbeat(dn, null);
|
||||
nodeManager.processHeartbeat(dn);
|
||||
}
|
||||
|
||||
// Wait for 2 seconds .. and heartbeat good nodes again.
|
||||
Thread.sleep(2 * 1000);
|
||||
|
||||
for (DatanodeDetails dn : nodeList) {
|
||||
nodeManager.sendHeartbeat(dn, null);
|
||||
nodeManager.processHeartbeat(dn);
|
||||
}
|
||||
|
||||
// Wait for 2 seconds, wait a total of 4 seconds to make sure that the
|
||||
@ -455,7 +444,7 @@ public void testScmDetectStaleAndDeadNode() throws IOException,
|
||||
|
||||
// heartbeat good nodes again.
|
||||
for (DatanodeDetails dn : nodeList) {
|
||||
nodeManager.sendHeartbeat(dn, null);
|
||||
nodeManager.processHeartbeat(dn);
|
||||
}
|
||||
|
||||
// 6 seconds is the dead window for this test , so we wait a total of
|
||||
@ -491,7 +480,7 @@ public void testScmDetectStaleAndDeadNode() throws IOException,
|
||||
public void testScmCheckForErrorOnNullDatanodeDetails() throws IOException,
|
||||
InterruptedException, TimeoutException {
|
||||
try (SCMNodeManager nodeManager = createNodeManager(getConf())) {
|
||||
nodeManager.sendHeartbeat(null, null);
|
||||
nodeManager.processHeartbeat(null);
|
||||
} catch (NullPointerException npe) {
|
||||
GenericTestUtils.assertExceptionContains("Heartbeat is missing " +
|
||||
"DatanodeDetails.", npe);
|
||||
@ -568,12 +557,9 @@ public void testScmClusterIsInExpectedState1() throws IOException,
|
||||
TestUtils.getDatanodeDetails(nodeManager);
|
||||
DatanodeDetails deadNode =
|
||||
TestUtils.getDatanodeDetails(nodeManager);
|
||||
nodeManager.sendHeartbeat(
|
||||
healthyNode, null);
|
||||
nodeManager.sendHeartbeat(
|
||||
staleNode, null);
|
||||
nodeManager.sendHeartbeat(
|
||||
deadNode, null);
|
||||
nodeManager.processHeartbeat(healthyNode);
|
||||
nodeManager.processHeartbeat(staleNode);
|
||||
nodeManager.processHeartbeat(deadNode);
|
||||
|
||||
// Sleep so that heartbeat processing thread gets to run.
|
||||
Thread.sleep(500);
|
||||
@ -599,16 +585,12 @@ public void testScmClusterIsInExpectedState1() throws IOException,
|
||||
* the 3 second windows.
|
||||
*/
|
||||
|
||||
nodeManager.sendHeartbeat(
|
||||
healthyNode, null);
|
||||
nodeManager.sendHeartbeat(
|
||||
staleNode, null);
|
||||
nodeManager.sendHeartbeat(
|
||||
deadNode, null);
|
||||
nodeManager.processHeartbeat(healthyNode);
|
||||
nodeManager.processHeartbeat(staleNode);
|
||||
nodeManager.processHeartbeat(deadNode);
|
||||
|
||||
Thread.sleep(1500);
|
||||
nodeManager.sendHeartbeat(
|
||||
healthyNode, null);
|
||||
nodeManager.processHeartbeat(healthyNode);
|
||||
Thread.sleep(2 * 1000);
|
||||
assertEquals(1, nodeManager.getNodeCount(HEALTHY));
|
||||
|
||||
@ -628,13 +610,10 @@ public void testScmClusterIsInExpectedState1() throws IOException,
|
||||
* staleNode to move to stale state and deadNode to move to dead state.
|
||||
*/
|
||||
|
||||
nodeManager.sendHeartbeat(
|
||||
healthyNode, null);
|
||||
nodeManager.sendHeartbeat(
|
||||
staleNode, null);
|
||||
nodeManager.processHeartbeat(healthyNode);
|
||||
nodeManager.processHeartbeat(staleNode);
|
||||
Thread.sleep(1500);
|
||||
nodeManager.sendHeartbeat(
|
||||
healthyNode, null);
|
||||
nodeManager.processHeartbeat(healthyNode);
|
||||
Thread.sleep(2 * 1000);
|
||||
|
||||
// 3.5 seconds have elapsed for stale node, so it moves into Stale.
|
||||
@ -667,12 +646,9 @@ public void testScmClusterIsInExpectedState1() throws IOException,
|
||||
* Cluster State : let us heartbeat all the nodes and verify that we get
|
||||
* back all the nodes in healthy state.
|
||||
*/
|
||||
nodeManager.sendHeartbeat(
|
||||
healthyNode, null);
|
||||
nodeManager.sendHeartbeat(
|
||||
staleNode, null);
|
||||
nodeManager.sendHeartbeat(
|
||||
deadNode, null);
|
||||
nodeManager.processHeartbeat(healthyNode);
|
||||
nodeManager.processHeartbeat(staleNode);
|
||||
nodeManager.processHeartbeat(deadNode);
|
||||
Thread.sleep(500);
|
||||
//Assert all nodes are healthy.
|
||||
assertEquals(3, nodeManager.getAllNodes().size());
|
||||
@ -693,7 +669,7 @@ private void heartbeatNodeSet(SCMNodeManager manager,
|
||||
int sleepDuration) throws InterruptedException {
|
||||
while (!Thread.currentThread().isInterrupted()) {
|
||||
for (DatanodeDetails dn : list) {
|
||||
manager.sendHeartbeat(dn, null);
|
||||
manager.processHeartbeat(dn);
|
||||
}
|
||||
Thread.sleep(sleepDuration);
|
||||
}
|
||||
@ -747,7 +723,6 @@ public void testScmClusterIsInExpectedState2() throws IOException,
|
||||
conf.setTimeDuration(OZONE_SCM_HEARTBEAT_INTERVAL, 1, SECONDS);
|
||||
conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, SECONDS);
|
||||
conf.setTimeDuration(OZONE_SCM_DEADNODE_INTERVAL, 6, SECONDS);
|
||||
conf.setInt(OZONE_SCM_MAX_HB_COUNT_TO_PROCESS, 7000);
|
||||
|
||||
|
||||
try (SCMNodeManager nodeManager = createNodeManager(conf)) {
|
||||
@ -779,7 +754,7 @@ public void testScmClusterIsInExpectedState2() throws IOException,
|
||||
// No Thread just one time HBs the node manager, so that these will be
|
||||
// marked as dead nodes eventually.
|
||||
for (DatanodeDetails dn : deadNodeList) {
|
||||
nodeManager.sendHeartbeat(dn, null);
|
||||
nodeManager.processHeartbeat(dn);
|
||||
}
|
||||
|
||||
|
||||
@ -883,54 +858,6 @@ public void testScmCanHandleScale() throws IOException,
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Asserts that SCM backs off from HB processing instead of going into an
|
||||
* infinite loop if SCM is flooded with too many heartbeats. This many not be
|
||||
* the best thing to do, but SCM tries to protect itself and logs an error
|
||||
* saying that it is getting flooded with heartbeats. In real world this can
|
||||
* lead to many nodes becoming stale or dead due to the fact that SCM is not
|
||||
* able to keep up with heartbeat processing. This test just verifies that SCM
|
||||
* will log that information.
|
||||
* @throws TimeoutException
|
||||
*/
|
||||
@Test
|
||||
public void testScmLogsHeartbeatFlooding() throws IOException,
|
||||
InterruptedException, TimeoutException {
|
||||
final int healthyCount = 3000;
|
||||
|
||||
// Make the HB process thread run slower.
|
||||
OzoneConfiguration conf = getConf();
|
||||
conf.setTimeDuration(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL, 500,
|
||||
TimeUnit.MILLISECONDS);
|
||||
conf.setTimeDuration(OZONE_SCM_HEARTBEAT_INTERVAL, 1, SECONDS);
|
||||
conf.setInt(OZONE_SCM_MAX_HB_COUNT_TO_PROCESS, 500);
|
||||
|
||||
try (SCMNodeManager nodeManager = createNodeManager(conf)) {
|
||||
List<DatanodeDetails> healthyList = createNodeSet(nodeManager,
|
||||
healthyCount);
|
||||
GenericTestUtils.LogCapturer logCapturer =
|
||||
GenericTestUtils.LogCapturer.captureLogs(SCMNodeManager.LOG);
|
||||
Runnable healthyNodeTask = () -> {
|
||||
try {
|
||||
// No wait in the HB sending loop.
|
||||
heartbeatNodeSet(nodeManager, healthyList, 0);
|
||||
} catch (InterruptedException ignored) {
|
||||
}
|
||||
};
|
||||
Thread thread1 = new Thread(healthyNodeTask);
|
||||
thread1.setDaemon(true);
|
||||
thread1.start();
|
||||
|
||||
GenericTestUtils.waitFor(() -> logCapturer.getOutput()
|
||||
.contains("SCM is being "
|
||||
+ "flooded by heartbeats. Not able to keep up"
|
||||
+ " with the heartbeat counts."),
|
||||
500, 20 * 1000);
|
||||
|
||||
thread1.interrupt();
|
||||
logCapturer.stopCapturing();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testScmEnterAndExitChillMode() throws IOException,
|
||||
@ -943,8 +870,7 @@ public void testScmEnterAndExitChillMode() throws IOException,
|
||||
nodeManager.setMinimumChillModeNodes(10);
|
||||
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(
|
||||
nodeManager);
|
||||
nodeManager.sendHeartbeat(
|
||||
datanodeDetails, null);
|
||||
nodeManager.processHeartbeat(datanodeDetails);
|
||||
String status = nodeManager.getChillModeStatus();
|
||||
Assert.assertThat(status, containsString("Still in chill " +
|
||||
"mode, waiting on nodes to report in."));
|
||||
@ -971,7 +897,7 @@ public void testScmEnterAndExitChillMode() throws IOException,
|
||||
// Assert that node manager force enter cannot be overridden by nodes HBs.
|
||||
for (int x = 0; x < 20; x++) {
|
||||
DatanodeDetails datanode = TestUtils.getDatanodeDetails(nodeManager);
|
||||
nodeManager.sendHeartbeat(datanode, null);
|
||||
nodeManager.processHeartbeat(datanode);
|
||||
}
|
||||
|
||||
Thread.sleep(500);
|
||||
@ -995,6 +921,8 @@ public void testScmEnterAndExitChillMode() throws IOException,
|
||||
* @throws TimeoutException
|
||||
*/
|
||||
@Test
|
||||
@Ignore
|
||||
// TODO: Enable this after we implement NodeReportEvent handler.
|
||||
public void testScmStatsFromNodeReport() throws IOException,
|
||||
InterruptedException, TimeoutException {
|
||||
OzoneConfiguration conf = getConf();
|
||||
@ -1015,11 +943,10 @@ public void testScmStatsFromNodeReport() throws IOException,
|
||||
List<StorageReportProto> reports = TestUtils
|
||||
.createStorageReport(capacity, used, free, storagePath,
|
||||
null, dnId, 1);
|
||||
nodeManager.sendHeartbeat(datanodeDetails,
|
||||
TestUtils.createNodeReport(reports));
|
||||
nodeManager.processHeartbeat(datanodeDetails);
|
||||
}
|
||||
GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(),
|
||||
100, 4 * 1000);
|
||||
//TODO: wait for heartbeat to be processed
|
||||
Thread.sleep(4 * 1000);
|
||||
assertEquals(nodeCount, nodeManager.getNodeCount(HEALTHY));
|
||||
assertEquals(capacity * nodeCount, (long) nodeManager.getStats()
|
||||
.getCapacity().get());
|
||||
@ -1038,6 +965,8 @@ public void testScmStatsFromNodeReport() throws IOException,
|
||||
* @throws TimeoutException
|
||||
*/
|
||||
@Test
|
||||
@Ignore
|
||||
// TODO: Enable this after we implement NodeReportEvent handler.
|
||||
public void testScmNodeReportUpdate() throws IOException,
|
||||
InterruptedException, TimeoutException {
|
||||
OzoneConfiguration conf = getConf();
|
||||
@ -1065,8 +994,7 @@ public void testScmNodeReportUpdate() throws IOException,
|
||||
.createStorageReport(capacity, scmUsed, remaining, storagePath,
|
||||
null, dnId, 1);
|
||||
|
||||
nodeManager.sendHeartbeat(datanodeDetails,
|
||||
TestUtils.createNodeReport(reports));
|
||||
nodeManager.processHeartbeat(datanodeDetails);
|
||||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
@ -1141,13 +1069,7 @@ public void testScmNodeReportUpdate() throws IOException,
|
||||
foundRemaining = nodeManager.getStats().getRemaining().get();
|
||||
assertEquals(0, foundRemaining);
|
||||
|
||||
// Send a new report to bring the dead node back to healthy
|
||||
String storagePath = testDir.getAbsolutePath() + "/" + dnId;
|
||||
List<StorageReportProto> reports = TestUtils
|
||||
.createStorageReport(capacity, expectedScmUsed, expectedRemaining,
|
||||
storagePath, null, dnId, 1);
|
||||
nodeManager.sendHeartbeat(datanodeDetails,
|
||||
TestUtils.createNodeReport(reports));
|
||||
nodeManager.processHeartbeat(datanodeDetails);
|
||||
|
||||
// Wait up to 5 seconds so that the dead node becomes healthy
|
||||
// Verify usage info should be updated.
|
||||
@ -1185,17 +1107,17 @@ public void testHandlingSCMCommandEvent() {
|
||||
|
||||
EventQueue eq = new EventQueue();
|
||||
try (SCMNodeManager nodemanager = createNodeManager(conf)) {
|
||||
eq.addHandler(SCMNodeManager.DATANODE_COMMAND, nodemanager);
|
||||
eq.addHandler(DATANODE_COMMAND, nodemanager);
|
||||
|
||||
nodemanager
|
||||
.register(datanodeDetails, TestUtils.createNodeReport(reports));
|
||||
eq.fireEvent(SCMNodeManager.DATANODE_COMMAND,
|
||||
eq.fireEvent(DATANODE_COMMAND,
|
||||
new CommandForDatanode(datanodeDetails.getUuid(),
|
||||
new CloseContainerCommand(1L, ReplicationType.STAND_ALONE)));
|
||||
|
||||
eq.processAll(1000L);
|
||||
List<SCMCommand> command =
|
||||
nodemanager.sendHeartbeat(datanodeDetails, null);
|
||||
nodemanager.processHeartbeat(datanodeDetails);
|
||||
Assert.assertEquals(1, command.size());
|
||||
Assert
|
||||
.assertEquals(command.get(0).getClass(), CloseContainerCommand.class);
|
||||
|
@ -20,8 +20,6 @@
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.proto
|
||||
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
|
||||
@ -40,6 +38,9 @@
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.apache.hadoop.hdds.scm.events.SCMEvents.CONTAINER_REPORT;
|
||||
import static org.apache.hadoop.hdds.scm.events.SCMEvents.NODE_REPORT;
|
||||
|
||||
/**
|
||||
* This class tests the behavior of SCMDatanodeHeartbeatDispatcher.
|
||||
*/
|
||||
@ -49,8 +50,6 @@ public class TestSCMDatanodeHeartbeatDispatcher {
|
||||
@Test
|
||||
public void testNodeReportDispatcher() throws IOException {
|
||||
|
||||
Configuration conf = new OzoneConfiguration();
|
||||
|
||||
AtomicInteger eventReceived = new AtomicInteger();
|
||||
|
||||
NodeReportProto nodeReport = NodeReportProto.getDefaultInstance();
|
||||
@ -60,10 +59,10 @@ public void testNodeReportDispatcher() throws IOException {
|
||||
@Override
|
||||
public <PAYLOAD, EVENT_TYPE extends Event<PAYLOAD>> void fireEvent(
|
||||
EVENT_TYPE event, PAYLOAD payload) {
|
||||
Assert.assertEquals(event,
|
||||
SCMDatanodeHeartbeatDispatcher.NODE_REPORT);
|
||||
Assert.assertEquals(event, NODE_REPORT);
|
||||
eventReceived.incrementAndGet();
|
||||
Assert.assertEquals(nodeReport, ((NodeReportFromDatanode)payload).getReport());
|
||||
Assert.assertEquals(nodeReport,
|
||||
((NodeReportFromDatanode)payload).getReport());
|
||||
|
||||
}
|
||||
});
|
||||
@ -84,7 +83,6 @@ public <PAYLOAD, EVENT_TYPE extends Event<PAYLOAD>> void fireEvent(
|
||||
@Test
|
||||
public void testContainerReportDispatcher() throws IOException {
|
||||
|
||||
Configuration conf = new OzoneConfiguration();
|
||||
|
||||
AtomicInteger eventReceived = new AtomicInteger();
|
||||
|
||||
@ -96,9 +94,9 @@ public void testContainerReportDispatcher() throws IOException {
|
||||
@Override
|
||||
public <PAYLOAD, EVENT_TYPE extends Event<PAYLOAD>> void fireEvent(
|
||||
EVENT_TYPE event, PAYLOAD payload) {
|
||||
Assert.assertEquals(event,
|
||||
SCMDatanodeHeartbeatDispatcher.CONTAINER_REPORT);
|
||||
Assert.assertEquals(containerReport, ((ContainerReportFromDatanode)payload).getReport());
|
||||
Assert.assertEquals(event, CONTAINER_REPORT);
|
||||
Assert.assertEquals(containerReport,
|
||||
((ContainerReportFromDatanode)payload).getReport());
|
||||
eventReceived.incrementAndGet();
|
||||
}
|
||||
});
|
||||
|
@ -21,7 +21,7 @@
|
||||
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
|
||||
import org.apache.hadoop.hdds.scm.node.CommandQueue;
|
||||
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
||||
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
|
||||
import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
|
||||
import org.apache.hadoop.hdds.protocol.proto
|
||||
@ -31,7 +31,6 @@
|
||||
import org.apache.hadoop.ozone.protocol.VersionResponse;
|
||||
import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
|
||||
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
@ -90,11 +89,11 @@ public Map<String, Integer> getNodeCount() {
|
||||
* Removes a data node from the management of this Node Manager.
|
||||
*
|
||||
* @param node - DataNode.
|
||||
* @throws UnregisteredNodeException
|
||||
* @throws NodeNotFoundException
|
||||
*/
|
||||
@Override
|
||||
public void removeNode(DatanodeDetails node)
|
||||
throws UnregisteredNodeException {
|
||||
throws NodeNotFoundException {
|
||||
nodeStateMap.remove(node);
|
||||
|
||||
}
|
||||
@ -201,16 +200,6 @@ public SCMNodeMetric getNodeStat(DatanodeDetails dd) {
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Wait for the heartbeat is processed by NodeManager.
|
||||
*
|
||||
* @return true if heartbeat has been processed.
|
||||
*/
|
||||
@Override
|
||||
public boolean waitForHeartbeatProcessed() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the node state of a specific node.
|
||||
*
|
||||
@ -240,22 +229,6 @@ public void close() throws IOException {
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* When an object implementing interface <code>Runnable</code> is used
|
||||
* to create a thread, starting the thread causes the object's
|
||||
* <code>run</code> method to be called in that separately executing
|
||||
* thread.
|
||||
* <p>
|
||||
* The general contract of the method <code>run</code> is that it may
|
||||
* take any action whatsoever.
|
||||
*
|
||||
* @see Thread#run()
|
||||
*/
|
||||
@Override
|
||||
public void run() {
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the version info from SCM.
|
||||
*
|
||||
@ -285,12 +258,10 @@ public RegisteredCommand register(DatanodeDetails dd,
|
||||
* Send heartbeat to indicate the datanode is alive and doing well.
|
||||
*
|
||||
* @param dd - Datanode Details.
|
||||
* @param nodeReport - node report.
|
||||
* @return SCMheartbeat response list
|
||||
*/
|
||||
@Override
|
||||
public List<SCMCommand> sendHeartbeat(DatanodeDetails dd,
|
||||
NodeReportProto nodeReport) {
|
||||
public List<SCMCommand> processHeartbeat(DatanodeDetails dd) {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -27,7 +27,7 @@
|
||||
import java.net.URISyntaxException;
|
||||
|
||||
/**
|
||||
* This class is the base CLI for scm, ksm and scmadm.
|
||||
* This class is the base CLI for scm, om and scmadm.
|
||||
*/
|
||||
public abstract class OzoneBaseCLI extends Configured implements Tool {
|
||||
|
||||
|
@ -24,9 +24,9 @@
|
||||
import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler;
|
||||
import org.apache.hadoop.hdds.scm.cli.SCMCLI;
|
||||
import org.apache.hadoop.hdds.scm.client.ScmClient;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||
|
||||
/**
|
||||
* The handler of close container command.
|
||||
@ -51,15 +51,15 @@ public void execute(CommandLine cmd) throws IOException {
|
||||
}
|
||||
String containerID = cmd.getOptionValue(OPT_CONTAINER_ID);
|
||||
|
||||
ContainerInfo container = getScmClient().
|
||||
getContainer(Long.parseLong(containerID));
|
||||
ContainerWithPipeline container = getScmClient().
|
||||
getContainerWithPipeline(Long.parseLong(containerID));
|
||||
if (container == null) {
|
||||
throw new IOException("Cannot close an non-exist container "
|
||||
+ containerID);
|
||||
}
|
||||
logOut("Closing container : %s.", containerID);
|
||||
getScmClient().closeContainer(container.getContainerID(),
|
||||
container.getPipeline());
|
||||
getScmClient()
|
||||
.closeContainer(container.getContainerInfo().getContainerID());
|
||||
logOut("Container closed.");
|
||||
}
|
||||
|
||||
|
@ -25,9 +25,9 @@
|
||||
import org.apache.commons.cli.Options;
|
||||
import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler;
|
||||
import org.apache.hadoop.hdds.scm.client.ScmClient;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||
|
||||
import static org.apache.hadoop.hdds.scm.cli.SCMCLI.CMD_WIDTH;
|
||||
import static org.apache.hadoop.hdds.scm.cli.SCMCLI.HELP_OP;
|
||||
@ -60,7 +60,7 @@ public void execute(CommandLine cmd) throws IOException {
|
||||
|
||||
String containerID = cmd.getOptionValue(OPT_CONTAINER_ID);
|
||||
|
||||
ContainerInfo container = getScmClient().getContainer(
|
||||
ContainerWithPipeline container = getScmClient().getContainerWithPipeline(
|
||||
Long.parseLong(containerID));
|
||||
if (container == null) {
|
||||
throw new IOException("Cannot delete an non-exist container "
|
||||
@ -68,8 +68,9 @@ public void execute(CommandLine cmd) throws IOException {
|
||||
}
|
||||
|
||||
logOut("Deleting container : %s.", containerID);
|
||||
getScmClient().deleteContainer(container.getContainerID(),
|
||||
container.getPipeline(), cmd.hasOption(OPT_FORCE));
|
||||
getScmClient()
|
||||
.deleteContainer(container.getContainerInfo().getContainerID(),
|
||||
container.getPipeline(), cmd.hasOption(OPT_FORCE));
|
||||
logOut("Container %s deleted.", containerID);
|
||||
}
|
||||
|
||||
|
@ -24,7 +24,6 @@
|
||||
import org.apache.commons.cli.Options;
|
||||
import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler;
|
||||
import org.apache.hadoop.hdds.scm.client.ScmClient;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||
.ContainerData;
|
||||
@ -33,6 +32,7 @@
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
||||
|
||||
import static org.apache.hadoop.hdds.scm.cli.SCMCLI.CMD_WIDTH;
|
||||
import static org.apache.hadoop.hdds.scm.cli.SCMCLI.HELP_OP;
|
||||
@ -68,13 +68,12 @@ public void execute(CommandLine cmd) throws IOException {
|
||||
}
|
||||
}
|
||||
String containerID = cmd.getOptionValue(OPT_CONTAINER_ID);
|
||||
ContainerInfo container = getScmClient().
|
||||
getContainer(Long.parseLong(containerID));
|
||||
ContainerWithPipeline container = getScmClient().
|
||||
getContainerWithPipeline(Long.parseLong(containerID));
|
||||
Preconditions.checkNotNull(container, "Container cannot be null");
|
||||
|
||||
ContainerData containerData =
|
||||
getScmClient().readContainer(container.getContainerID(),
|
||||
container.getPipeline());
|
||||
ContainerData containerData = getScmClient().readContainer(container
|
||||
.getContainerInfo().getContainerID(), container.getPipeline());
|
||||
|
||||
// Print container report info.
|
||||
logOut("Container id: %s", containerID);
|
||||
|
@ -3070,25 +3070,6 @@ TraceScope newSrcDstTraceScope(String description, String src, String dst) {
|
||||
return scope;
|
||||
}
|
||||
|
||||
/**
|
||||
* Full detailed tracing for read requests: path, position in the file,
|
||||
* and length.
|
||||
*
|
||||
* @param reqLen requested length
|
||||
*/
|
||||
TraceScope newReaderTraceScope(String description, String path, long pos,
|
||||
int reqLen) {
|
||||
TraceScope scope = newPathTraceScope(description, path);
|
||||
scope.addKVAnnotation("pos", Long.toString(pos));
|
||||
scope.addKVAnnotation("reqLen", Integer.toString(reqLen));
|
||||
return scope;
|
||||
}
|
||||
|
||||
/** Add the returned length info to the scope. */
|
||||
void addRetLenToReaderScope(TraceScope scope, int retLen) {
|
||||
scope.addKVAnnotation("retLen", Integer.toString(retLen));
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the erasure coding policy information for the specified path
|
||||
*
|
||||
|
@ -85,8 +85,6 @@
|
||||
import org.apache.hadoop.util.StopWatch;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.htrace.core.SpanId;
|
||||
import org.apache.htrace.core.TraceScope;
|
||||
import org.apache.htrace.core.Tracer;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
@ -641,7 +639,6 @@ protected BlockReader getBlockReader(LocatedBlock targetBlock,
|
||||
setClientCacheContext(dfsClient.getClientContext()).
|
||||
setUserGroupInformation(dfsClient.ugi).
|
||||
setConfiguration(dfsClient.getConfiguration()).
|
||||
setTracer(dfsClient.getTracer()).
|
||||
build();
|
||||
}
|
||||
|
||||
@ -821,31 +818,14 @@ public synchronized int read(@Nonnull final byte buf[], int off, int len)
|
||||
}
|
||||
ReaderStrategy byteArrayReader =
|
||||
new ByteArrayStrategy(buf, off, len, readStatistics, dfsClient);
|
||||
try (TraceScope scope =
|
||||
dfsClient.newReaderTraceScope("DFSInputStream#byteArrayRead",
|
||||
src, getPos(), len)) {
|
||||
int retLen = readWithStrategy(byteArrayReader);
|
||||
if (retLen < len) {
|
||||
dfsClient.addRetLenToReaderScope(scope, retLen);
|
||||
}
|
||||
return retLen;
|
||||
}
|
||||
return readWithStrategy(byteArrayReader);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized int read(final ByteBuffer buf) throws IOException {
|
||||
ReaderStrategy byteBufferReader =
|
||||
new ByteBufferStrategy(buf, readStatistics, dfsClient);
|
||||
int reqLen = buf.remaining();
|
||||
try (TraceScope scope =
|
||||
dfsClient.newReaderTraceScope("DFSInputStream#byteBufferRead",
|
||||
src, getPos(), reqLen)){
|
||||
int retLen = readWithStrategy(byteBufferReader);
|
||||
if (retLen < reqLen) {
|
||||
dfsClient.addRetLenToReaderScope(scope, retLen);
|
||||
}
|
||||
return retLen;
|
||||
}
|
||||
return readWithStrategy(byteBufferReader);
|
||||
}
|
||||
|
||||
private DNAddrPair chooseDataNode(LocatedBlock block,
|
||||
@ -1026,16 +1006,12 @@ private Callable<ByteBuffer> getFromOneDataNode(final DNAddrPair datanode,
|
||||
final ByteBuffer bb,
|
||||
final CorruptedBlocks corruptedBlocks,
|
||||
final int hedgedReadId) {
|
||||
final SpanId parentSpanId = Tracer.getCurrentSpanId();
|
||||
return new Callable<ByteBuffer>() {
|
||||
@Override
|
||||
public ByteBuffer call() throws Exception {
|
||||
DFSClientFaultInjector.get().sleepBeforeHedgedGet();
|
||||
try (TraceScope ignored = dfsClient.getTracer().
|
||||
newScope("hedgedRead" + hedgedReadId, parentSpanId)) {
|
||||
actualGetFromOneDataNode(datanode, start, end, bb, corruptedBlocks);
|
||||
return bb;
|
||||
}
|
||||
actualGetFromOneDataNode(datanode, start, end, bb, corruptedBlocks);
|
||||
return bb;
|
||||
}
|
||||
};
|
||||
}
|
||||
@ -1336,16 +1312,8 @@ public int read(long position, byte[] buffer, int offset, int length)
|
||||
if (length == 0) {
|
||||
return 0;
|
||||
}
|
||||
try (TraceScope scope = dfsClient.
|
||||
newReaderTraceScope("DFSInputStream#byteArrayPread",
|
||||
src, position, length)) {
|
||||
ByteBuffer bb = ByteBuffer.wrap(buffer, offset, length);
|
||||
int retLen = pread(position, bb);
|
||||
if (retLen < length) {
|
||||
dfsClient.addRetLenToReaderScope(scope, retLen);
|
||||
}
|
||||
return retLen;
|
||||
}
|
||||
ByteBuffer bb = ByteBuffer.wrap(buffer, offset, length);
|
||||
return pread(position, bb);
|
||||
}
|
||||
|
||||
private int pread(long position, ByteBuffer buffer)
|
||||
@ -1423,7 +1391,7 @@ protected void reportCheckSumFailure(CorruptedBlocks corruptedBlocks,
|
||||
|
||||
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap =
|
||||
corruptedBlocks.getCorruptionMap();
|
||||
if (corruptedBlockMap.isEmpty()) {
|
||||
if (corruptedBlockMap == null) {
|
||||
return;
|
||||
}
|
||||
List<LocatedBlock> reportList = new ArrayList<>(corruptedBlockMap.size());
|
||||
|
@ -396,7 +396,7 @@ static String concatSuffixes(String... suffixes) {
|
||||
* @param keys Set of keys to look for in the order of preference
|
||||
* @return a map(nameserviceId to map(namenodeId to InetSocketAddress))
|
||||
*/
|
||||
static Map<String, Map<String, InetSocketAddress>> getAddresses(
|
||||
public static Map<String, Map<String, InetSocketAddress>> getAddresses(
|
||||
Configuration conf, String defaultAddress, String... keys) {
|
||||
Collection<String> nameserviceIds = getNameServiceIds(conf);
|
||||
return getAddressesForNsIds(conf, nameserviceIds, defaultAddress, keys);
|
||||
@ -426,7 +426,7 @@ static Map<String, Map<String, InetSocketAddress>> getAddressesForNsIds(
|
||||
return ret;
|
||||
}
|
||||
|
||||
static Map<String, InetSocketAddress> getAddressesForNameserviceId(
|
||||
public static Map<String, InetSocketAddress> getAddressesForNameserviceId(
|
||||
Configuration conf, String nsId, String defaultValue, String... keys) {
|
||||
Collection<String> nnIds = getNameNodeIds(conf, nsId);
|
||||
Map<String, InetSocketAddress> ret = Maps.newLinkedHashMap();
|
||||
@ -751,14 +751,14 @@ public static InterruptedIOException toInterruptedIOException(String message,
|
||||
public static class CorruptedBlocks {
|
||||
private Map<ExtendedBlock, Set<DatanodeInfo>> corruptionMap;
|
||||
|
||||
public CorruptedBlocks() {
|
||||
this.corruptionMap = new HashMap<>();
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicate a block replica on the specified datanode is corrupted
|
||||
*/
|
||||
public void addCorruptedBlock(ExtendedBlock blk, DatanodeInfo node) {
|
||||
if (corruptionMap == null) {
|
||||
corruptionMap = new HashMap<>();
|
||||
}
|
||||
|
||||
Set<DatanodeInfo> dnSet = corruptionMap.get(blk);
|
||||
if (dnSet == null) {
|
||||
dnSet = new HashSet<>();
|
||||
@ -770,7 +770,8 @@ public void addCorruptedBlock(ExtendedBlock blk, DatanodeInfo node) {
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the map that contains all the corruption entries.
|
||||
* @return the map that contains all the corruption entries, or null if
|
||||
* there were no corrupted entries
|
||||
*/
|
||||
public Map<ExtendedBlock, Set<DatanodeInfo>> getCorruptionMap() {
|
||||
return corruptionMap;
|
||||
|
@ -184,6 +184,9 @@ public interface HdfsClientConfigKeys {
|
||||
"dfs.namenode.snapshot.capture.openfiles";
|
||||
boolean DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES_DEFAULT = false;
|
||||
|
||||
String DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS =
|
||||
"dfs.provided.aliasmap.inmemory.dnrpc-address";
|
||||
|
||||
/**
|
||||
* These are deprecated config keys to client code.
|
||||
*/
|
||||
|
@ -75,7 +75,6 @@
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.htrace.core.Tracer;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
@ -189,11 +188,6 @@ public boolean getSupportsReceiptVerification() {
|
||||
*/
|
||||
private Configuration configuration;
|
||||
|
||||
/**
|
||||
* The HTrace tracer to use.
|
||||
*/
|
||||
private Tracer tracer;
|
||||
|
||||
/**
|
||||
* Information about the domain socket path we should use to connect to the
|
||||
* local peer-- or null if we haven't examined the local domain socket.
|
||||
@ -298,11 +292,6 @@ public BlockReaderFactory setConfiguration(
|
||||
return this;
|
||||
}
|
||||
|
||||
public BlockReaderFactory setTracer(Tracer tracer) {
|
||||
this.tracer = tracer;
|
||||
return this;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static void setFailureInjectorForTesting(FailureInjector injector) {
|
||||
failureInjector = injector;
|
||||
@ -451,7 +440,7 @@ private BlockReader getLegacyBlockReaderLocal() throws IOException {
|
||||
try {
|
||||
return BlockReaderLocalLegacy.newBlockReader(conf,
|
||||
userGroupInformation, configuration, fileName, block, token,
|
||||
datanode, startOffset, length, storageType, tracer);
|
||||
datanode, startOffset, length, storageType);
|
||||
} catch (RemoteException remoteException) {
|
||||
ioe = remoteException.unwrapRemoteException(
|
||||
InvalidToken.class, AccessControlException.class);
|
||||
@ -509,7 +498,6 @@ private BlockReader getBlockReaderLocal() throws IOException {
|
||||
setVerifyChecksum(verifyChecksum).
|
||||
setCachingStrategy(cachingStrategy).
|
||||
setStorageType(storageType).
|
||||
setTracer(tracer).
|
||||
build();
|
||||
}
|
||||
|
||||
@ -610,6 +598,11 @@ private ShortCircuitReplicaInfo requestFileDescriptors(DomainPeer peer,
|
||||
sock.recvFileInputStreams(fis, buf, 0, buf.length);
|
||||
ShortCircuitReplica replica = null;
|
||||
try {
|
||||
if (fis[0] == null || fis[1] == null) {
|
||||
throw new IOException("the datanode " + datanode + " failed to " +
|
||||
"pass a file descriptor (might have reached open file limit).");
|
||||
}
|
||||
|
||||
ExtendedBlockId key =
|
||||
new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
|
||||
if (buf[0] == USE_RECEIPT_VERIFICATION.getNumber()) {
|
||||
@ -860,7 +853,7 @@ private BlockReader getRemoteBlockReader(Peer peer) throws IOException {
|
||||
return BlockReaderRemote.newBlockReader(
|
||||
fileName, block, token, startOffset, length,
|
||||
verifyChecksum, clientName, peer, datanode,
|
||||
clientContext.getPeerCache(), cachingStrategy, tracer,
|
||||
clientContext.getPeerCache(), cachingStrategy,
|
||||
networkDistance);
|
||||
}
|
||||
|
||||
|
@ -35,8 +35,6 @@
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
import org.apache.hadoop.util.DirectBufferPool;
|
||||
import org.apache.hadoop.util.Timer;
|
||||
import org.apache.htrace.core.TraceScope;
|
||||
import org.apache.htrace.core.Tracer;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@ -83,7 +81,6 @@ public static class Builder {
|
||||
private long dataPos;
|
||||
private ExtendedBlock block;
|
||||
private StorageType storageType;
|
||||
private Tracer tracer;
|
||||
private ShortCircuitConf shortCircuitConf;
|
||||
|
||||
public Builder(ShortCircuitConf conf) {
|
||||
@ -131,11 +128,6 @@ public Builder setStorageType(StorageType storageType) {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setTracer(Tracer tracer) {
|
||||
this.tracer = tracer;
|
||||
return this;
|
||||
}
|
||||
|
||||
public BlockReaderLocal build() {
|
||||
Preconditions.checkNotNull(replica);
|
||||
return new BlockReaderLocal(this);
|
||||
@ -244,11 +236,6 @@ public BlockReaderLocal build() {
|
||||
*/
|
||||
private StorageType storageType;
|
||||
|
||||
/**
|
||||
* The Tracer to use.
|
||||
*/
|
||||
private final Tracer tracer;
|
||||
|
||||
private BlockReaderLocal(Builder builder) {
|
||||
this.replica = builder.replica;
|
||||
this.dataIn = replica.getDataStream().getChannel();
|
||||
@ -278,7 +265,6 @@ private BlockReaderLocal(Builder builder) {
|
||||
}
|
||||
this.maxReadaheadLength = maxReadaheadChunks * bytesPerChecksum;
|
||||
this.storageType = builder.storageType;
|
||||
this.tracer = builder.tracer;
|
||||
|
||||
if (builder.shortCircuitConf.isScrMetricsEnabled()) {
|
||||
metricsInitializationLock.lock();
|
||||
@ -360,52 +346,49 @@ private synchronized int drainDataBuf(ByteBuffer buf) {
|
||||
*/
|
||||
private synchronized int fillBuffer(ByteBuffer buf, boolean canSkipChecksum)
|
||||
throws IOException {
|
||||
try (TraceScope ignored = tracer.newScope(
|
||||
"BlockReaderLocal#fillBuffer(" + block.getBlockId() + ")")) {
|
||||
int total = 0;
|
||||
long startDataPos = dataPos;
|
||||
int startBufPos = buf.position();
|
||||
while (buf.hasRemaining()) {
|
||||
int nRead = blockReaderIoProvider.read(dataIn, buf, dataPos);
|
||||
if (nRead < 0) {
|
||||
break;
|
||||
}
|
||||
dataPos += nRead;
|
||||
total += nRead;
|
||||
}
|
||||
if (canSkipChecksum) {
|
||||
freeChecksumBufIfExists();
|
||||
return total;
|
||||
}
|
||||
if (total > 0) {
|
||||
try {
|
||||
buf.limit(buf.position());
|
||||
buf.position(startBufPos);
|
||||
createChecksumBufIfNeeded();
|
||||
int checksumsNeeded = (total + bytesPerChecksum - 1) /
|
||||
bytesPerChecksum;
|
||||
checksumBuf.clear();
|
||||
checksumBuf.limit(checksumsNeeded * checksumSize);
|
||||
long checksumPos = BlockMetadataHeader.getHeaderSize()
|
||||
+ ((startDataPos / bytesPerChecksum) * checksumSize);
|
||||
while (checksumBuf.hasRemaining()) {
|
||||
int nRead = checksumIn.read(checksumBuf, checksumPos);
|
||||
if (nRead < 0) {
|
||||
throw new IOException("Got unexpected checksum file EOF at " +
|
||||
checksumPos + ", block file position " + startDataPos +
|
||||
" for block " + block + " of file " + filename);
|
||||
}
|
||||
checksumPos += nRead;
|
||||
}
|
||||
checksumBuf.flip();
|
||||
|
||||
checksum.verifyChunkedSums(buf, checksumBuf, filename, startDataPos);
|
||||
} finally {
|
||||
buf.position(buf.limit());
|
||||
}
|
||||
int total = 0;
|
||||
long startDataPos = dataPos;
|
||||
int startBufPos = buf.position();
|
||||
while (buf.hasRemaining()) {
|
||||
int nRead = blockReaderIoProvider.read(dataIn, buf, dataPos);
|
||||
if (nRead < 0) {
|
||||
break;
|
||||
}
|
||||
dataPos += nRead;
|
||||
total += nRead;
|
||||
}
|
||||
if (canSkipChecksum) {
|
||||
freeChecksumBufIfExists();
|
||||
return total;
|
||||
}
|
||||
if (total > 0) {
|
||||
try {
|
||||
buf.limit(buf.position());
|
||||
buf.position(startBufPos);
|
||||
createChecksumBufIfNeeded();
|
||||
int checksumsNeeded = (total + bytesPerChecksum - 1) /
|
||||
bytesPerChecksum;
|
||||
checksumBuf.clear();
|
||||
checksumBuf.limit(checksumsNeeded * checksumSize);
|
||||
long checksumPos = BlockMetadataHeader.getHeaderSize()
|
||||
+ ((startDataPos / bytesPerChecksum) * checksumSize);
|
||||
while (checksumBuf.hasRemaining()) {
|
||||
int nRead = checksumIn.read(checksumBuf, checksumPos);
|
||||
if (nRead < 0) {
|
||||
throw new IOException("Got unexpected checksum file EOF at " +
|
||||
checksumPos + ", block file position " + startDataPos +
|
||||
" for block " + block + " of file " + filename);
|
||||
}
|
||||
checksumPos += nRead;
|
||||
}
|
||||
checksumBuf.flip();
|
||||
|
||||
checksum.verifyChunkedSums(buf, checksumBuf, filename, startDataPos);
|
||||
} finally {
|
||||
buf.position(buf.limit());
|
||||
}
|
||||
}
|
||||
return total;
|
||||
}
|
||||
|
||||
private boolean createNoChecksumContext() {
|
||||
|
@ -51,8 +51,6 @@
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
import org.apache.hadoop.util.DirectBufferPool;
|
||||
import org.apache.htrace.core.TraceScope;
|
||||
import org.apache.htrace.core.Tracer;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@ -184,7 +182,6 @@ private void removeBlockLocalPathInfo(ExtendedBlock b) {
|
||||
private long startOffset;
|
||||
private final String filename;
|
||||
private long blockId;
|
||||
private final Tracer tracer;
|
||||
|
||||
/**
|
||||
* The only way this object can be instantiated.
|
||||
@ -193,8 +190,8 @@ static BlockReaderLocalLegacy newBlockReader(DfsClientConf conf,
|
||||
UserGroupInformation userGroupInformation,
|
||||
Configuration configuration, String file, ExtendedBlock blk,
|
||||
Token<BlockTokenIdentifier> token, DatanodeInfo node,
|
||||
long startOffset, long length, StorageType storageType,
|
||||
Tracer tracer) throws IOException {
|
||||
long startOffset, long length, StorageType storageType)
|
||||
throws IOException {
|
||||
final ShortCircuitConf scConf = conf.getShortCircuitConf();
|
||||
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node
|
||||
.getIpcPort());
|
||||
@ -239,11 +236,10 @@ static BlockReaderLocalLegacy newBlockReader(DfsClientConf conf,
|
||||
long firstChunkOffset = startOffset
|
||||
- (startOffset % checksum.getBytesPerChecksum());
|
||||
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk,
|
||||
startOffset, checksum, true, dataIn, firstChunkOffset, checksumIn,
|
||||
tracer);
|
||||
startOffset, checksum, true, dataIn, firstChunkOffset, checksumIn);
|
||||
} else {
|
||||
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk,
|
||||
startOffset, dataIn, tracer);
|
||||
startOffset, dataIn);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
// remove from cache
|
||||
@ -320,17 +316,17 @@ private static int getSlowReadBufferNumChunks(int bufferSizeBytes,
|
||||
}
|
||||
|
||||
private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
|
||||
ExtendedBlock block, long startOffset, FileInputStream dataIn,
|
||||
Tracer tracer) throws IOException {
|
||||
ExtendedBlock block, long startOffset, FileInputStream dataIn)
|
||||
throws IOException {
|
||||
this(conf, hdfsfile, block, startOffset,
|
||||
DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false,
|
||||
dataIn, startOffset, null, tracer);
|
||||
dataIn, startOffset, null);
|
||||
}
|
||||
|
||||
private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
|
||||
ExtendedBlock block, long startOffset, DataChecksum checksum,
|
||||
boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
|
||||
FileInputStream checksumIn, Tracer tracer) throws IOException {
|
||||
FileInputStream checksumIn) throws IOException {
|
||||
this.filename = hdfsfile;
|
||||
this.checksum = checksum;
|
||||
this.verifyChecksum = verifyChecksum;
|
||||
@ -369,7 +365,6 @@ private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
|
||||
bufferPool.returnBuffer(checksumBuff);
|
||||
}
|
||||
}
|
||||
this.tracer = tracer;
|
||||
}
|
||||
|
||||
/**
|
||||
@ -377,23 +372,20 @@ private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
|
||||
*/
|
||||
private int fillBuffer(FileInputStream stream, ByteBuffer buf)
|
||||
throws IOException {
|
||||
try (TraceScope ignored = tracer.
|
||||
newScope("BlockReaderLocalLegacy#fillBuffer(" + blockId + ")")) {
|
||||
int bytesRead = stream.getChannel().read(buf);
|
||||
if (bytesRead < 0) {
|
||||
int bytesRead = stream.getChannel().read(buf);
|
||||
if (bytesRead < 0) {
|
||||
//EOF
|
||||
return bytesRead;
|
||||
}
|
||||
while (buf.remaining() > 0) {
|
||||
int n = stream.getChannel().read(buf);
|
||||
if (n < 0) {
|
||||
//EOF
|
||||
return bytesRead;
|
||||
}
|
||||
while (buf.remaining() > 0) {
|
||||
int n = stream.getChannel().read(buf);
|
||||
if (n < 0) {
|
||||
//EOF
|
||||
return bytesRead;
|
||||
}
|
||||
bytesRead += n;
|
||||
}
|
||||
return bytesRead;
|
||||
bytesRead += n;
|
||||
}
|
||||
return bytesRead;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -49,11 +49,9 @@
|
||||
import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
import org.apache.htrace.core.TraceScope;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import org.apache.htrace.core.Tracer;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@ -121,8 +119,6 @@ public class BlockReaderRemote implements BlockReader {
|
||||
|
||||
private boolean sentStatusCode = false;
|
||||
|
||||
private final Tracer tracer;
|
||||
|
||||
private final int networkDistance;
|
||||
|
||||
@VisibleForTesting
|
||||
@ -133,19 +129,22 @@ public Peer getPeer() {
|
||||
@Override
|
||||
public synchronized int read(byte[] buf, int off, int len)
|
||||
throws IOException {
|
||||
UUID randomId = (LOG.isTraceEnabled() ? UUID.randomUUID() : null);
|
||||
LOG.trace("Starting read #{} file {} from datanode {}",
|
||||
randomId, filename, datanodeID.getHostName());
|
||||
boolean logTraceEnabled = LOG.isTraceEnabled();
|
||||
UUID randomId = null;
|
||||
if (logTraceEnabled) {
|
||||
randomId = UUID.randomUUID();
|
||||
LOG.trace("Starting read #{} file {} from datanode {}",
|
||||
randomId, filename, datanodeID.getHostName());
|
||||
}
|
||||
|
||||
if (curDataSlice == null ||
|
||||
curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
|
||||
try (TraceScope ignored = tracer.newScope(
|
||||
"BlockReaderRemote2#readNextPacket(" + blockId + ")")) {
|
||||
readNextPacket();
|
||||
}
|
||||
readNextPacket();
|
||||
}
|
||||
|
||||
LOG.trace("Finishing read #{}", randomId);
|
||||
if (logTraceEnabled) {
|
||||
LOG.trace("Finishing read #{}", randomId);
|
||||
}
|
||||
|
||||
if (curDataSlice.remaining() == 0) {
|
||||
// we're at EOF now
|
||||
@ -163,10 +162,7 @@ public synchronized int read(byte[] buf, int off, int len)
|
||||
public synchronized int read(ByteBuffer buf) throws IOException {
|
||||
if (curDataSlice == null ||
|
||||
(curDataSlice.remaining() == 0 && bytesNeededToFinish > 0)) {
|
||||
try (TraceScope ignored = tracer.newScope(
|
||||
"BlockReaderRemote2#readNextPacket(" + blockId + ")")) {
|
||||
readNextPacket();
|
||||
}
|
||||
readNextPacket();
|
||||
}
|
||||
if (curDataSlice.remaining() == 0) {
|
||||
// we're at EOF now
|
||||
@ -280,7 +276,6 @@ protected BlockReaderRemote(String file, long blockId,
|
||||
long startOffset, long firstChunkOffset,
|
||||
long bytesToRead, Peer peer,
|
||||
DatanodeID datanodeID, PeerCache peerCache,
|
||||
Tracer tracer,
|
||||
int networkDistance) {
|
||||
// Path is used only for printing block and file information in debug
|
||||
this.peer = peer;
|
||||
@ -300,7 +295,6 @@ protected BlockReaderRemote(String file, long blockId,
|
||||
this.bytesNeededToFinish = bytesToRead + (startOffset - firstChunkOffset);
|
||||
bytesPerChecksum = this.checksum.getBytesPerChecksum();
|
||||
checksumSize = this.checksum.getChecksumSize();
|
||||
this.tracer = tracer;
|
||||
this.networkDistance = networkDistance;
|
||||
}
|
||||
|
||||
@ -397,7 +391,6 @@ public static BlockReader newBlockReader(String file,
|
||||
Peer peer, DatanodeID datanodeID,
|
||||
PeerCache peerCache,
|
||||
CachingStrategy cachingStrategy,
|
||||
Tracer tracer,
|
||||
int networkDistance) throws IOException {
|
||||
// in and out will be closed when sock is closed (by the caller)
|
||||
final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
|
||||
@ -431,7 +424,7 @@ public static BlockReader newBlockReader(String file,
|
||||
|
||||
return new BlockReaderRemote(file, block.getBlockId(), checksum,
|
||||
verifyChecksum, startOffset, firstChunkOffset, len, peer, datanodeID,
|
||||
peerCache, tracer, networkDistance);
|
||||
peerCache, networkDistance);
|
||||
}
|
||||
|
||||
static void checkSuccess(
|
||||
|
@ -37,7 +37,7 @@ public class ReplicaNotFoundException extends IOException {
|
||||
"Cannot recover append/close to a replica that's not FINALIZED and not RBW"
|
||||
+ " ";
|
||||
public final static String NON_EXISTENT_REPLICA =
|
||||
"Cannot append to a non-existent replica ";
|
||||
"Replica does not exist ";
|
||||
public final static String UNEXPECTED_GS_REPLICA =
|
||||
"Cannot append to a replica with unexpected generation stamp ";
|
||||
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user