|
@@ -55,11 +55,11 @@ import java.util.Set;
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
import java.util.concurrent.ConcurrentMap;
|
|
import java.util.concurrent.ConcurrentMap;
|
|
|
|
|
|
-import org.junit.Assert;
|
|
|
|
import org.apache.commons.lang.StringUtils;
|
|
import org.apache.commons.lang.StringUtils;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.fs.AbstractFileSystem;
|
|
import org.apache.hadoop.fs.AbstractFileSystem;
|
|
|
|
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
import org.apache.hadoop.fs.FileContext;
|
|
import org.apache.hadoop.fs.FileContext;
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
@@ -67,8 +67,11 @@ import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.RemoteIterator;
|
|
import org.apache.hadoop.fs.RemoteIterator;
|
|
import org.apache.hadoop.fs.UnsupportedFileSystemException;
|
|
import org.apache.hadoop.fs.UnsupportedFileSystemException;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
|
+import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.security.Credentials;
|
|
import org.apache.hadoop.security.Credentials;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
|
+import org.apache.hadoop.security.token.Token;
|
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
|
import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
|
@@ -93,10 +96,11 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat;
|
|
import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat;
|
|
-import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils;
|
|
|
|
import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey;
|
|
import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey;
|
|
import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogReader;
|
|
import org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogReader;
|
|
import org.apache.hadoop.yarn.logaggregation.ContainerLogsRetentionPolicy;
|
|
import org.apache.hadoop.yarn.logaggregation.ContainerLogsRetentionPolicy;
|
|
|
|
+import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils;
|
|
|
|
+import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
|
|
import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
|
|
import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
|
|
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
|
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
|
import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
|
|
import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
|
|
@@ -107,19 +111,22 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestContainerM
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
|
|
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.TestNonAggregatingLogHandler;
|
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.TestNonAggregatingLogHandler;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppFinishedEvent;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppFinishedEvent;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppStartedEvent;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppStartedEvent;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerContainerFinishedEvent;
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerContainerFinishedEvent;
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
import org.apache.hadoop.yarn.util.Records;
|
|
import org.apache.hadoop.yarn.util.Records;
|
|
|
|
+import org.junit.Assert;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
import org.mockito.ArgumentCaptor;
|
|
import org.mockito.ArgumentCaptor;
|
|
import org.mockito.Mockito;
|
|
import org.mockito.Mockito;
|
|
import org.mortbay.util.MultiException;
|
|
import org.mortbay.util.MultiException;
|
|
|
|
|
|
|
|
+import com.google.common.base.Supplier;
|
|
|
|
+
|
|
//@Ignore
|
|
//@Ignore
|
|
public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
|
|
|
|
@@ -152,6 +159,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
dispatcher = createDispatcher();
|
|
dispatcher = createDispatcher();
|
|
appEventHandler = mock(EventHandler.class);
|
|
appEventHandler = mock(EventHandler.class);
|
|
dispatcher.register(ApplicationEventType.class, appEventHandler);
|
|
dispatcher.register(ApplicationEventType.class, appEventHandler);
|
|
|
|
+ UserGroupInformation.setConfiguration(conf);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -1424,6 +1432,64 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
|
|
dispatcher.stop();
|
|
dispatcher.stop();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+
|
|
|
|
+ @Test (timeout = 20000)
|
|
|
|
+ public void testAddNewTokenSentFromRMForLogAggregation() throws Exception {
|
|
|
|
+ Configuration conf = new YarnConfiguration();
|
|
|
|
+ conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
|
|
|
|
+ "kerberos");
|
|
|
|
+ UserGroupInformation.setConfiguration(conf);
|
|
|
|
+ DrainDispatcher dispatcher = createDispatcher();
|
|
|
|
+ dispatcher.register(ApplicationEventType.class, appEventHandler);
|
|
|
|
+
|
|
|
|
+ ApplicationId application1 = BuilderUtils.newApplicationId(1234, 1);
|
|
|
|
+ Application mockApp = mock(Application.class);
|
|
|
|
+ when(mockApp.getContainers()).thenReturn(
|
|
|
|
+ new HashMap<ContainerId, Container>());
|
|
|
|
+ this.context.getApplications().put(application1, mockApp);
|
|
|
|
+ @SuppressWarnings("resource")
|
|
|
|
+ LogAggregationService logAggregationService =
|
|
|
|
+ new LogAggregationService(dispatcher, this.context, this.delSrvc,
|
|
|
|
+ super.dirsHandler);
|
|
|
|
+ logAggregationService.init(this.conf);
|
|
|
|
+ logAggregationService.start();
|
|
|
|
+ logAggregationService.handle(new LogHandlerAppStartedEvent(application1,
|
|
|
|
+ this.user, null, ContainerLogsRetentionPolicy.ALL_CONTAINERS, this.acls,
|
|
|
|
+ Records.newRecord(LogAggregationContext.class)));
|
|
|
|
+
|
|
|
|
+ // Inject new token for log-aggregation after app log-aggregator init
|
|
|
|
+ Text userText1 = new Text("user1");
|
|
|
|
+ RMDelegationTokenIdentifier dtId1 =
|
|
|
|
+ new RMDelegationTokenIdentifier(userText1, new Text("renewer1"),
|
|
|
|
+ userText1);
|
|
|
|
+ final Token<RMDelegationTokenIdentifier> token1 =
|
|
|
|
+ new Token<RMDelegationTokenIdentifier>(dtId1.getBytes(),
|
|
|
|
+ "password1".getBytes(), dtId1.getKind(), new Text("service1"));
|
|
|
|
+ Credentials credentials = new Credentials();
|
|
|
|
+ credentials.addToken(userText1, token1);
|
|
|
|
+ this.context.getSystemCredentialsForApps().put(application1, credentials);
|
|
|
|
+
|
|
|
|
+ logAggregationService.handle(new LogHandlerAppFinishedEvent(application1));
|
|
|
|
+
|
|
|
|
+ final UserGroupInformation ugi =
|
|
|
|
+ ((AppLogAggregatorImpl) logAggregationService.getAppLogAggregators()
|
|
|
|
+ .get(application1)).getUgi();
|
|
|
|
+
|
|
|
|
+ GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
|
+ public Boolean get() {
|
|
|
|
+ boolean hasNewToken = false;
|
|
|
|
+ for (Token<?> token : ugi.getCredentials().getAllTokens()) {
|
|
|
|
+ if (token.equals(token1)) {
|
|
|
|
+ hasNewToken = true;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return hasNewToken;
|
|
|
|
+ }
|
|
|
|
+ }, 1000, 20000);
|
|
|
|
+ logAggregationService.stop();
|
|
|
|
+ dispatcher.stop();
|
|
|
|
+ }
|
|
|
|
+
|
|
private int numOfLogsAvailable(LogAggregationService logAggregationService,
|
|
private int numOfLogsAvailable(LogAggregationService logAggregationService,
|
|
ApplicationId appId, boolean sizeLimited, String lastLogFile)
|
|
ApplicationId appId, boolean sizeLimited, String lastLogFile)
|
|
throws IOException {
|
|
throws IOException {
|