private long getDataVerifTimeoutMinutes() { return this.state.getPropAsLong(COMPACTION_VERIFICATION_TIMEOUT_MINUTES, DEFAULT_COMPACTION_VERIFICATION_TIMEOUT_MINUTES); }
private long getCompactionTimeoutMinutes() { return this.state.getPropAsLong(COMPACTION_MR_JOB_TIMEOUT_MINUTES, DEFAULT_COMPACTION_MR_JOB_TIMEOUT_MINUTES); }
private long getTargetFileSize() { return this.dataset.jobProps().getPropAsLong(COMPACTION_JOB_TARGET_OUTPUT_FILE_SIZE, DEFAULT_COMPACTION_JOB_TARGET_OUTPUT_FILE_SIZE); }
private void setCompactionSLATimestamp(Dataset dataset) { // Set up SLA timestamp only if this dataset will be compacted and MRCompactor.COMPACTION_INPUT_PATH_TIME is present. if ((this.recompactFromOutputPaths || !MRCompactor.datasetAlreadyCompacted(this.fs, dataset, this.renameSourceDirEnabled)) && dataset.jobProps().contains(MRCompactor.COMPACTION_INPUT_PATH_TIME)) { long timeInMills = dataset.jobProps().getPropAsLong(MRCompactor.COMPACTION_INPUT_PATH_TIME); // Set the upstream time to partition + 1 day. E.g. for 2015/10/13 the upstream time is midnight of 2015/10/14 CompactionSlaEventHelper.setUpstreamTimeStamp(this.state, timeInMills + TimeUnit.MILLISECONDS.convert(1, TimeUnit.DAYS)); } }
/** * Refer to {@link MRCompactorAvroKeyDedupJobRunner#setNumberOfReducers(Job)} */ protected void setNumberOfReducers(Job job) throws IOException { // get input size long inputSize = 0; for (Path inputPath : this.mapReduceInputPaths) { inputSize += this.fs.getContentSummary(inputPath).getLength(); } // get target file size long targetFileSize = this.state.getPropAsLong(MRCompactorAvroKeyDedupJobRunner.COMPACTION_JOB_TARGET_OUTPUT_FILE_SIZE, MRCompactorAvroKeyDedupJobRunner.DEFAULT_COMPACTION_JOB_TARGET_OUTPUT_FILE_SIZE); // get max reducers int maxNumReducers = state.getPropAsInt(MRCompactorAvroKeyDedupJobRunner.COMPACTION_JOB_MAX_NUM_REDUCERS, MRCompactorAvroKeyDedupJobRunner.DEFAULT_COMPACTION_JOB_MAX_NUM_REDUCERS); int numReducers = Math.min(Ints.checkedCast(inputSize / targetFileSize) + 1, maxNumReducers); // get use prime reducers boolean usePrimeReducers = state.getPropAsBoolean(MRCompactorAvroKeyDedupJobRunner.COMPACTION_JOB_USE_PRIME_REDUCERS, MRCompactorAvroKeyDedupJobRunner.DEFAULT_COMPACTION_JOB_USE_PRIME_REDUCERS); if (usePrimeReducers && numReducers != 1) { numReducers = Primes.nextPrime(numReducers); } job.setNumReduceTasks(numReducers); }
@SuppressWarnings({ "serial", "unchecked" }) protected static <T> Optional<T> populateField(State state, String key, TypeToken<T> token) { if (state.contains(key)) { Optional<T> fieldValue; if (new TypeToken<Boolean>() {}.isAssignableFrom(token)) { fieldValue = (Optional<T>) Optional.of(state.getPropAsBoolean(key)); } else if (new TypeToken<Integer>() {}.isAssignableFrom(token)) { fieldValue = (Optional<T>) Optional.of(state.getPropAsInt(key)); } else if (new TypeToken<Long>() {}.isAssignableFrom(token)) { fieldValue = (Optional<T>) Optional.of(state.getPropAsLong(key)); } else if (new TypeToken<List<String>>() {}.isAssignableFrom(token)) { fieldValue = (Optional<T>) Optional.of(state.getPropAsList(key)); } else { fieldValue = (Optional<T>) Optional.of(state.getProp(key)); } state.removeProp(key); return fieldValue; } return Optional.<T> absent(); }
/** * Get an {@link SlaEventSubmitterBuilder} that has dataset urn, partition, record count, previous publish timestamp * and dedupe status set. * The caller MUST set eventSubmitter, eventname before submitting. */ public static SlaEventSubmitterBuilder getEventSubmitterBuilder(Dataset dataset, Optional<Job> job, FileSystem fs) { SlaEventSubmitterBuilder builder = SlaEventSubmitter.builder().datasetUrn(dataset.getUrn()) .partition(dataset.jobProps().getProp(MRCompactor.COMPACTION_JOB_DEST_PARTITION, "")) .dedupeStatus(getOutputDedupeStatus(dataset.jobProps())); long previousPublishTime = getPreviousPublishTime(dataset, fs); long upstreamTime = dataset.jobProps().getPropAsLong(SlaEventKeys.UPSTREAM_TS_IN_MILLI_SECS_KEY, -1l); long recordCount = getRecordCount(job); // Previous publish only exists when this is a recompact job if (previousPublishTime != -1l) { builder.previousPublishTimestamp(Long.toString(previousPublishTime)); } // Upstream time is the logical time represented by the compaction input directory if (upstreamTime != -1l) { builder.upstreamTimestamp(Long.toString(upstreamTime)); } if (recordCount != -1l) { builder.recordCount(Long.toString(recordCount)); } return builder; }