package com.linkedin.camus.etl.kafka; import com.linkedin.camus.etl.kafka.common.DateUtils; import com.linkedin.camus.etl.kafka.common.EmailClient; import com.linkedin.camus.etl.kafka.common.EtlCounts; import com.linkedin.camus.etl.kafka.common.EtlKey; import com.linkedin.camus.etl.kafka.common.ExceptionWritable; import com.linkedin.camus.etl.kafka.common.Source; import com.linkedin.camus.etl.kafka.mapred.EtlInputFormat; import com.linkedin.camus.etl.kafka.mapred.EtlMapper; import com.linkedin.camus.etl.kafka.mapred.EtlMultiOutputFormat; import com.linkedin.camus.etl.kafka.mapred.EtlRecordReader; import com.linkedin.camus.etl.kafka.reporter.BaseReporter; import java.io.BufferedReader; import java.io.File; import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; import java.lang.reflect.InvocationTargetException; import java.net.URISyntaxException; import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Properties; import java.util.regex.Pattern; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; import org.apache.commons.cli.HelpFormatter; import org.apache.commons.cli.OptionBuilder; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; import org.apache.commons.math3.util.Pair; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.filecache.DistributedCache; import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.mapred.JobClient; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.TIPStatus; import org.apache.hadoop.mapred.TaskCompletionEvent; import org.apache.hadoop.mapred.TaskReport; import org.apache.hadoop.mapreduce.Counter; import org.apache.hadoop.mapreduce.CounterGroup; import org.apache.hadoop.mapreduce.Counters; import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.OutputFormat; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; import org.apache.log4j.Logger; import org.apache.log4j.xml.DOMConfigurator; import org.codehaus.jackson.map.DeserializationConfig; import org.codehaus.jackson.map.ObjectMapper; import org.codehaus.jackson.type.TypeReference; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.format.DateTimeFormatter; import com.google.common.collect.Lists; import com.google.common.collect.Maps; public class CamusJob extends Configured implements Tool { public static final String ETL_EXECUTION_BASE_PATH = "etl.execution.base.path"; public static final String ETL_EXECUTION_HISTORY_PATH = "etl.execution.history.path"; public static final String ETL_COUNTS_PATH = "etl.counts.path"; public static final String ETL_COUNTS_CLASS = "etl.counts.class"; public static final String ETL_COUNTS_CLASS_DEFAULT = "com.linkedin.camus.etl.kafka.common.EtlCounts"; public static final String ETL_KEEP_COUNT_FILES = "etl.keep.count.files"; public static final String ETL_BASEDIR_QUOTA_OVERIDE = "etl.basedir.quota.overide"; public static final String ETL_EXECUTION_HISTORY_MAX_OF_QUOTA = "etl.execution.history.max.of.quota"; public static final String ETL_FAIL_ON_ERRORS = "etl.fail.on.errors"; public static final String ETL_FAIL_ON_OFFSET_OUTOFRANGE = "etl.fail.on.offset.outofrange"; public static final String ETL_FAIL_ON_OFFSET_OUTOFRANGE_DEFAULT = Boolean.TRUE.toString(); public static final String ETL_MAX_PERCENT_SKIPPED_SCHEMANOTFOUND = "etl.max.percent.skipped.schemanotfound"; public static final String ETL_MAX_PERCENT_SKIPPED_SCHEMANOTFOUND_DEFAULT = "0.1"; public static final String ETL_MAX_PERCENT_SKIPPED_OTHER = "etl.max.percent.skipped.other"; public static final String ETL_MAX_PERCENT_SKIPPED_OTHER_DEFAULT = "0.1"; public static final String ETL_MAX_ERRORS_TO_PRINT_FROM_FILE = "etl.max.errors.to.print.from.file"; public static final String ETL_MAX_ERRORS_TO_PRINT_FROM_FILE_DEFAULT = "10"; public static final String ZK_AUDIT_HOSTS = "zookeeper.audit.hosts"; public static final String KAFKA_MONITOR_TIER = "kafka.monitor.tier"; public static final String CAMUS_MESSAGE_ENCODER_CLASS = "camus.message.encoder.class"; public static final String BROKER_URI_FILE = "brokers.uri"; public static final String POST_TRACKING_COUNTS_TO_KAFKA = "post.tracking.counts.to.kafka"; public static final String KAFKA_FETCH_REQUEST_MAX_WAIT = "kafka.fetch.request.max.wait"; public static final String KAFKA_FETCH_REQUEST_MIN_BYTES = "kafka.fetch.request.min.bytes"; public static final String KAFKA_FETCH_REQUEST_CORRELATION_ID = "kafka.fetch.request.correlationid"; public static final String KAFKA_CLIENT_NAME = "kafka.client.name"; public static final String KAFKA_FETCH_BUFFER_SIZE = "kafka.fetch.buffer.size"; public static final String KAFKA_BROKERS = "kafka.brokers"; public static final String KAFKA_HOST_URL = "kafka.host.url"; public static final String KAFKA_HOST_PORT = "kafka.host.port"; public static final String KAFKA_TIMEOUT_VALUE = "kafka.timeout.value"; public static final String CAMUS_REPORTER_CLASS = "etl.reporter.class"; public static final String LOG4J_CONFIGURATION = "log4j.configuration"; private static org.apache.log4j.Logger log; private Job hadoopJob = null; private final Properties props; private DateTimeFormatter dateFmt = DateUtils.getDateTimeFormatter("YYYY-MM-dd-HH-mm-ss", DateTimeZone.UTC); public CamusJob() throws IOException { this(new Properties()); } public CamusJob(Properties props) throws IOException { this(props, org.apache.log4j.Logger.getLogger(CamusJob.class)); } public CamusJob(Properties props, Logger log) throws IOException { this.props = props; this.log = log; } private static HashMap<String, Long> timingMap = new HashMap<String, Long>(); public static void startTiming(String name) { timingMap.put(name, (timingMap.get(name) == null ? 0 : timingMap.get(name)) - System.currentTimeMillis()); } public static void stopTiming(String name) { timingMap.put(name, (timingMap.get(name) == null ? 0 : timingMap.get(name)) + System.currentTimeMillis()); } public static void setTime(String name) { timingMap.put(name, (timingMap.get(name) == null ? 0 : timingMap.get(name)) + System.currentTimeMillis()); } private Job createJob(Properties props) throws IOException { Job job; if (getConf() == null) { setConf(new Configuration()); } populateConf(props, getConf(), log); job = new Job(getConf()); job.setJarByClass(CamusJob.class); if (job.getConfiguration().get("camus.job.name") != null) { job.setJobName(job.getConfiguration().get("camus.job.name")); } else { job.setJobName("Camus Job"); } if (System.getenv("HADOOP_TOKEN_FILE_LOCATION") != null) { job.getConfiguration().set("mapreduce.job.credentials.binary", System.getenv("HADOOP_TOKEN_FILE_LOCATION")); } this.hadoopJob = job; return job; } public static void populateConf(Properties props, Configuration conf, Logger log) throws IOException { for (Object key : props.keySet()) { conf.set(key.toString(), props.getProperty(key.toString())); } FileSystem fs = FileSystem.get(conf); String hadoopCacheJarDir = conf.get("hdfs.default.classpath.dir", null); List<Pattern> jarFilterString = new ArrayList<Pattern>(); for (String str : Arrays.asList(conf.getStrings("cache.jar.filter.list", new String[0]))) { jarFilterString.add(Pattern.compile(str)); } if (hadoopCacheJarDir != null) { FileStatus[] status = fs.listStatus(new Path(hadoopCacheJarDir)); if (status != null) { for (int i = 0; i < status.length; ++i) { if (!status[i].isDir()) { log.info("Adding Jar to Distributed Cache Archive File:" + status[i].getPath()); boolean filterMatch = false; for (Pattern p : jarFilterString) { if (p.matcher(status[i].getPath().getName()).matches()) { filterMatch = true; break; } } if (!filterMatch) DistributedCache.addFileToClassPath(status[i].getPath(), conf, fs); } } } else { System.out.println("hdfs.default.classpath.dir " + hadoopCacheJarDir + " is empty."); } } // Adds External jars to hadoop classpath String externalJarList = conf.get("hadoop.external.jarFiles", null); if (externalJarList != null) { String[] jarFiles = externalJarList.split(","); for (String jarFile : jarFiles) { log.info("Adding external jar File:" + jarFile); boolean filterMatch = false; for (Pattern p : jarFilterString) { if (p.matcher(new Path(jarFile).getName()).matches()) { filterMatch = true; break; } } if (!filterMatch) DistributedCache.addFileToClassPath(new Path(jarFile), conf, fs); } } } public void run() throws Exception { run(EtlInputFormat.class, EtlMultiOutputFormat.class); } public void run(Class<? extends InputFormat> inputFormatClass, Class<? extends OutputFormat> outputFormatClass) throws Exception { startTiming("pre-setup"); startTiming("total"); EmailClient.setup(props); Job job = createJob(props); if (getLog4jConfigure(job)) { DOMConfigurator.configure("log4j.xml"); } FileSystem fs = FileSystem.get(job.getConfiguration()); log.info("Dir Destination set to: " + EtlMultiOutputFormat.getDestinationPath(job)); Path execBasePath = new Path(props.getProperty(ETL_EXECUTION_BASE_PATH)); Path execHistory = new Path(props.getProperty(ETL_EXECUTION_HISTORY_PATH)); if (!fs.exists(execBasePath)) { log.info("The execution base path does not exist. Creating the directory"); fs.mkdirs(execBasePath); } if (!fs.exists(execHistory)) { log.info("The history base path does not exist. Creating the directory."); fs.mkdirs(execHistory); } // enforcing max retention on the execution directories to avoid // exceeding HDFS quota. retention is set to a percentage of available // quota. ContentSummary content = fs.getContentSummary(execBasePath); long limit = (long) (content.getQuota() * job.getConfiguration().getFloat(ETL_EXECUTION_HISTORY_MAX_OF_QUOTA, (float) .5)); limit = limit == 0 ? 50000 : limit; if (props.containsKey(ETL_BASEDIR_QUOTA_OVERIDE)) { limit = Long.valueOf(props.getProperty(ETL_BASEDIR_QUOTA_OVERIDE)); } long currentCount = content.getFileCount() + content.getDirectoryCount(); FileStatus[] executions = fs.listStatus(execHistory); Arrays.sort(executions, new Comparator<FileStatus>() { public int compare(FileStatus f1, FileStatus f2) { return f1.getPath().getName().compareTo(f2.getPath().getName()); } }); // removes oldest directory until we get under required % of count // quota. Won't delete the most recent directory. for (int i = 0; i < executions.length - 1 && limit < currentCount; i++) { FileStatus stat = executions[i]; log.info("removing old execution: " + stat.getPath().getName()); ContentSummary execContent = fs.getContentSummary(stat.getPath()); currentCount -= execContent.getFileCount() + execContent.getDirectoryCount(); fs.delete(stat.getPath(), true); } // removing failed exectutions if we need room if (limit < currentCount) { FileStatus[] failedExecutions = fs.listStatus(execBasePath, new PathFilter() { public boolean accept(Path path) { try { dateFmt.parseDateTime(path.getName()); return true; } catch (IllegalArgumentException e) { return false; } } }); Arrays.sort(failedExecutions, new Comparator<FileStatus>() { public int compare(FileStatus f1, FileStatus f2) { return f1.getPath().getName().compareTo(f2.getPath().getName()); } }); for (int i = 0; i < failedExecutions.length && limit < currentCount; i++) { FileStatus stat = failedExecutions[i]; log.info("removing failed execution: " + stat.getPath().getName()); ContentSummary execContent = fs.getContentSummary(stat.getPath()); currentCount -= execContent.getFileCount() + execContent.getDirectoryCount(); fs.delete(stat.getPath(), true); } } // determining most recent execution and using as the starting point for // this execution if (executions.length > 0) { Path previous = executions[executions.length - 1].getPath(); FileInputFormat.setInputPaths(job, previous); log.info("Previous execution: " + previous.toString()); } else { System.out.println("No previous execution, all topics pulled from earliest available offset"); } // creating new execution dir. offsets, error_logs, and count files will // be written to this directory. data is not written to the // output directory in a normal run, but instead written to the // appropriate date-partitioned subdir in camus.destination.path DateTimeFormatter dateFmt = DateUtils.getDateTimeFormatter("YYYY-MM-dd-HH-mm-ss", DateTimeZone.UTC); String executionDate = new DateTime().toString(dateFmt); Path newExecutionOutput = new Path(execBasePath, executionDate); FileOutputFormat.setOutputPath(job, newExecutionOutput); log.info("New execution temp location: " + newExecutionOutput.toString()); EtlInputFormat.setLogger(log); job.setMapperClass(EtlMapper.class); job.setInputFormatClass(inputFormatClass); job.setOutputFormatClass(outputFormatClass); job.setNumReduceTasks(0); stopTiming("pre-setup"); job.submit(); job.waitForCompletion(true); // dump all counters Counters counters = job.getCounters(); for (String groupName : counters.getGroupNames()) { CounterGroup group = counters.getGroup(groupName); log.info("Group: " + group.getDisplayName()); for (Counter counter : group) { log.info(counter.getDisplayName() + ":\t" + counter.getValue()); } } stopTiming("hadoop"); startTiming("commit"); boolean hadExecutionErrors = checkExecutionErrors(fs, newExecutionOutput); checkIfTooManySkippedMsg(counters); // Send Tracking counts to Kafka String etlCountsClassName = props.getProperty(ETL_COUNTS_CLASS, ETL_COUNTS_CLASS_DEFAULT); Class<? extends EtlCounts> etlCountsClass = (Class<? extends EtlCounts>) Class.forName(etlCountsClassName); sendTrackingCounts(job, fs, newExecutionOutput, etlCountsClass); Path newHistory = new Path(execHistory, executionDate); log.info("Moving execution to history : " + newHistory); fs.rename(newExecutionOutput, newHistory); log.info("Job finished"); stopTiming("commit"); stopTiming("total"); createReport(job, timingMap); if (!job.isSuccessful()) { JobClient client = new JobClient(new JobConf(job.getConfiguration())); TaskCompletionEvent[] tasks = job.getTaskCompletionEvents(0); if (tasks.length > 0) { for (TaskReport task : client.getMapTaskReports(tasks[0].getTaskAttemptId().getJobID())) { if (task.getCurrentStatus().equals(TIPStatus.FAILED)) { for (String s : task.getDiagnostics()) { System.err.println("task error: " + s); } } } } throw new RuntimeException("hadoop job failed"); } if (hadExecutionErrors && props.getProperty(ETL_FAIL_ON_ERRORS, Boolean.FALSE.toString()).equalsIgnoreCase(Boolean.TRUE.toString())) { throw new RuntimeException("Camus saw errors, check stderr"); } if (EtlInputFormat.reportJobFailureDueToOffsetOutOfRange) { EtlInputFormat.reportJobFailureDueToOffsetOutOfRange = false; if (props.getProperty(ETL_FAIL_ON_OFFSET_OUTOFRANGE, ETL_FAIL_ON_OFFSET_OUTOFRANGE_DEFAULT) .equalsIgnoreCase(Boolean.TRUE.toString())) { throw new RuntimeException("Some topics skipped due to offsets from Kafka metadata out of range."); } } if (EtlInputFormat.reportJobFailureUnableToGetOffsetFromKafka) { EtlInputFormat.reportJobFailureUnableToGetOffsetFromKafka = false; throw new RuntimeException("Some topics skipped due to failure in getting latest offset from Kafka leaders."); } if (EtlInputFormat.reportJobFailureDueToLeaderNotAvailable) { EtlInputFormat.reportJobFailureDueToLeaderNotAvailable = false; throw new RuntimeException("Some topic partitions skipped due to Kafka leader not available."); } } private void checkIfTooManySkippedMsg(Counters counters) { double maxPercentSkippedSchemaNotFound = Double.parseDouble(props.getProperty(ETL_MAX_PERCENT_SKIPPED_SCHEMANOTFOUND, ETL_MAX_PERCENT_SKIPPED_SCHEMANOTFOUND_DEFAULT)); double maxPercentSkippedOther = Double.parseDouble(props.getProperty(ETL_MAX_PERCENT_SKIPPED_OTHER, ETL_MAX_PERCENT_SKIPPED_OTHER_DEFAULT)); long actualSkippedSchemaNotFound = 0; long actualSkippedOther = 0; long actualDecodeSuccessful = 0; for (String groupName : counters.getGroupNames()) { if (groupName.equals(EtlRecordReader.KAFKA_MSG.class.getName())) { CounterGroup group = counters.getGroup(groupName); for (Counter counter : group) { if (counter.getDisplayName().equals(EtlRecordReader.KAFKA_MSG.DECODE_SUCCESSFUL.toString())) { actualDecodeSuccessful = counter.getValue(); } else if (counter.getDisplayName().equals(EtlRecordReader.KAFKA_MSG.SKIPPED_SCHEMA_NOT_FOUND.toString())) { actualSkippedSchemaNotFound = counter.getValue(); } else if (counter.getDisplayName().equals(EtlRecordReader.KAFKA_MSG.SKIPPED_OTHER.toString())) { actualSkippedOther = counter.getValue(); } } } } checkIfTooManySkippedMsg(maxPercentSkippedSchemaNotFound, actualSkippedSchemaNotFound, actualDecodeSuccessful, "schema not found"); checkIfTooManySkippedMsg(maxPercentSkippedOther, actualSkippedOther, actualDecodeSuccessful, "other"); } private void checkIfTooManySkippedMsg(double maxPercentAllowed, long actualSkipped, long actualSuccessful, String reason) { if (actualSkipped == 0 && actualSuccessful == 0) { return; } double actualSkippedPercent = (double)actualSkipped / (double)(actualSkipped + actualSuccessful) * 100; if (actualSkippedPercent > maxPercentAllowed) { String message = "job failed: " + actualSkippedPercent + "% messages skipped due to " + reason + ", maximum allowed is " + maxPercentAllowed + "%"; log.error(message); throw new RuntimeException(message); } } public void cancel() throws IOException { if (this.hadoopJob != null) { this.hadoopJob.killJob(); } } private Map<String, List<Pair<EtlKey, ExceptionWritable>>> readErrors(final FileSystem fs, final Path newExecutionOutput) throws IOException { final int maxErrorsFromFile = Integer.parseInt(props.getProperty(ETL_MAX_ERRORS_TO_PRINT_FROM_FILE, ETL_MAX_ERRORS_TO_PRINT_FROM_FILE_DEFAULT)); final Map<String, List<Pair<EtlKey, ExceptionWritable>>> errors = Maps.newHashMap(); for (final FileStatus f : fs.listStatus(newExecutionOutput, new PrefixFilter(EtlMultiOutputFormat.ERRORS_PREFIX))) { int errorCounter = 0; final Path filePath = f.getPath(); final SequenceFile.Reader reader = new SequenceFile.Reader(fs, filePath, fs.getConf()); EtlKey key = new EtlKey(); ExceptionWritable value = new ExceptionWritable(); final List<Pair<EtlKey, ExceptionWritable>> errorsFromFile = Lists.<Pair<EtlKey, ExceptionWritable>>newArrayList(); while (reader.next(key, value)) { errorCounter++; if (errorCounter <= maxErrorsFromFile) { errorsFromFile.add( new Pair<EtlKey, ExceptionWritable>(new EtlKey(key), new ExceptionWritable(value.toString()))); } } if (errorCounter > 0) { if (errorCounter > maxErrorsFromFile) { errorsFromFile.add( new Pair<EtlKey, ExceptionWritable>( new EtlKey(key), new ExceptionWritable("... Too many errors to show. " + "Skipped " + (errorCounter - maxErrorsFromFile) + " ..."))); } errors.put(filePath.toString(), errorsFromFile); } reader.close(); } return errors; } private boolean checkExecutionErrors(final FileSystem fs, final Path newExecutionOutput) throws IOException { Map<String, List<Pair<EtlKey, ExceptionWritable>>> errors = readErrors(fs, newExecutionOutput); // Print any potential errors encountered if (!errors.isEmpty()) log.error("Errors encountered during job run:"); for (final Entry<String, List<Pair<EtlKey, ExceptionWritable>>> fileEntry : errors.entrySet()) { final String filePath = fileEntry.getKey(); final List<Pair<EtlKey, ExceptionWritable>> errorsFromFile = fileEntry.getValue(); if (errorsFromFile.size() > 0) { log.error("Errors from file [" + filePath + "]"); } for (final Pair<EtlKey, ExceptionWritable> errorEntry : errorsFromFile) { final EtlKey errorKey = errorEntry.getKey(); final ExceptionWritable errorValue = errorEntry.getValue(); log.error("Error for EtlKey [" + errorKey + "]: " + errorValue.toString()); } } return !errors.isEmpty(); } // Posts the tracking counts to Kafka public void sendTrackingCounts(JobContext job, FileSystem fs, Path newExecutionOutput, Class<? extends EtlCounts> etlCountsClass) throws IOException, URISyntaxException, IllegalArgumentException, SecurityException, InstantiationException, IllegalAccessException, InvocationTargetException, NoSuchMethodException { if (EtlMultiOutputFormat.isRunTrackingPost(job)) { FileStatus[] gstatuses = fs.listStatus(newExecutionOutput, new PrefixFilter("counts")); HashMap<String, EtlCounts> allCounts = new HashMap<String, EtlCounts>(); for (FileStatus gfileStatus : gstatuses) { FSDataInputStream fdsis = fs.open(gfileStatus.getPath()); BufferedReader br = new BufferedReader(new InputStreamReader(fdsis), 1048576); StringBuffer buffer = new StringBuffer(); String temp = ""; while ((temp = br.readLine()) != null) { buffer.append(temp); } ObjectMapper mapper = new ObjectMapper(); mapper.configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false); ArrayList<EtlCounts> countsObjects = mapper.readValue(buffer.toString(), new TypeReference<ArrayList<EtlCounts>>() { }); for (EtlCounts count : countsObjects) { String topic = count.getTopic(); if (allCounts.containsKey(topic)) { EtlCounts existingCounts = allCounts.get(topic); existingCounts.setEndTime(Math.max(existingCounts.getEndTime(), count.getEndTime())); existingCounts.setLastTimestamp(Math.max(existingCounts.getLastTimestamp(), count.getLastTimestamp())); existingCounts.setStartTime(Math.min(existingCounts.getStartTime(), count.getStartTime())); existingCounts.setFirstTimestamp(Math.min(existingCounts.getFirstTimestamp(), count.getFirstTimestamp())); existingCounts.setErrorCount(existingCounts.getErrorCount() + count.getErrorCount()); existingCounts.setGranularity(count.getGranularity()); existingCounts.setTopic(count.getTopic()); for (Entry<String, Source> entry : count.getCounts().entrySet()) { Source source = entry.getValue(); if (existingCounts.getCounts().containsKey(source.toString())) { Source old = existingCounts.getCounts().get(source.toString()); old.setCount(old.getCount() + source.getCount()); existingCounts.getCounts().put(old.toString(), old); } else { existingCounts.getCounts().put(source.toString(), source); } allCounts.put(topic, existingCounts); } } else { allCounts.put(topic, count); } } } for (FileStatus countFile : fs.listStatus(newExecutionOutput, new PrefixFilter("counts"))) { if (props.getProperty(ETL_KEEP_COUNT_FILES, "false").equals("true")) { fs.rename(countFile.getPath(), new Path(props.getProperty(ETL_COUNTS_PATH), countFile.getPath().getName())); } else { fs.delete(countFile.getPath(), true); } } String brokerList = getKafkaBrokers(job); for (EtlCounts finalEtlCounts : allCounts.values()) { EtlCounts finalCounts = etlCountsClass.getDeclaredConstructor(EtlCounts.class).newInstance(finalEtlCounts); finalCounts.postTrackingCountToKafka(job.getConfiguration(), props.getProperty(KAFKA_MONITOR_TIER), brokerList); } } } /** * Creates a diagnostic report based on provided logger * defaults to TimeLogger which focusses on timing breakdowns. Useful * for determining where to optimize. * * @param job * @param timingMap * @throws IOException */ private void createReport(Job job, Map<String, Long> timingMap) throws IOException, ClassNotFoundException { Class cls = job.getConfiguration().getClassByName(getReporterClass(job)); ((BaseReporter) ReflectionUtils.newInstance(cls, job.getConfiguration())).report(job, timingMap); } /** * Path filter that filters based on prefix */ private class PrefixFilter implements PathFilter { private final String prefix; public PrefixFilter(String prefix) { this.prefix = prefix; } public boolean accept(Path path) { // TODO Auto-generated method stub return path.getName().startsWith(prefix); } } public static void main(String[] args) throws Exception { CamusJob job = new CamusJob(); ToolRunner.run(job, args); } @SuppressWarnings("static-access") @Override public int run(String[] args) throws Exception { Options options = new Options(); options.addOption("p", true, "properties filename from the classpath"); options.addOption("P", true, "external properties filename (hdfs: or local FS)"); options.addOption(OptionBuilder.withArgName("property=value").hasArgs(2).withValueSeparator() .withDescription("use value for given property").create("D")); CommandLineParser parser = new PosixParser(); CommandLine cmd = parser.parse(options, args); if (!(cmd.hasOption('p') || cmd.hasOption('P'))) { HelpFormatter formatter = new HelpFormatter(); formatter.printHelp("CamusJob.java", options); return 1; } if (cmd.hasOption('p')) props.load(this.getClass().getClassLoader().getResourceAsStream(cmd.getOptionValue('p'))); if (cmd.hasOption('P')) { String pathname = cmd.getOptionValue('P'); InputStream fStream; if (pathname.startsWith("hdfs:")) { Path pt = new Path(pathname); FileSystem fs = FileSystem.get(new Configuration()); fStream = fs.open(pt); } else { File file = new File(pathname); fStream = new FileInputStream(file); } props.load(fStream); fStream.close(); } props.putAll(cmd.getOptionProperties("D")); run(); return 0; } // Temporarily adding all Kafka parameters here public static boolean getPostTrackingCountsToKafka(Job job) { return job.getConfiguration().getBoolean(POST_TRACKING_COUNTS_TO_KAFKA, true); } public static int getKafkaFetchRequestMinBytes(JobContext context) { return context.getConfiguration().getInt(KAFKA_FETCH_REQUEST_MIN_BYTES, 1024); } public static int getKafkaFetchRequestMaxWait(JobContext job) { return job.getConfiguration().getInt(KAFKA_FETCH_REQUEST_MAX_WAIT, 1000); } public static String getKafkaBrokers(JobContext job) { String brokers = job.getConfiguration().get(KAFKA_BROKERS); if (brokers == null) { brokers = job.getConfiguration().get(KAFKA_HOST_URL); if (brokers != null) { log.warn("The configuration properties " + KAFKA_HOST_URL + " and " + KAFKA_HOST_PORT + " are deprecated. Please switch to using " + KAFKA_BROKERS); return brokers + ":" + job.getConfiguration().getInt(KAFKA_HOST_PORT, 10251); } } return brokers; } public static int getKafkaFetchRequestCorrelationId(JobContext job) { return job.getConfiguration().getInt(KAFKA_FETCH_REQUEST_CORRELATION_ID, -1); } public static String getKafkaClientName(JobContext job) { return job.getConfiguration().get(KAFKA_CLIENT_NAME); } public static String getKafkaFetchRequestBufferSize(JobContext job) { return job.getConfiguration().get(KAFKA_FETCH_BUFFER_SIZE); } public static int getKafkaTimeoutValue(JobContext job) { int timeOut = job.getConfiguration().getInt(KAFKA_TIMEOUT_VALUE, 30000); return timeOut; } public static int getKafkaBufferSize(JobContext job) { return job.getConfiguration().getInt(KAFKA_FETCH_BUFFER_SIZE, 1024 * 1024); } public static boolean getLog4jConfigure(JobContext job) { return job.getConfiguration().getBoolean(LOG4J_CONFIGURATION, false); } public static String getReporterClass(JobContext job) { return job.getConfiguration().get(CAMUS_REPORTER_CLASS, "com.linkedin.camus.etl.kafka.reporter.TimeReporter"); } }