Merge from trunk to branch

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/fs-encryption@1619194 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Andrew Wang 2014-08-20 18:19:13 +00:00
commit 834fccf02b
10 changed files with 309 additions and 24 deletions

View File

@ -520,6 +520,12 @@ Release 2.6.0 - UNRELEASED
HDFS-6569. OOB message can't be sent to the client when DataNode shuts down for upgrade
(brandonli)
HDFS-6868. portmap and nfs3 are documented as hadoop commands instead of hdfs
(brandonli)
HDFS-6870. Blocks and INodes could leak for Rename with overwrite flag. (Yi
Liu via jing9)
Release 2.5.0 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -666,15 +666,20 @@ public class FSDirectory implements Closeable {
tx.updateMtimeAndLease(timestamp);
// Collect the blocks and remove the lease for previous dst
long filesDeleted = -1;
boolean filesDeleted = false;
if (removedDst != null) {
undoRemoveDst = false;
if (removedNum > 0) {
BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
List<INode> removedINodes = new ChunkedArrayList<INode>();
filesDeleted = removedDst.cleanSubtree(Snapshot.CURRENT_STATE_ID,
dstIIP.getLatestSnapshotId(), collectedBlocks, removedINodes,
true).get(Quota.NAMESPACE);
if (!removedDst.isInLatestSnapshot(dstIIP.getLatestSnapshotId())) {
removedDst.destroyAndCollectBlocks(collectedBlocks, removedINodes);
filesDeleted = true;
} else {
filesDeleted = removedDst.cleanSubtree(Snapshot.CURRENT_STATE_ID,
dstIIP.getLatestSnapshotId(), collectedBlocks, removedINodes,
true).get(Quota.NAMESPACE) >= 0;
}
getFSNamesystem().removePathAndBlocks(src, collectedBlocks,
removedINodes, false);
}
@ -687,7 +692,7 @@ public class FSDirectory implements Closeable {
}
tx.updateQuotasInSourceTree();
return filesDeleted >= 0;
return filesDeleted;
}
} finally {
if (undoRemoveSrc) {

View File

@ -209,7 +209,7 @@ HDFS NFS Gateway
[[2]] Start package included portmap (needs root privileges):
-------------------------
hadoop portmap
hdfs portmap
OR
@ -224,7 +224,7 @@ HDFS NFS Gateway
as long as the user has read access to the Kerberos keytab defined in "nfs.keytab.file".
-------------------------
hadoop nfs3
hdfs nfs3
OR

View File

@ -27,6 +27,9 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.Options.Rename;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.junit.Test;
@ -125,4 +128,45 @@ public class TestDFSRename {
if (cluster != null) {cluster.shutdown();}
}
}
/**
* Check the blocks of dst file are cleaned after rename with overwrite
*/
@Test(timeout = 120000)
public void testRenameWithOverwrite() throws Exception {
final short replFactor = 2;
final long blockSize = 512;
Configuration conf = new Configuration();
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).
numDataNodes(replFactor).build();
DistributedFileSystem dfs = cluster.getFileSystem();
try {
long fileLen = blockSize*3;
String src = "/foo/src";
String dst = "/foo/dst";
Path srcPath = new Path(src);
Path dstPath = new Path(dst);
DFSTestUtil.createFile(dfs, srcPath, fileLen, replFactor, 1);
DFSTestUtil.createFile(dfs, dstPath, fileLen, replFactor, 1);
LocatedBlocks lbs = NameNodeAdapter.getBlockLocations(
cluster.getNameNode(), dst, 0, fileLen);
BlockManager bm = NameNodeAdapter.getNamesystem(cluster.getNameNode()).
getBlockManager();
assertTrue(bm.getStoredBlock(lbs.getLocatedBlocks().get(0).getBlock().
getLocalBlock()) != null);
dfs.rename(srcPath, dstPath, Rename.OVERWRITE);
assertTrue(bm.getStoredBlock(lbs.getLocatedBlocks().get(0).getBlock().
getLocalBlock()) == null);
} finally {
if (dfs != null) {
dfs.close();
}
if (cluster != null) {
cluster.shutdown();
}
}
}
}

View File

