|
@@ -20,6 +20,10 @@ package org.apache.hadoop.tools.dynamometer.workloadgenerator.audit;
|
|
|
import com.google.common.collect.Lists;
|
|
|
import java.util.Optional;
|
|
|
import java.util.function.Function;
|
|
|
+
|
|
|
+import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.mapreduce.Job;
|
|
|
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
|
|
|
import org.apache.hadoop.tools.dynamometer.workloadgenerator.WorkloadDriver;
|
|
|
import org.apache.hadoop.tools.dynamometer.workloadgenerator.WorkloadMapper;
|
|
|
import java.io.IOException;
|
|
@@ -35,7 +39,6 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
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;
|
|
@@ -73,9 +76,11 @@ import static org.apache.hadoop.tools.dynamometer.workloadgenerator.audit.AuditR
|
|
|
* are replayed. For example, a rate factor of 2 would make the replay occur
|
|
|
* twice as fast, and a rate factor of 0.5 would make it occur half as fast.
|
|
|
*/
|
|
|
-public class AuditReplayMapper extends WorkloadMapper<LongWritable, Text> {
|
|
|
+public class AuditReplayMapper extends WorkloadMapper<LongWritable, Text,
|
|
|
+ UserCommandKey, CountTimeWritable> {
|
|
|
|
|
|
public static final String INPUT_PATH_KEY = "auditreplay.input-path";
|
|
|
+ public static final String OUTPUT_PATH_KEY = "auditreplay.output-path";
|
|
|
public static final String NUM_THREADS_KEY = "auditreplay.num-threads";
|
|
|
public static final int NUM_THREADS_DEFAULT = 1;
|
|
|
public static final String CREATE_BLOCKS_KEY = "auditreplay.create-blocks";
|
|
@@ -170,11 +175,6 @@ public class AuditReplayMapper extends WorkloadMapper<LongWritable, Text> {
|
|
|
private AuditCommandParser commandParser;
|
|
|
private ScheduledThreadPoolExecutor progressExecutor;
|
|
|
|
|
|
- @Override
|
|
|
- public Class<? extends InputFormat> getInputFormat(Configuration conf) {
|
|
|
- return NoSplitTextInputFormat.class;
|
|
|
- }
|
|
|
-
|
|
|
@Override
|
|
|
public String getDescription() {
|
|
|
return "This mapper replays audit log files.";
|
|
@@ -185,6 +185,7 @@ public class AuditReplayMapper extends WorkloadMapper<LongWritable, Text> {
|
|
|
return Lists.newArrayList(
|
|
|
INPUT_PATH_KEY
|
|
|
+ " (required): Path to directory containing input files.",
|
|
|
+ OUTPUT_PATH_KEY + " (required): Path to destination for output files.",
|
|
|
NUM_THREADS_KEY + " (default " + NUM_THREADS_DEFAULT
|
|
|
+ "): Number of threads to use per mapper for replay.",
|
|
|
CREATE_BLOCKS_KEY + " (default " + CREATE_BLOCKS_DEFAULT
|
|
@@ -199,7 +200,8 @@ public class AuditReplayMapper extends WorkloadMapper<LongWritable, Text> {
|
|
|
|
|
|
@Override
|
|
|
public boolean verifyConfigurations(Configuration conf) {
|
|
|
- return conf.get(INPUT_PATH_KEY) != null;
|
|
|
+ return conf.get(INPUT_PATH_KEY) != null
|
|
|
+ && conf.get(OUTPUT_PATH_KEY) != null;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -256,7 +258,8 @@ public class AuditReplayMapper extends WorkloadMapper<LongWritable, Text> {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void cleanup(Mapper.Context context) throws InterruptedException {
|
|
|
+ public void cleanup(Mapper.Context context)
|
|
|
+ throws InterruptedException, IOException {
|
|
|
for (AuditReplayThread t : threads) {
|
|
|
// Add in an indicator for each thread to shut down after the last real
|
|
|
// command
|
|
@@ -266,6 +269,7 @@ public class AuditReplayMapper extends WorkloadMapper<LongWritable, Text> {
|
|
|
for (AuditReplayThread t : threads) {
|
|
|
t.join();
|
|
|
t.drainCounters(context);
|
|
|
+ t.drainCommandLatencies(context);
|
|
|
if (t.getException() != null) {
|
|
|
threadException = Optional.of(t.getException());
|
|
|
}
|
|
@@ -287,4 +291,21 @@ public class AuditReplayMapper extends WorkloadMapper<LongWritable, Text> {
|
|
|
LOG.info("Percentage of invalid ops: " + percentageOfInvalidOps);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void configureJob(Job job) {
|
|
|
+ job.setMapOutputKeyClass(UserCommandKey.class);
|
|
|
+ job.setMapOutputValueClass(CountTimeWritable.class);
|
|
|
+ job.setInputFormatClass(NoSplitTextInputFormat.class);
|
|
|
+
|
|
|
+ job.setNumReduceTasks(1);
|
|
|
+ job.setReducerClass(AuditReplayReducer.class);
|
|
|
+ job.setOutputKeyClass(UserCommandKey.class);
|
|
|
+ job.setOutputValueClass(CountTimeWritable.class);
|
|
|
+ job.setOutputFormatClass(TextOutputFormat.class);
|
|
|
+
|
|
|
+ TextOutputFormat.setOutputPath(job, new Path(
|
|
|
+ job.getConfiguration().get(OUTPUT_PATH_KEY)));
|
|
|
+ job.getConfiguration().set(TextOutputFormat.SEPARATOR, ",");
|
|
|
+ }
|
|
|
}
|