Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closes #20. Add a progress updating thread to the audit workload replay #23

Merged
merged 1 commit into from
Apr 6, 2018
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.DelayQueue;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
Expand All @@ -24,6 +25,7 @@
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.InputFormat;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;

Expand Down Expand Up @@ -137,6 +139,7 @@ public enum CommandType {
private DelayQueue<AuditReplayCommand> commandQueue;
private Function<Long, Long> relativeToAbsoluteTimestamp;
private AuditCommandParser commandParser;
private ScheduledThreadPoolExecutor progressExecutor;

@Override
public Class<? extends InputFormat> getInputFormat(Configuration conf) {
Expand Down Expand Up @@ -167,7 +170,7 @@ public boolean verifyConfigurations(Configuration conf) {
}

@Override
public void setup(Mapper.Context context) throws IOException {
public void setup(final Mapper.Context context) throws IOException {
Configuration conf = context.getConfiguration();
// WorkloadDriver ensures that the starttimestamp is set
startTimestampMs = conf.getLong(WorkloadDriver.START_TIMESTAMP_MS, -1);
Expand All @@ -189,6 +192,16 @@ public Long apply(Long input) {

LOG.info("Starting " + numThreads + " threads");

progressExecutor = new ScheduledThreadPoolExecutor(1);
// half of the timeout or once per minute if none specified
long progressFrequencyMs = conf.getLong(MRJobConfig.TASK_TIMEOUT, 2 * 60 * 1000) / 2;
progressExecutor.scheduleAtFixedRate(new Runnable() {
@Override
public void run() {
context.progress();
}
}, progressFrequencyMs, progressFrequencyMs, TimeUnit.MILLISECONDS);

threads = new ArrayList<>();
ConcurrentMap<String, FileSystem> fsCache = new ConcurrentHashMap<>();
commandQueue = new DelayQueue<>();
Expand Down Expand Up @@ -226,6 +239,7 @@ public void cleanup(Mapper.Context context) throws InterruptedException {
threadException = Optional.of(t.getException());
}
}
progressExecutor.shutdown();

if (threadException.isPresent()) {
throw new RuntimeException("Exception in AuditReplayThread", threadException.get());
Expand Down