HDFS-6564. Use slf4j instead of common-logging in hdfs-client. Contributed by Rakesh R.

This commit is contained in:
Haohui Mai 2015-06-23 11:40:21 -07:00
parent d481684c7c
commit 122cad6aec
8 changed files with 49 additions and 42 deletions

View File

@ -35,6 +35,16 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-common</artifactId>
<scope>provided</scope>
<exclusions>
<exclusion>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
</exclusion>
<exclusion>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
</exclusions>
</dependency>
</dependencies>

View File

@ -317,10 +317,9 @@ static Map<String, InetSocketAddress> getAddressesForNameserviceId(
if (address != null) {
InetSocketAddress isa = NetUtils.createSocketAddr(address);
if (isa.isUnresolved()) {
LOG.warn("Namenode for " + nsId +
" remains unresolved for ID " + nnId +
". Check your hdfs-site.xml file to " +
"ensure namenodes are configured properly.");
LOG.warn("Namenode for {} remains unresolved for ID {}. Check your "
+ "hdfs-site.xml file to ensure namenodes are configured "
+ "properly.", nsId, nnId);
}
ret.put(nnId, isa);
}

View File

@ -158,13 +158,11 @@ public List<StorageType> chooseStorageTypes(final short replication,
// remove excess storage types after fallback replacement.
diff(storageTypes, excess, null);
if (storageTypes.size() < expectedSize) {
LOG.warn("Failed to place enough replicas: expected size is " + expectedSize
+ " but only " + storageTypes.size() + " storage types can be selected "
+ "(replication=" + replication
+ ", selected=" + storageTypes
+ ", unavailable=" + unavailables
+ ", removed=" + removed
+ ", policy=" + this + ")");
LOG.warn("Failed to place enough replicas: expected size is {}"
+ " but only {} storage types can be selected (replication={},"
+ " selected={}, unavailable={}" + ", removed={}" + ", policy={}"
+ ")", expectedSize, storageTypes.size(), replication, storageTypes,
unavailables, removed, this);
}
return storageTypes;
}

View File

@ -24,8 +24,6 @@
import org.apache.commons.lang.builder.EqualsBuilder;
import org.apache.commons.lang.builder.HashCodeBuilder;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.InvalidRequestException;
@ -41,7 +39,6 @@
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class CachePoolInfo {
public static final Log LOG = LogFactory.getLog(CachePoolInfo.class);
/**
* Indicates that the pool does not have a maximum relative expiry.

View File

@ -21,8 +21,6 @@
import java.net.InetSocketAddress;
import java.net.URI;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.DelegationTokenRenewer;
@ -37,6 +35,8 @@
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenRenewer;
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting;
@ -115,11 +115,11 @@ interface TokenManagementDelegator {
private final DTSelecorByKind dtSelector;
private final T fs;
private boolean hasInitedToken;
private final Log LOG;
private final Logger LOG;
private final Text serviceName;
TokenAspect(T fs, final Text serviceName, final Text kind) {
this.LOG = LogFactory.getLog(fs.getClass());
this.LOG = LoggerFactory.getLogger(fs.getClass());
this.fs = fs;
this.dtSelector = new DTSelecorByKind(kind);
this.serviceName = serviceName;
@ -134,8 +134,8 @@ synchronized void ensureTokenInitialized() throws IOException {
if (token != null) {
fs.setDelegationToken(token);
addRenewAction(fs);
if(LOG.isDebugEnabled()) {
LOG.debug("Created new DT for " + token.getService());
if (LOG.isDebugEnabled()) {
LOG.debug("Created new DT for {}", token.getService());
}
}
hasInitedToken = true;
@ -149,8 +149,8 @@ public synchronized void reset() {
synchronized void initDelegationToken(UserGroupInformation ugi) {
Token<?> token = selectDelegationToken(ugi);
if (token != null) {
if(LOG.isDebugEnabled()) {
LOG.debug("Found existing DT for " + token.getService());
if (LOG.isDebugEnabled()) {
LOG.debug("Found existing DT for {}", token.getService());
}
fs.setDelegationToken(token);
hasInitedToken = true;

View File

@ -28,8 +28,6 @@
import javax.net.ssl.HttpsURLConnection;
import javax.net.ssl.SSLSocketFactory;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -38,6 +36,8 @@
import org.apache.hadoop.security.authentication.client.AuthenticationException;
import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
import org.apache.hadoop.security.ssl.SSLFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting;
@ -47,7 +47,8 @@
@InterfaceAudience.LimitedPrivate({ "HDFS" })
@InterfaceStability.Unstable
public class URLConnectionFactory {
private static final Log LOG = LogFactory.getLog(URLConnectionFactory.class);
private static final Logger LOG = LoggerFactory
.getLogger(URLConnectionFactory.class);
/**
* Timeout for socket connects and reads
@ -154,16 +155,14 @@ public URLConnection openConnection(URL url, boolean isSpnego)
throws IOException, AuthenticationException {
if (isSpnego) {
if (LOG.isDebugEnabled()) {
LOG.debug("open AuthenticatedURL connection" + url);
LOG.debug("open AuthenticatedURL connection {}", url);
}
UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
return new AuthenticatedURL(new KerberosUgiAuthenticator(),
connConfigurator).openConnection(url, authToken);
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("open URL connection");
}
URLConnection connection = url.openConnection();
if (connection instanceof HttpURLConnection) {
connConfigurator.configure((HttpURLConnection) connection);

View File

@ -36,8 +36,6 @@
import javax.ws.rs.core.MediaType;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.CommonConfigurationKeys;
@ -81,6 +79,8 @@
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.StringUtils;
import org.codehaus.jackson.map.ObjectMapper;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
@ -89,7 +89,8 @@
/** A FileSystem for HDFS over the web. */
public class WebHdfsFileSystem extends FileSystem
implements DelegationTokenRenewer.Renewable, TokenAspect.TokenManagementDelegator {
public static final Log LOG = LogFactory.getLog(WebHdfsFileSystem.class);
public static final Logger LOG = LoggerFactory
.getLogger(WebHdfsFileSystem.class);
/** WebHdfs version. */
public static final int VERSION = 1;
/** Http URI: http://namenode:port/{PATH_PREFIX}/path/to/file */
@ -221,14 +222,14 @@ protected synchronized Token<?> getDelegationToken() throws IOException {
// to get another token to match hdfs/rpc behavior
if (token != null) {
if(LOG.isDebugEnabled()) {
LOG.debug("Using UGI token: " + token);
LOG.debug("Using UGI token: {}", token);
}
canRefreshDelegationToken = false;
} else {
token = getDelegationToken(null);
if (token != null) {
if(LOG.isDebugEnabled()) {
LOG.debug("Fetched new token: " + token);
LOG.debug("Fetched new token: {}", token);
}
} else { // security is disabled
canRefreshDelegationToken = false;
@ -245,7 +246,7 @@ synchronized boolean replaceExpiredDelegationToken() throws IOException {
if (canRefreshDelegationToken) {
Token<?> token = getDelegationToken(null);
if(LOG.isDebugEnabled()) {
LOG.debug("Replaced expired token: " + token);
LOG.debug("Replaced expired token: {}", token);
}
setDelegationToken(token);
replaced = (token != null);
@ -430,7 +431,7 @@ private URL getNamenodeURL(String path, String query) throws IOException {
final URL url = new URL(getTransportScheme(), nnAddr.getHostName(),
nnAddr.getPort(), path + '?' + query);
if (LOG.isTraceEnabled()) {
LOG.trace("url=" + url);
LOG.trace("url={}", url);
}
return url;
}
@ -467,7 +468,7 @@ URL toUrl(final HttpOpParam.Op op, final Path fspath,
+ Param.toSortedString("&", parameters);
final URL url = getNamenodeURL(path, query);
if (LOG.isTraceEnabled()) {
LOG.trace("url=" + url);
LOG.trace("url={}", url);
}
return url;
}
@ -658,9 +659,9 @@ private void shouldRetry(final IOException ioe, final int retry
a.action == RetryPolicy.RetryAction.RetryDecision.FAILOVER_AND_RETRY;
if (isRetry || isFailoverAndRetry) {
LOG.info("Retrying connect to namenode: " + nnAddr
+ ". Already tried " + retry + " time(s); retry policy is "
+ retryPolicy + ", delay " + a.delayMillis + "ms.");
LOG.info("Retrying connect to namenode: {}. Already tried {}"
+ " time(s); retry policy is {}, delay {}ms.", nnAddr, retry,
retryPolicy, a.delayMillis);
if (isFailoverAndRetry) {
resetStateToFailOver();
@ -757,7 +758,7 @@ final T getResponse(HttpURLConnection conn) throws IOException {
final IOException ioe =
new IOException("Response decoding failure: "+e.toString(), e);
if (LOG.isDebugEnabled()) {
LOG.debug(ioe);
LOG.debug("Response decoding failure: {}", e.toString(), e);
}
throw ioe;
} finally {
@ -1212,7 +1213,7 @@ public synchronized void close() throws IOException {
}
} catch (IOException ioe) {
if (LOG.isDebugEnabled()) {
LOG.debug("Token cancel failed: " + ioe);
LOG.debug("Token cancel failed: ", ioe);
}
} finally {
super.close();

View File

@ -658,6 +658,9 @@ Release 2.8.0 - UNRELEASED
HDFS-8192. Eviction should key off used locked memory instead of
ram disk free space. (Arpit Agarwal)
HDFS-6564. Use slf4j instead of common-logging in hdfs-client.
(Rakesh R via wheat9)
OPTIMIZATIONS
HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than