@InterfaceAudience.Public @InterfaceStability.Unstable public abstract class LogAggregationFileController extends Object
Modifier and Type | Field and Description |
---|---|
protected static FsPermission |
APP_DIR_PERMISSIONS
Permissions for the Application directory.
|
protected static FsPermission |
APP_LOG_FILE_UMASK
Umask for the log file.
|
protected Configuration |
conf |
protected String |
fileControllerName |
protected boolean |
fsSupportsChmod |
protected Path |
remoteRootLogDir |
protected String |
remoteRootLogDirSuffix |
protected int |
retentionSize |
protected static FsPermission |
TLDIR_PERMISSIONS
Permissions for the top level directory under which app directories will be
created.
|
Constructor and Description |
---|
LogAggregationFileController() |
Modifier and Type | Method and Description |
---|---|
protected String |
aggregatedLogSuffix(String fileName)
Create the aggregated log suffix.
|
protected boolean |
belongsToAppAttempt(ApplicationAttemptId appAttemptId,
String containerIdStr) |
protected boolean |
checkExists(FileSystem fs,
Path path,
FsPermission fsPerm) |
protected void |
cleanOldLogs(Path remoteNodeLogFileForApp,
NodeId nodeId,
UserGroupInformation userUgi) |
protected void |
closePrintStream(OutputStream out) |
abstract void |
closeWriter()
Close the writer.
|
void |
createAppDir(String user,
ApplicationId appId,
UserGroupInformation userUgi)
Create remote Application directory for log aggregation.
|
protected void |
createDir(FileSystem fs,
Path path,
FsPermission fsPerm) |
abstract Map<ApplicationAccessType,String> |
getApplicationAcls(Path aggregatedLogPath,
ApplicationId appId)
Returns ACLs for the application.
|
org.apache.hadoop.fs.RemoteIterator<FileStatus> |
getApplicationDirectoriesOfUser(String user)
Gets all application directories of a user.
|
abstract String |
getApplicationOwner(Path aggregatedLogPath,
ApplicationId appId)
Returns the owner of the application.
|
String |
getFileControllerName()
Get the name of the file controller.
|
protected FileSystem |
getFileSystem(Configuration conf) |
Map<String,List<org.apache.hadoop.yarn.logaggregation.ContainerLogFileInfo>> |
getLogMetaFilesOfNode(org.apache.hadoop.yarn.logaggregation.ExtendedLogMetaRequest logRequest,
FileStatus currentNodeFile,
ApplicationId appId)
Returns log file metadata for a node grouped by containers.
|
org.apache.hadoop.fs.RemoteIterator<FileStatus> |
getNodeFilesOfApplicationDirectory(FileStatus appDir)
Gets all node files in an application directory.
|
Path |
getOlderRemoteAppLogDir(ApplicationId appId,
String appOwner)
Get the older remote application directory for log aggregation.
|
Path |
getRemoteAppLogDir(ApplicationId appId,
String appOwner)
Get the remote application directory for log aggregation.
|
Path |
getRemoteNodeLogFileForApp(ApplicationId appId,
String user,
NodeId nodeId)
Get the remote aggregated log path.
|
Path |
getRemoteRootLogDir()
Get the remote root log directory.
|
String |
getRemoteRootLogDirSuffix()
Get the log aggregation directory suffix.
|
void |
initialize(Configuration conf,
String controllerName)
Initialize the log file controller.
|
abstract void |
initializeWriter(org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileControllerContext context)
Initialize the writer.
|
protected abstract void |
initInternal(Configuration conf)
Derived classes initialize themselves using this method.
|
boolean |
isFsSupportsChmod() |
abstract void |
postWrite(org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileControllerContext record)
Operations needed after write the log content.
|
abstract boolean |
readAggregatedLogs(org.apache.hadoop.yarn.logaggregation.ContainerLogsRequest logRequest,
OutputStream os)
Output container log.
|
abstract List<org.apache.hadoop.yarn.logaggregation.ContainerLogMeta> |
readAggregatedLogsMeta(org.apache.hadoop.yarn.logaggregation.ContainerLogsRequest logRequest)
Return a list of
ContainerLogMeta for an application
from Remote FileSystem. |
abstract void |
renderAggregatedLogsBlock(org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block html,
org.apache.hadoop.yarn.webapp.View.ViewContext context)
Render Aggregated Logs block.
|
void |
verifyAndCreateRemoteLogDir()
Verify and create the remote log directory.
|
abstract void |
write(AggregatedLogFormat.LogKey logKey,
org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogValue logValue)
Write the log content.
|
protected static final FsPermission TLDIR_PERMISSIONS
protected static final FsPermission APP_DIR_PERMISSIONS
protected static final FsPermission APP_LOG_FILE_UMASK
protected Configuration conf
protected Path remoteRootLogDir
protected String remoteRootLogDirSuffix
protected int retentionSize
protected String fileControllerName
protected boolean fsSupportsChmod
public void initialize(Configuration conf, String controllerName)
conf
- the ConfigurationcontrollerName
- the log controller class nameprotected abstract void initInternal(Configuration conf)
conf
- the Configurationpublic Path getRemoteRootLogDir()
public String getRemoteRootLogDirSuffix()
public String getFileControllerName()
public abstract void initializeWriter(org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileControllerContext context) throws IOException
context
- the LogAggregationFileControllerContext
IOException
- if fails to initialize the writerpublic abstract void closeWriter() throws org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationDFSException
org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationDFSException
- if the closing of the writer fails
(for example due to HDFS quota being exceeded)public abstract void write(AggregatedLogFormat.LogKey logKey, org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogValue logValue) throws IOException
logKey
- the log keylogValue
- the log contentIOException
- if fails to write the logspublic abstract void postWrite(org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileControllerContext record) throws Exception
record
- the LogAggregationFileControllerContext
Exception
- if anything failsprotected void closePrintStream(OutputStream out)
public abstract boolean readAggregatedLogs(org.apache.hadoop.yarn.logaggregation.ContainerLogsRequest logRequest, OutputStream os) throws IOException
logRequest
- ContainerLogsRequest
os
- the output streamIOException
- if we can not access the log file.public abstract List<org.apache.hadoop.yarn.logaggregation.ContainerLogMeta> readAggregatedLogsMeta(org.apache.hadoop.yarn.logaggregation.ContainerLogsRequest logRequest) throws IOException
ContainerLogMeta
for an application
from Remote FileSystem.logRequest
- ContainerLogsRequest
ContainerLogMeta
IOException
- if there is no available log filepublic Map<String,List<org.apache.hadoop.yarn.logaggregation.ContainerLogFileInfo>> getLogMetaFilesOfNode(org.apache.hadoop.yarn.logaggregation.ExtendedLogMetaRequest logRequest, FileStatus currentNodeFile, ApplicationId appId) throws IOException
logRequest
- extended query information holdercurrentNodeFile
- file status of a node in an application directoryappId
- id of the application, which is the same as in node pathIOException
- if there is no node filepublic org.apache.hadoop.fs.RemoteIterator<FileStatus> getApplicationDirectoriesOfUser(String user) throws IOException
user
- name of the userIOException
- if user directory does not existpublic org.apache.hadoop.fs.RemoteIterator<FileStatus> getNodeFilesOfApplicationDirectory(FileStatus appDir) throws IOException
appDir
- application directoryIOException
- if file context is not reachablepublic abstract void renderAggregatedLogsBlock(org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block html, org.apache.hadoop.yarn.webapp.View.ViewContext context)
html
- the htmlcontext
- the ViewContextpublic abstract String getApplicationOwner(Path aggregatedLogPath, ApplicationId appId) throws IOException
aggregatedLogPath
- the aggregatedLog pathappId
- the ApplicationIdIOException
- if we can not get the application ownerpublic abstract Map<ApplicationAccessType,String> getApplicationAcls(Path aggregatedLogPath, ApplicationId appId) throws IOException
aggregatedLogPath
- the aggregatedLog path.appId
- the ApplicationIdIOException
- if we can not get the application aclspublic void verifyAndCreateRemoteLogDir()
public void createAppDir(String user, ApplicationId appId, UserGroupInformation userUgi)
user
- the userappId
- the application IDuserUgi
- the UGI@VisibleForTesting protected FileSystem getFileSystem(Configuration conf) throws IOException
IOException
protected void createDir(FileSystem fs, Path path, FsPermission fsPerm) throws IOException
IOException
protected boolean checkExists(FileSystem fs, Path path, FsPermission fsPerm) throws IOException
IOException
public Path getRemoteNodeLogFileForApp(ApplicationId appId, String user, NodeId nodeId)
appId
- the ApplicationIduser
- the Application OwnernodeId
- the NodeManager Idpublic Path getRemoteAppLogDir(ApplicationId appId, String appOwner) throws IOException
appId
- the Application IDappOwner
- the Application OwnerIOException
- if can not find the remote application directorypublic Path getOlderRemoteAppLogDir(ApplicationId appId, String appOwner) throws IOException
appId
- the Application IDappOwner
- the Application OwnerIOException
- if can not find the remote application directoryprotected void cleanOldLogs(Path remoteNodeLogFileForApp, NodeId nodeId, UserGroupInformation userUgi)
protected String aggregatedLogSuffix(String fileName)
fileName
- the File Namepublic boolean isFsSupportsChmod()
protected boolean belongsToAppAttempt(ApplicationAttemptId appAttemptId, String containerIdStr)
Copyright © 2024 Apache Software Foundation. All rights reserved.