Skip to content

Commit

Permalink
RANGER-5151: fix for error while writing audit logs to HDFS (#535)
Browse files Browse the repository at this point in the history
(cherry picked from commit bcf39e2)
  • Loading branch information
mneethiraj committed Feb 20, 2025
1 parent 5353956 commit 648a09c
Show file tree
Hide file tree
Showing 2 changed files with 41 additions and 34 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -225,23 +225,10 @@ public void closeFileIfNeeded() {
logWriter.flush();
closeWriter();
resetWriter();
setNextRollOverTime();

currentFileName = null;
reUseLastLogFile = false;

if (!rollOverByDuration) {
try {
if(StringUtils.isEmpty(rolloverPeriod) ) {
rolloverPeriod = rollingTimeUtil.convertRolloverSecondsToRolloverPeriod(fileRolloverSec);
}
nextRollOverTime = rollingTimeUtil.computeNextRollingTime(rolloverPeriod);
} catch ( Exception e) {
logger.warn("Rollover by file.rollover.period failed", e);
logger.warn("Using the file.rollover.sec for {} audit file rollover...", fileSystemScheme);
nextRollOverTime = rollOverByDuration();
}
} else {
nextRollOverTime = rollOverByDuration();
}
}

if (logger.isDebugEnabled()) {
Expand All @@ -262,7 +249,7 @@ public PrintWriter createWriter() throws Exception {
if (logWriter == null) {
boolean appendMode = false;
// if append is supported, reuse last log file
if (reUseLastLogFile && fileSystem.hasPathCapability(auditPath, CommonPathCapabilities.FS_APPEND)) {
if (reUseLastLogFile && isAppendEnabled()) {
logger.info("Appending to last log file. auditPath = {}", fullPath);
try {
ostream = fileSystem.append(auditPath);
Expand Down Expand Up @@ -389,4 +376,33 @@ public String getFileSystemScheme() {
public void setFileExtension(String fileExtension) {
this.fileExtension = fileExtension;
}

private void setNextRollOverTime() {
if (!rollOverByDuration) {
try {
if (StringUtils.isEmpty(rolloverPeriod)) {
rolloverPeriod = rollingTimeUtil.convertRolloverSecondsToRolloverPeriod(fileRolloverSec);
}

nextRollOverTime = rollingTimeUtil.computeNextRollingTime(rolloverPeriod);
} catch (Exception e) {
logger.warn("Rollover by file.rollover.period failed", e);
logger.warn("Using the file.rollover.sec for {} audit file rollover...", fileSystemScheme);

nextRollOverTime = rollOverByDuration();
}
} else {
nextRollOverTime = rollOverByDuration();
}
}

private boolean isAppendEnabled() {
try {
return fileSystem.hasPathCapability(auditPath, CommonPathCapabilities.FS_APPEND);
} catch (Throwable t) {
logger.warn("Failed to check if audit log file {} can be appended. Will create a new file.", auditPath, t);
}

return false;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,29 +31,23 @@
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.Collection;
import java.util.Collections;
import java.util.Map;
import java.util.Properties;

/**
* Writes the Ranger audit to HDFS as JSON text
*/
public class RangerJSONAuditWriter extends AbstractRangerAuditWriter {

private static final Logger logger = LoggerFactory.getLogger(RangerJSONAuditWriter.class);

public static final String PROP_HDFS_ROLLOVER_ENABLE_PERIODIC_ROLLOVER = "file.rollover.enable.periodic.rollover";
public static final String PROP_HDFS_ROLLOVER_PERIODIC_ROLLOVER_CHECK_TIME = "file.rollover.periodic.rollover.check.sec";

protected String JSON_FILE_EXTENSION = ".log";

/*
* When enableAuditFilePeriodicRollOver is enabled, Audit File in HDFS would be closed by the defined period in
* xasecure.audit.destination.hdfs.file.rollover.sec. By default xasecure.audit.destination.hdfs.file.rollover.sec = 86400 sec
* and file will be closed midnight. Custom rollover time can be set by defining file.rollover.sec to desire time in seconds.
*/
private boolean enableAuditFilePeriodicRollOver = false;

/*
Time frequency of next occurrence of periodic rollover check. By Default every 60 seconds the check is done.
Time frequency of next occurrence of periodic rollover check. By Default every 60 seconds the check is done if enabled
*/
private long periodicRollOverCheckTimeinSec;

Expand All @@ -65,7 +59,7 @@ public void init(Properties props, String propPrefix, String auditProviderName,
super.init(props, propPrefix, auditProviderName, auditConfigs);

// start AuditFilePeriodicRollOverTask if enabled.
enableAuditFilePeriodicRollOver = MiscUtil.getBooleanProperty(props, propPrefix + "." + PROP_HDFS_ROLLOVER_ENABLE_PERIODIC_ROLLOVER, false);
boolean enableAuditFilePeriodicRollOver = MiscUtil.getBooleanProperty(props, propPrefix + "." + PROP_HDFS_ROLLOVER_ENABLE_PERIODIC_ROLLOVER, false);
if (enableAuditFilePeriodicRollOver) {
periodicRollOverCheckTimeinSec = MiscUtil.getLongProperty(props, propPrefix + "." + PROP_HDFS_ROLLOVER_PERIODIC_ROLLOVER_CHECK_TIME, 60L);
try {
Expand Down Expand Up @@ -152,6 +146,7 @@ public Boolean run() throws Exception {
});
ret = retVal.booleanValue();
logger.info("Flushing HDFS audit File :" + file.getAbsolutePath() + file.getName());

return ret;
}

Expand All @@ -161,17 +156,13 @@ public boolean logFile(File file) throws Exception {
}

synchronized public PrintWriter getLogFileStream() throws Exception {
if (!enableAuditFilePeriodicRollOver) {
// when periodic rollover is enabled closing of file is done by the file rollover monitoring task and hence don't need to
// close the file inline with audit logging.
closeFileIfNeeded();
}
closeFileIfNeeded();

// Either there are no open log file or the previous one has been rolled over
PrintWriter logWriter = createWriter();
return logWriter;
}


public void flush() {
if (logger.isDebugEnabled()) {
logger.debug("==> JSONWriter.flush() called. name=" + auditProviderName);
Expand Down Expand Up @@ -233,7 +224,7 @@ public void run() {
logger.debug("==> AuditFilePeriodicRollOverTask.run()");
}
try {
closeFileIfNeeded();
logJSON(Collections.emptyList());
} catch (Exception excp) {
logger.error("AuditFilePeriodicRollOverTask Failed. Aborting..", excp);
}
Expand All @@ -243,4 +234,4 @@ public void run() {
}
}

}
}

0 comments on commit 648a09c

Please sign in to comment.