@ -50,6 +50,9 @@ Release 2.6.0 - UNRELEASED
YARN-2411. Support simple user and group mappings to queues. (Ram Venkatesh
via jianhe)
YARN-2174. Enable HTTPs for the writer REST API of TimelineServer.
(Zhijie Shen via jianhe)
IMPROVEMENTS
YARN-2197. Add a link to YARN CHANGES.txt in the left side of doc
@ -217,6 +220,9 @@ Release 2.6.0 - UNRELEASED
YARN-2249. Avoided AM release requests being lost on work preserving RM
restart. (Jian He via zjshen)
YARN-2034. Description for yarn.nodemanager.localizer.cache.target-size-mb
is incorrect (Chen He via jlowe)
Release 2.5.0 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -606,7 +606,11 @@ public class YarnConfiguration extends Configuration {
public static final long DEFAULT_NM_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS =
10 * 60 * 1000;
/** Target size of localizer cache in MB, per local directory.*/
/**
* Target size of localizer cache in MB, per nodemanager. It is a target
* retention size that only includes resources with PUBLIC and PRIVATE
* visibility and excludes resources with APPLICATION visibility
*/
public static final String NM_LOCALIZER_CACHE_TARGET_SIZE_MB =
NM_PREFIX + "localizer.cache.target-size-mb";
public static final long DEFAULT_NM_LOCALIZER_CACHE_TARGET_SIZE_MB = 10 * 1024;

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
import org.apache.hadoop.security.authentication.client.AuthenticationException;
import org.apache.hadoop.security.authentication.client.Authenticator;
import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
import org.apache.hadoop.security.authentication.client.KerberosAuthenticator;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.yarn.api.records.timeline.TimelineDelegationTokenResponse;
@ -53,10 +54,13 @@ import com.google.common.annotations.VisibleForTesting;
public class TimelineAuthenticator extends KerberosAuthenticator {
private static ObjectMapper mapper;
private static TimelineAuthenticator authenticator;
private static ConnectionConfigurator connConfigurator;
static {
mapper = new ObjectMapper();
YarnJacksonJaxbJsonProvider.configObjectMapper(mapper);
authenticator = new TimelineAuthenticator();
}
/**
@ -98,6 +102,11 @@ public class TimelineAuthenticator extends KerberosAuthenticator {
}
}
public static void setStaticConnectionConfigurator(
ConnectionConfigurator connConfigurator) {
TimelineAuthenticator.connConfigurator = connConfigurator;
}
public static Token<TimelineDelegationTokenIdentifier> getDelegationToken(
URL url, AuthenticatedURL.Token token, String renewer) throws IOException {
TimelineDelegationTokenOperation op =
@ -107,7 +116,7 @@ public class TimelineAuthenticator extends KerberosAuthenticator {
params.put(TimelineAuthenticationConsts.RENEWER_PARAM, renewer);
url = appendParams(url, params);
AuthenticatedURL aUrl =
new AuthenticatedURL(new TimelineAuthenticator());
new AuthenticatedURL(authenticator, connConfigurator);
try {
HttpURLConnection conn = aUrl.openConnection(url, token);
conn.setRequestMethod(op.getHttpMethod());
@ -137,7 +146,7 @@ public class TimelineAuthenticator extends KerberosAuthenticator {
dToken.encodeToUrlString());
url = appendParams(url, params);
AuthenticatedURL aUrl =
new AuthenticatedURL(new TimelineAuthenticator());
new AuthenticatedURL(authenticator, connConfigurator);
try {
HttpURLConnection conn = aUrl.openConnection(url, token);
conn.setRequestMethod(
@ -164,7 +173,7 @@ public class TimelineAuthenticator extends KerberosAuthenticator {
dToken.encodeToUrlString());
url = appendParams(url, params);
AuthenticatedURL aUrl =
new AuthenticatedURL(new TimelineAuthenticator());
new AuthenticatedURL(authenticator, connConfigurator);
try {
HttpURLConnection conn = aUrl.openConnection(url, token);
conn.setRequestMethod(TimelineDelegationTokenOperation.CANCELDELEGATIONTOKEN

View File

@ -23,10 +23,15 @@ import java.io.IOException;
import java.net.HttpURLConnection;
import java.net.URI;
import java.net.URL;
import java.net.URLConnection;
import java.security.GeneralSecurityException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import javax.net.ssl.HostnameVerifier;
import javax.net.ssl.HttpsURLConnection;
import javax.net.ssl.SSLSocketFactory;
import javax.ws.rs.core.MediaType;
import org.apache.commons.cli.CommandLine;
@ -42,6 +47,8 @@ import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
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.apache.hadoop.security.token.Token;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
@ -74,7 +81,10 @@ public class TimelineClientImpl extends TimelineClient {
private static final String RESOURCE_URI_STR = "/ws/v1/timeline/";
private static final String URL_PARAM_USER_NAME = "user.name";
private static final Joiner JOINER = Joiner.on("");
public final static int DEFAULT_SOCKET_TIMEOUT = 1 * 60 * 1000; // 1 minute
private static Options opts;
static {
opts = new Options();
opts.addOption("put", true, "Put the TimelineEntities in a JSON file");
@ -89,15 +99,6 @@ public class TimelineClientImpl extends TimelineClient {
public TimelineClientImpl() {
super(TimelineClientImpl.class.getName());
ClientConfig cc = new DefaultClientConfig();
cc.getClasses().add(YarnJacksonJaxbJsonProvider.class);
if (UserGroupInformation.isSecurityEnabled()) {
urlFactory = new KerberosAuthenticatedURLConnectionFactory();
client = new Client(new URLConnectionClientHandler(urlFactory), cc);
} else {
client = new Client(new URLConnectionClientHandler(
new PseudoAuthenticatedURLConnectionFactory()), cc);
}
}
protected void serviceInit(Configuration conf) throws Exception {
@ -107,6 +108,17 @@ public class TimelineClientImpl extends TimelineClient {
if (!isEnabled) {
LOG.info("Timeline service is not enabled");
} else {
ClientConfig cc = new DefaultClientConfig();
cc.getClasses().add(YarnJacksonJaxbJsonProvider.class);
ConnectionConfigurator connConfigurator = newConnConfigurator(conf);
if (UserGroupInformation.isSecurityEnabled()) {
TimelineAuthenticator.setStaticConnectionConfigurator(connConfigurator);
urlFactory = new KerberosAuthenticatedURLConnectionFactory(connConfigurator);
client = new Client(new URLConnectionClientHandler(urlFactory), cc);
} else {
client = new Client(new URLConnectionClientHandler(
new PseudoAuthenticatedURLConnectionFactory(connConfigurator)), cc);
}
if (YarnConfiguration.useHttps(conf)) {
resURI = URI
.create(JOINER.join("https://", conf.get(
@ -182,6 +194,13 @@ public class TimelineClientImpl extends TimelineClient {
private static class PseudoAuthenticatedURLConnectionFactory
implements HttpURLConnectionFactory {
private ConnectionConfigurator connConfigurator;
public PseudoAuthenticatedURLConnectionFactory(
ConnectionConfigurator connConfigurator) {
this.connConfigurator = connConfigurator;
}
@Override
public HttpURLConnection getHttpURLConnection(URL url) throws IOException {
Map<String, String> params = new HashMap<String, String>();
@ -191,7 +210,7 @@ public class TimelineClientImpl extends TimelineClient {
if (LOG.isDebugEnabled()) {
LOG.debug("URL with delegation token: " + url);
}
return (HttpURLConnection) url.openConnection();
return connConfigurator.configure((HttpURLConnection) url.openConnection());
}
}
@ -202,10 +221,13 @@ public class TimelineClientImpl extends TimelineClient {
private TimelineAuthenticator authenticator;
private Token<TimelineDelegationTokenIdentifier> dToken;
private Text service;
private ConnectionConfigurator connConfigurator;
public KerberosAuthenticatedURLConnectionFactory() {
public KerberosAuthenticatedURLConnectionFactory(
ConnectionConfigurator connConfigurator) {
token = new AuthenticatedURL.Token();
authenticator = new TimelineAuthenticator();
this.connConfigurator = connConfigurator;
}
@Override
@ -226,7 +248,8 @@ public class TimelineClientImpl extends TimelineClient {
LOG.debug("URL with delegation token: " + url);
}
}
return new AuthenticatedURL(authenticator).openConnection(url, token);
return new AuthenticatedURL(
authenticator, connConfigurator).openConnection(url, token);
} catch (AuthenticationException e) {
LOG.error("Authentication failed when openning connection [" + url
+ "] with token [" + token + "].", e);
@ -255,6 +278,57 @@ public class TimelineClientImpl extends TimelineClient {
}
private static ConnectionConfigurator newConnConfigurator(Configuration conf) {
try {
return newSslConnConfigurator(DEFAULT_SOCKET_TIMEOUT, conf);
} catch (Exception e) {
LOG.debug("Cannot load customized ssl related configuration. " +
"Fallback to system-generic settings.", e);
return DEFAULT_TIMEOUT_CONN_CONFIGURATOR;
}
}
private static final ConnectionConfigurator DEFAULT_TIMEOUT_CONN_CONFIGURATOR =
new ConnectionConfigurator() {
@Override
public HttpURLConnection configure(HttpURLConnection conn)
throws IOException {
setTimeouts(conn, DEFAULT_SOCKET_TIMEOUT);
return conn;
}
};
private static ConnectionConfigurator newSslConnConfigurator(final int timeout,
Configuration conf) throws IOException, GeneralSecurityException {
final SSLFactory factory;
final SSLSocketFactory sf;
final HostnameVerifier hv;
factory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
factory.init();
sf = factory.createSSLSocketFactory();
hv = factory.getHostnameVerifier();
return new ConnectionConfigurator() {
@Override
public HttpURLConnection configure(HttpURLConnection conn)
throws IOException {
if (conn instanceof HttpsURLConnection) {
HttpsURLConnection c = (HttpsURLConnection) conn;
c.setSSLSocketFactory(sf);
c.setHostnameVerifier(hv);
}
setTimeouts(conn, timeout);
return conn;
}
};
}
private static void setTimeouts(URLConnection connection, int socketTimeout) {
connection.setConnectTimeout(socketTimeout);
connection.setReadTimeout(socketTimeout);
}
public static void main(String[] argv) throws Exception {
CommandLine cliParser = new GnuParser().parse(opts, argv);
if (cliParser.hasOption("put")) {

View File

@ -757,7 +757,10 @@
</property>
<property>
<description>Target size of localizer cache in MB, per local directory.</description>
<description>Target size of localizer cache in MB, per nodemanager. It is
a target retention size that only includes resources with PUBLIC and
PRIVATE visibility and excludes resources with APPLICATION visibility
</description>
<name>yarn.nodemanager.localizer.cache.target-size-mb</name>
<value>10240</value>
</property>

View File

@ -0,0 +1,134 @@
/**
* 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.yarn.server.timeline.webapp;
import java.io.File;
import java.util.EnumSet;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryServer;
import org.apache.hadoop.yarn.server.applicationhistoryservice.webapp.AHSWebApp;
import org.apache.hadoop.yarn.server.timeline.MemoryTimelineStore;
import org.apache.hadoop.yarn.server.timeline.TimelineReader.Field;
import org.apache.hadoop.yarn.server.timeline.TimelineStore;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import com.sun.jersey.api.client.ClientResponse;
public class TestTimelineWebServicesWithSSL {
private static final String BASEDIR =
System.getProperty("test.build.dir", "target/test-dir") + "/"
+ TestTimelineWebServicesWithSSL.class.getSimpleName();
private static String keystoresDir;
private static String sslConfDir;
private static ApplicationHistoryServer timelineServer;
private static TimelineStore store;
private static Configuration conf;
@BeforeClass
public static void setupServer() throws Exception {
conf = new YarnConfiguration();
conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STORE,
MemoryTimelineStore.class, TimelineStore.class);
conf.set(YarnConfiguration.YARN_HTTP_POLICY_KEY, "HTTPS_ONLY");
File base = new File(BASEDIR);
FileUtil.fullyDelete(base);
base.mkdirs();
keystoresDir = new File(BASEDIR).getAbsolutePath();
sslConfDir =
KeyStoreTestUtil.getClasspathDir(TestTimelineWebServicesWithSSL.class);
KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
conf.addResource("ssl-server.xml");
conf.addResource("ssl-client.xml");
timelineServer = new ApplicationHistoryServer();
timelineServer.init(conf);
timelineServer.start();
store = timelineServer.getTimelineStore();
}
@AfterClass
public static void tearDownServer() throws Exception {
if (timelineServer != null) {
timelineServer.stop();
}
AHSWebApp.resetInstance();
}
@Test
public void testPutEntities() throws Exception {
TestTimelineClient client = new TestTimelineClient();
try {
client.init(conf);
client.start();
TimelineEntity expectedEntity = new TimelineEntity();
expectedEntity.setEntityType("test entity type");
expectedEntity.setEntityId("test entity id");
TimelineEvent event = new TimelineEvent();
event.setEventType("test event type");
event.setTimestamp(0L);
expectedEntity.addEvent(event);
TimelinePutResponse response = client.putEntities(expectedEntity);
Assert.assertEquals(0, response.getErrors().size());
Assert.assertTrue(client.resp.toString().contains("https"));
TimelineEntity actualEntity = store.getEntity(
expectedEntity.getEntityId(), expectedEntity.getEntityType(),
EnumSet.allOf(Field.class));
Assert.assertNotNull(actualEntity);
Assert.assertEquals(
expectedEntity.getEntityId(), actualEntity.getEntityId());
Assert.assertEquals(
expectedEntity.getEntityType(), actualEntity.getEntityType());
} finally {
client.stop();
client.close();
}
}
private static class TestTimelineClient extends TimelineClientImpl {
private ClientResponse resp;
@Override
public ClientResponse doPostingEntities(TimelineEntities entities) {
resp = super.doPostingEntities(entities);
return resp;
}
}
}