|
@@ -1,43 +1,61 @@
|
|
/**
|
|
/**
|
|
- * 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
|
|
|
|
|
|
+ * Licensed to the Apache Software Foundation (ASF) under one or more
|
|
|
|
+ * contributor license agreements. See the NOTICE file distributed with this
|
|
|
|
+ * work for additional information regarding copyright ownership. The ASF
|
|
|
|
+ * licenses this file to you under the Apache License, Version 2.0 (the
|
|
|
|
+ * "License"); you may not use this file except in compliance with the License.
|
|
|
|
+ * You may obtain a copy of the License at
|
|
* <p>
|
|
* <p>
|
|
* http://www.apache.org/licenses/LICENSE-2.0
|
|
* http://www.apache.org/licenses/LICENSE-2.0
|
|
* <p>
|
|
* <p>
|
|
* Unless required by applicable law or agreed to in writing, software
|
|
* 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.
|
|
|
|
|
|
+ * 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;
|
|
package org.apache.hadoop.ozone;
|
|
|
|
|
|
import com.google.common.base.Optional;
|
|
import com.google.common.base.Optional;
|
|
|
|
|
|
|
|
+import com.google.common.net.HostAndPort;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.net.NetUtils;
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
|
+import org.apache.hadoop.scm.ScmConfigKeys;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import java.net.InetSocketAddress;
|
|
import java.net.InetSocketAddress;
|
|
import java.util.HashMap;
|
|
import java.util.HashMap;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.*;
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.*;
|
|
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_DEADNODE_INTERVAL_DEFAULT;
|
|
|
|
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_DEADNODE_INTERVAL_MS;
|
|
|
|
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS;
|
|
|
|
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS;
|
|
|
|
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_STALENODE_INTERVAL_DEFAULT;
|
|
|
|
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_STALENODE_INTERVAL_MS;
|
|
|
|
|
|
+import static org.apache.hadoop.ozone.OzoneConfigKeys
|
|
|
|
+ .OZONE_SCM_DEADNODE_INTERVAL_DEFAULT;
|
|
|
|
+import static org.apache.hadoop.ozone.OzoneConfigKeys
|
|
|
|
+ .OZONE_SCM_DEADNODE_INTERVAL_MS;
|
|
|
|
+import static org.apache.hadoop.ozone.OzoneConfigKeys
|
|
|
|
+ .OZONE_SCM_HEARTBEAT_INTERVAL_SECONDS;
|
|
|
|
+
|
|
|
|
+import static org.apache.hadoop.ozone.OzoneConfigKeys
|
|
|
|
+ .OZONE_SCM_HEARTBEAT_LOG_WARN_DEFAULT;
|
|
|
|
+import static org.apache.hadoop.ozone.OzoneConfigKeys
|
|
|
|
+ .OZONE_SCM_HEARTBEAT_LOG_WARN_INTERVAL_COUNT;
|
|
|
|
+import static org.apache.hadoop.ozone.OzoneConfigKeys
|
|
|
|
+ .OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS;
|
|
|
|
+
|
|
|
|
+import static org.apache.hadoop.ozone.OzoneConfigKeys
|
|
|
|
+ .OZONE_SCM_HEARTBEAT_RPC_TIMEOUT;
|
|
|
|
+import static org.apache.hadoop.ozone.OzoneConfigKeys
|
|
|
|
+ .OZONE_SCM_HEARTBEAT_RPC_TIMEOUT_DEFAULT;
|
|
|
|
+import static org.apache.hadoop.ozone.OzoneConfigKeys
|
|
|
|
+ .OZONE_SCM_STALENODE_INTERVAL_DEFAULT;
|
|
|
|
+import static org.apache.hadoop.ozone.OzoneConfigKeys
|
|
|
|
+ .OZONE_SCM_STALENODE_INTERVAL_MS;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Utility methods for Ozone and Container Clients.
|
|
* Utility methods for Ozone and Container Clients.
|
|
@@ -51,6 +69,7 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_STALENODE_INTERV
|
|
public final class OzoneClientUtils {
|
|
public final class OzoneClientUtils {
|
|
private static final Logger LOG = LoggerFactory.getLogger(
|
|
private static final Logger LOG = LoggerFactory.getLogger(
|
|
OzoneClientUtils.class);
|
|
OzoneClientUtils.class);
|
|
|
|
+ private static final int NO_PORT = -1;
|
|
|
|
|
|
/**
|
|
/**
|
|
* The service ID of the solitary Ozone SCM service.
|
|
* The service ID of the solitary Ozone SCM service.
|
|
@@ -139,7 +158,7 @@ public final class OzoneClientUtils {
|
|
|
|
|
|
return NetUtils.createSocketAddr(
|
|
return NetUtils.createSocketAddr(
|
|
host.or(OZONE_SCM_CLIENT_BIND_HOST_DEFAULT) + ":" +
|
|
host.or(OZONE_SCM_CLIENT_BIND_HOST_DEFAULT) + ":" +
|
|
- port.or(OZONE_SCM_CLIENT_PORT_DEFAULT));
|
|
|
|
|
|
+ port.or(OZONE_SCM_CLIENT_PORT_DEFAULT));
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -160,7 +179,7 @@ public final class OzoneClientUtils {
|
|
|
|
|
|
return NetUtils.createSocketAddr(
|
|
return NetUtils.createSocketAddr(
|
|
host.or(OZONE_SCM_DATANODE_BIND_HOST_DEFAULT) + ":" +
|
|
host.or(OZONE_SCM_DATANODE_BIND_HOST_DEFAULT) + ":" +
|
|
- port.or(OZONE_SCM_DATANODE_PORT_DEFAULT));
|
|
|
|
|
|
+ port.or(OZONE_SCM_DATANODE_PORT_DEFAULT));
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -168,7 +187,7 @@ public final class OzoneClientUtils {
|
|
* Each config value may be absent, or if present in the format
|
|
* Each config value may be absent, or if present in the format
|
|
* host:port (the :port part is optional).
|
|
* host:port (the :port part is optional).
|
|
*
|
|
*
|
|
- * @param conf
|
|
|
|
|
|
+ * @param conf - Conf
|
|
* @param keys a list of configuration key names.
|
|
* @param keys a list of configuration key names.
|
|
*
|
|
*
|
|
* @return first hostname component found from the given keys, or absent.
|
|
* @return first hostname component found from the given keys, or absent.
|
|
@@ -176,51 +195,65 @@ public final class OzoneClientUtils {
|
|
* or host:port format.
|
|
* or host:port format.
|
|
*/
|
|
*/
|
|
static Optional<String> getHostNameFromConfigKeys(
|
|
static Optional<String> getHostNameFromConfigKeys(
|
|
- Configuration conf, String ... keys) {
|
|
|
|
|
|
+ Configuration conf, String... keys) {
|
|
for (final String key : keys) {
|
|
for (final String key : keys) {
|
|
final String value = conf.getTrimmed(key);
|
|
final String value = conf.getTrimmed(key);
|
|
- if (value != null && !value.isEmpty()) {
|
|
|
|
- String[] splits = value.split(":");
|
|
|
|
-
|
|
|
|
- if(splits.length < 1 || splits.length > 2) {
|
|
|
|
- throw new IllegalArgumentException(
|
|
|
|
- "Invalid value " + value + " for config key " + key +
|
|
|
|
- ". It should be in 'host' or 'host:port' format");
|
|
|
|
- }
|
|
|
|
- return Optional.of(splits[0]);
|
|
|
|
|
|
+ final Optional<String> hostName = getHostName(value);
|
|
|
|
+ if (hostName.isPresent()) {
|
|
|
|
+ return hostName;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return Optional.absent();
|
|
return Optional.absent();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Gets the hostname or Indicates that it is absent.
|
|
|
|
+ * @param value host or host:port
|
|
|
|
+ * @return hostname
|
|
|
|
+ */
|
|
|
|
+ public static Optional<String> getHostName(String value) {
|
|
|
|
+ if ((value == null) || value.isEmpty()) {
|
|
|
|
+ return Optional.absent();
|
|
|
|
+ }
|
|
|
|
+ return Optional.of(HostAndPort.fromString(value).getHostText());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Gets the port if there is one, throws otherwise.
|
|
|
|
+ * @param value String in host:port format.
|
|
|
|
+ * @return Port
|
|
|
|
+ */
|
|
|
|
+ public static Optional<Integer> getHostPort(String value) {
|
|
|
|
+ if((value == null) || value.isEmpty()) {
|
|
|
|
+ return Optional.absent();
|
|
|
|
+ }
|
|
|
|
+ int port = HostAndPort.fromString(value).getPortOrDefault(NO_PORT);
|
|
|
|
+ if (port == NO_PORT) {
|
|
|
|
+ return Optional.absent();
|
|
|
|
+ } else {
|
|
|
|
+ return Optional.of(port);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Retrieve the port number, trying the supplied config keys in order.
|
|
* Retrieve the port number, trying the supplied config keys in order.
|
|
* Each config value may be absent, or if present in the format
|
|
* Each config value may be absent, or if present in the format
|
|
* host:port (the :port part is optional).
|
|
* host:port (the :port part is optional).
|
|
*
|
|
*
|
|
- * @param conf
|
|
|
|
|
|
+ * @param conf Conf
|
|
* @param keys a list of configuration key names.
|
|
* @param keys a list of configuration key names.
|
|
*
|
|
*
|
|
* @return first port number component found from the given keys, or absent.
|
|
* @return first port number component found from the given keys, or absent.
|
|
* @throws IllegalArgumentException if any values are not in the 'host'
|
|
* @throws IllegalArgumentException if any values are not in the 'host'
|
|
* or host:port format.
|
|
* or host:port format.
|
|
*/
|
|
*/
|
|
- static Optional<Integer> getPortNumberFromConfigKeys(
|
|
|
|
- Configuration conf, String ... keys) {
|
|
|
|
|
|
+ public static Optional<Integer> getPortNumberFromConfigKeys(
|
|
|
|
+ Configuration conf, String... keys) {
|
|
for (final String key : keys) {
|
|
for (final String key : keys) {
|
|
final String value = conf.getTrimmed(key);
|
|
final String value = conf.getTrimmed(key);
|
|
- if (value != null && !value.isEmpty()) {
|
|
|
|
- String[] splits = value.split(":");
|
|
|
|
-
|
|
|
|
- if(splits.length < 1 || splits.length > 2) {
|
|
|
|
- throw new IllegalArgumentException(
|
|
|
|
- "Invalid value " + value + " for config key " + key +
|
|
|
|
- ". It should be in 'host' or 'host:port' format");
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- if (splits.length == 2) {
|
|
|
|
- return Optional.of(Integer.parseInt(splits[1]));
|
|
|
|
- }
|
|
|
|
|
|
+ final Optional<Integer> hostPort = getHostPort(value);
|
|
|
|
+ if (hostPort.isPresent()) {
|
|
|
|
+ return hostPort;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return Optional.absent();
|
|
return Optional.absent();
|
|
@@ -259,7 +292,7 @@ public final class OzoneClientUtils {
|
|
* @return long
|
|
* @return long
|
|
*/
|
|
*/
|
|
private static long sanitizeUserArgs(long valueTocheck, long baseValue,
|
|
private static long sanitizeUserArgs(long valueTocheck, long baseValue,
|
|
- long minFactor, long maxFactor)
|
|
|
|
|
|
+ long minFactor, long maxFactor)
|
|
throws IllegalArgumentException {
|
|
throws IllegalArgumentException {
|
|
if ((valueTocheck >= (baseValue * minFactor)) &&
|
|
if ((valueTocheck >= (baseValue * minFactor)) &&
|
|
(valueTocheck <= (baseValue * maxFactor))) {
|
|
(valueTocheck <= (baseValue * maxFactor))) {
|
|
@@ -270,7 +303,6 @@ public final class OzoneClientUtils {
|
|
throw new IllegalArgumentException(errMsg);
|
|
throw new IllegalArgumentException(errMsg);
|
|
}
|
|
}
|
|
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Returns the interval in which the heartbeat processor thread runs.
|
|
* Returns the interval in which the heartbeat processor thread runs.
|
|
*
|
|
*
|
|
@@ -282,7 +314,6 @@ public final class OzoneClientUtils {
|
|
OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS_DEFAULT);
|
|
OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL_MS_DEFAULT);
|
|
}
|
|
}
|
|
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Heartbeat Interval - Defines the heartbeat frequency from a datanode to
|
|
* Heartbeat Interval - Defines the heartbeat frequency from a datanode to
|
|
* SCM.
|
|
* SCM.
|
|
@@ -295,7 +326,6 @@ public final class OzoneClientUtils {
|
|
OZONE_SCM_HEARBEAT_INTERVAL_SECONDS_DEFAULT);
|
|
OZONE_SCM_HEARBEAT_INTERVAL_SECONDS_DEFAULT);
|
|
}
|
|
}
|
|
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Get the Stale Node interval, which is used by SCM to flag a datanode as
|
|
* Get the Stale Node interval, which is used by SCM to flag a datanode as
|
|
* stale, if the heartbeat from that node has been missing for this duration.
|
|
* stale, if the heartbeat from that node has been missing for this duration.
|
|
@@ -340,7 +370,6 @@ public final class OzoneClientUtils {
|
|
return staleNodeIntevalMs;
|
|
return staleNodeIntevalMs;
|
|
}
|
|
}
|
|
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Gets the interval for dead node flagging. This has to be a value that is
|
|
* Gets the interval for dead node flagging. This has to be a value that is
|
|
* greater than stale node value, and by transitive relation we also know
|
|
* greater than stale node value, and by transitive relation we also know
|
|
@@ -374,8 +403,42 @@ public final class OzoneClientUtils {
|
|
* @param conf Configration
|
|
* @param conf Configration
|
|
* @return - int -- Number of HBs to process
|
|
* @return - int -- Number of HBs to process
|
|
*/
|
|
*/
|
|
- public static int getMaxHBToProcessPerLoop(Configuration conf){
|
|
|
|
|
|
+ public static int getMaxHBToProcessPerLoop(Configuration conf) {
|
|
return conf.getInt(OZONE_SCM_MAX_HB_COUNT_TO_PROCESS,
|
|
return conf.getInt(OZONE_SCM_MAX_HB_COUNT_TO_PROCESS,
|
|
OZONE_SCM_MAX_HB_COUNT_TO_PROCESS_DEFAULT);
|
|
OZONE_SCM_MAX_HB_COUNT_TO_PROCESS_DEFAULT);
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Timeout value for the RPC from Datanode to SCM, primarily used for
|
|
|
|
+ * Heartbeats and container reports.
|
|
|
|
+ *
|
|
|
|
+ * @param conf - Ozone Config
|
|
|
|
+ * @return - Rpc timeout in Milliseconds.
|
|
|
|
+ */
|
|
|
|
+ public static long getScmRpcTimeOutInMilliseconds(Configuration conf) {
|
|
|
|
+ return conf.getTimeDuration(OZONE_SCM_HEARTBEAT_RPC_TIMEOUT,
|
|
|
|
+ OZONE_SCM_HEARTBEAT_RPC_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Log Warn interval.
|
|
|
|
+ *
|
|
|
|
+ * @param conf - Ozone Config
|
|
|
|
+ * @return - Log warn interval.
|
|
|
|
+ */
|
|
|
|
+ public static int getLogWarnInterval(Configuration conf) {
|
|
|
|
+ return conf.getInt(OZONE_SCM_HEARTBEAT_LOG_WARN_INTERVAL_COUNT,
|
|
|
|
+ OZONE_SCM_HEARTBEAT_LOG_WARN_DEFAULT);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * returns the Container port.
|
|
|
|
+ * @param conf - Conf
|
|
|
|
+ * @return port number.
|
|
|
|
+ */
|
|
|
|
+ public static int getContainerPort(Configuration conf) {
|
|
|
|
+ return conf.getInt(ScmConfigKeys.DFS_CONTAINER_IPC_PORT, ScmConfigKeys
|
|
|
|
+ .DFS_CONTAINER_IPC_PORT_DEFAULT);
|
|
|
|
+ }
|
|
|
|
+
|
|
}
|
|
}
|