From dc11b8217a34dc04aa077ddc0d21a6b4d1f7e491 Mon Sep 17 00:00:00 2001 From: Piyush Narang Date: Tue, 13 Dec 2016 17:15:26 -0800 Subject: [PATCH 1/3] Estimator memory of container for mapper/reducer. --- build.sbt | 2 +- .../scala/com/twitter/scalding/Config.scala | 15 + .../twitter/scalding/ExecutionContext.scala | 4 +- .../scalding/memory_estimation/Common.scala | 318 ++++++++++++++++++ .../HRavenMemoryService.scala | 232 +++++++++++++ 5 files changed, 569 insertions(+), 2 deletions(-) create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/memory_estimation/Common.scala create mode 100644 scalding-hraven/src/main/scala/com/twitter/scalding/hraven/memory_estimation/HRavenMemoryService.scala diff --git a/build.sbt b/build.sbt index 444db5f1e9..8c0272d059 100644 --- a/build.sbt +++ b/build.sbt @@ -25,7 +25,7 @@ val elephantbirdVersion = "4.15" val hadoopLzoVersion = "0.4.19" val hadoopVersion = "2.5.0" val hbaseVersion = "0.94.10" -val hravenVersion = "0.9.17.t05" +val hravenVersion = "1.0.1" val jacksonVersion = "2.8.7" val json4SVersion = "3.5.0" val paradiseVersion = "2.1.0" diff --git a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala index 20f09be652..3b1d3bc752 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/Config.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/Config.scala @@ -443,6 +443,21 @@ object Config { /** Whether the number of reducers has been set explicitly using a `withReducers` */ val WithReducersSetExplicitly = "scalding.with.reducers.set.explicitly" + /** Name of parameter to specify which class to use as the default estimator. */ + val MemoryEstimators = "scalding.memory.estimator.classes" + + /** Hadoop map memory */ + val MapMemory = "mapreduce.map.memory.mb" + + /** Hadoop map java opts */ + val MapJavaOpts = "mapreduce.map.java.opts" + + /** Hadoop reduce java opts */ + val ReduceJavaOpts = "mapreduce.reduce.java.opts" + + /** Hadoop reduce memory */ + val ReduceMemory = "mapreduce.reduce.memory.mb" + /** Manual description for use in .dot and MR step names set using a `withDescription`. */ val PipeDescriptions = "scalding.pipe.descriptions" val StepDescriptions = "scalding.step.descriptions" diff --git a/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala b/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala index 23644b23b5..4ee8ec52a9 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala @@ -19,6 +19,7 @@ import cascading.flow.hadoop.HadoopFlow import cascading.flow.{ Flow, FlowDef, FlowListener, FlowStepListener, FlowStepStrategy } import cascading.flow.planner.BaseFlowStep import cascading.pipe.Pipe +import com.twitter.scalding.memory_estimation.MemoryEstimatorStepStrategy import com.twitter.scalding.reducer_estimation.ReducerEstimatorStepStrategy import com.twitter.scalding.serialization.CascadingBinaryComparator import org.apache.hadoop.mapred.JobConf @@ -93,13 +94,14 @@ trait ExecutionContext { mode match { case _: HadoopMode => val reducerEstimatorStrategy: Seq[FlowStepStrategy[JobConf]] = config.get(Config.ReducerEstimators).toList.map(_ => ReducerEstimatorStepStrategy) + val memoryEstimatorStrategy: Seq[FlowStepStrategy[JobConf]] = config.get(Config.MemoryEstimators).toList.map(_ => MemoryEstimatorStepStrategy) val otherStrategies: Seq[FlowStepStrategy[JobConf]] = config.getFlowStepStrategies.map { case Success(fn) => fn(mode, configWithId) case Failure(e) => throw new Exception("Failed to decode flow step strategy when submitting job", e) } - val optionalFinalStrategy = FlowStepStrategies().sumOption(reducerEstimatorStrategy ++ otherStrategies) + val optionalFinalStrategy = FlowStepStrategies().sumOption(reducerEstimatorStrategy ++ memoryEstimatorStrategy ++ otherStrategies) optionalFinalStrategy.foreach { strategy => flow.setFlowStepStrategy(strategy) diff --git a/scalding-core/src/main/scala/com/twitter/scalding/memory_estimation/Common.scala b/scalding-core/src/main/scala/com/twitter/scalding/memory_estimation/Common.scala new file mode 100644 index 0000000000..adfdde7934 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/memory_estimation/Common.scala @@ -0,0 +1,318 @@ +package com.twitter.scalding.memory_estimation + +import cascading.flow.{ Flow, FlowStep, FlowStepStrategy } +import com.twitter.algebird.Monoid +import com.twitter.scalding.{ Config, StringUtility } +import com.twitter.scalding.reducer_estimation.{ FlowStepKeys, FlowStrategyInfo } +import java.util.{ List => JList } +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapred.JobConf +import org.slf4j.LoggerFactory +import scala.collection.JavaConverters._ +import scala.util.{ Failure, Success, Try } + +/** + * Info about a prior FlowStep, provided by implementers of HistoryService + */ +final case class FlowStepMemoryHistory(keys: FlowStepKeys, + submitTime: Long, + launchTime: Long, + finishTime: Long, + totalMaps: Long, + totalReduces: Long, + finishedMaps: Long, + finishedReduces: Long, + failedMaps: Long, + failedReduces: Long, + mapFileBytesRead: Long, + mapFileBytesWritten: Long, + reduceFileBytesRead: Long, + hdfsBytesRead: Long, + hdfsBytesWritten: Long, + mapperTimeMillis: Long, + reducerTimeMillis: Long, + reduceShuffleBytes: Long, + cost: Double, + tasks: Seq[Task]) + +final case class Task( + taskType: String, + committedHeapBytes: Long, + physicalMemoryBytes: Long, + cpuMilliSeconds: Long, + gcTimeMillis: Long) + +/** + * Provider of information about prior runs. + */ +trait MemoryService { + def fetchHistory(info: FlowStrategyInfo, maxHistory: Int): Try[Seq[FlowStepMemoryHistory]] +} + +object MemoryEstimatorConfig { + + /** Output param: what the original job map memory was. */ + val originalMapMemory = "scalding.map.memory.estimator.original" + + /** Output param: what the original job map memory was. */ + val originalReduceMemory = "scalding.reduce.memory.estimator.original" + + /** Maximum number of history items to use for memory estimation. */ + val maxHistoryKey = "scalding.memory.estimator.max.history" + + /** + * Value of alpha for exponential smoothing. + * Lower values ensure more smoothing and less importance to newer data + * Higher values provide lesser smoothing and more importance to newer data + */ + val alphaKey = "scalding.memory.estimator.alpha" + + /** Indicates how much to scale the memory estimate after it's calculated */ + val memoryScaleFactor = "scalding.memory.scale.factor" + + //yarn allocates in increments. So we might as well round up our container ask + val yarnSchedulerIncrementAllocationMB: String = "yarn.scheduler.increment-allocation-mb" + + def getMaxHistory(conf: JobConf): Int = conf.getInt(maxHistoryKey, 5) + + def getAlpha(conf: JobConf): Double = conf.getDouble(alphaKey, 1.0) + + def getScaleFactor(conf: JobConf): Double = conf.getDouble(memoryScaleFactor, 1.2) + + def getYarnSchedulerIncrement(conf: JobConf): Int = conf.getInt(yarnSchedulerIncrementAllocationMB, 512) + + // max container is 8GB, given XmxToMemoryScaleFactor we want to ensure this + // estimate when multiplied by that stays below 8GB + val maxMemoryEstimate: Double = 6500.0 * 1024 * 1024 + + // min container is 1G, multiplying by XmxToMemoryScaleFactor keeps us under the + // min container size + val minMemoryEstimate: Double = 800.0 * 1024 * 1024 + + val XmxToMemoryScaleFactor: Double = 1.25 + +} + +// Tuple(MapMemory in MB, ReduceMemory in MB), or None to keep the default. +case class MemoryEstimate(mapMemory: Option[Long], reduceMemory: Option[Long]) + +trait MemoryEstimator { + /** + * Estimate Map / Reduce memory settings. Called for each FlowStep before + * it is scheduled. Custom memory estimators should override this rather than + * apply() directly. + * + * @param info Holds information about the overall flow (.flow), + * previously-run steps (.predecessorSteps), + * and the current step (.step). + * @return MemoryEstimate. + */ + def estimateMemory(info: FlowStrategyInfo): Option[MemoryEstimate] +} + +trait HistoryMemoryEstimator extends MemoryEstimator { + + private val LOG = LoggerFactory.getLogger(this.getClass) + + def memoryService: MemoryService + + override def estimateMemory(info: FlowStrategyInfo): Option[MemoryEstimate] = { + val conf = info.step.getConfig + val maxHistory = MemoryEstimatorConfig.getMaxHistory(conf) + val alpha = MemoryEstimatorConfig.getAlpha(conf) + val scaleFactor = MemoryEstimatorConfig.getScaleFactor(conf) + val yarnSchedulerIncrementMB = MemoryEstimatorConfig.getYarnSchedulerIncrement(conf) + + LOG.info(s"Attempting to estimate memory settings with maxHistory: $maxHistory, alpha: $alpha, scaleFactor: $scaleFactor, schedulerIncrement: $yarnSchedulerIncrementMB") + + memoryService.fetchHistory(info, maxHistory) match { + case Success(h) if h.isEmpty => + LOG.warn("No matching history found.") + None + case Success(h) => + LOG.info(s"${h.length} history entries found.") + val estimate = estimateMemory(info, h, alpha, scaleFactor) + LOG.info(s"Memory estimate: $estimate") + Some(estimate) + case Failure(f) => + LOG.warn(s"Unable to fetch history in $getClass", f) + None + } + } + + private def estimateMemory(info: FlowStrategyInfo, history: Seq[FlowStepMemoryHistory], alpha: Double, scaleFactor: Double): MemoryEstimate = { + // iterate over mem history + // collect: for maps, list of max memory in past runs + // for reduce, list of max memory in past runs + // compute smoothed memory est + // multiple by scale factor + // cap estimate to max size if needed + // handle gc + // return + val maxMemoryList: Seq[(Option[Long], Option[Long])] = history.map{ h => getMapReduceMemory(h) } + val maxMapList: Seq[Long] = maxMemoryList.flatMap(_._1) + val maxReduceList: Seq[Long] = maxMemoryList.flatMap(_._2) + + val mapSmoothEst: Double = smoothedAverage(maxMapList, alpha) + val reduceSmoothEst: Double = smoothedAverage(maxReduceList, alpha) + + val mapScaledEst = mapSmoothEst * scaleFactor + val reduceScaledEst = reduceSmoothEst * scaleFactor + + //todo handle gc values + val cappedMapEst = cappedMemoryEstimateMB(mapScaledEst) + val cappedReduceEst = cappedMemoryEstimateMB(reduceScaledEst) + + LOG.info(s"Calculated map val for: $maxMapList smoothAvg = $mapSmoothEst, scaled: $mapScaledEst, capped: $cappedMapEst") + LOG.info(s"Calculated reduce val for: $maxReduceList smoothAvg = $reduceSmoothEst, scaled: $reduceScaledEst, capped: $cappedReduceEst") + + MemoryEstimate(cappedMapEst, cappedReduceEst) + } + + private def getMapReduceMemory(history: FlowStepMemoryHistory): (Option[Long], Option[Long]) = { + LOG.info(s"Processing tasks: ${history.tasks}") + val reduceTasks: Seq[Task] = history.tasks.filter { t => t.taskType == "REDUCE" } + val mapTasks: Seq[Task] = history.tasks.filter { t => t.taskType == "MAP" } + + // handle empty task list due to either no task history / lack of reducers + val maxReduceCommittedHeap: Option[Long] = if (reduceTasks.isEmpty) None else Some(reduceTasks.map(_.committedHeapBytes).max) + val maxMapCommittedHeap: Option[Long] = if (mapTasks.isEmpty) None else Some(mapTasks.map(_.committedHeapBytes).max) + + LOG.info(s"Calculated max committed heap for job: ${history.keys}, map: $maxMapCommittedHeap reduce: $maxReduceCommittedHeap") + (maxMapCommittedHeap, maxReduceCommittedHeap) + } + + // memoryEstimate = (currentMemoryValue * alpha) + (1 - alpha) * oldEstimate + private def smoothedAverage(memoryList: Seq[Long], alpha: Double): Double = { + memoryList.foldLeft(0.0){ (oldEstimate, currentVal) => (currentVal * alpha) + (1 - alpha) * oldEstimate } + } + + // calculate the capped Xmx memory estimate + private def cappedMemoryEstimateMB(memoryEst: Double): Option[Long] = { + val memoryEstimateBytes: Option[Double] = + if (memoryEst > MemoryEstimatorConfig.maxMemoryEstimate) + Some(MemoryEstimatorConfig.maxMemoryEstimate) + else if (memoryEst == 0) + None + else if (memoryEst < MemoryEstimatorConfig.minMemoryEstimate) + Some(MemoryEstimatorConfig.minMemoryEstimate) + else + Some(memoryEst) + + memoryEstimateBytes.map{ est => (est / (1024 * 1024)).toLong } + } + +} + +case class FallbackMemoryEstimator(first: MemoryEstimator, fallback: MemoryEstimator) extends MemoryEstimator { + private val LOG = LoggerFactory.getLogger(this.getClass) + + override def estimateMemory(info: FlowStrategyInfo): Option[MemoryEstimate] = + first.estimateMemory(info).orElse { + LOG.warn(s"$first estimator failed. Falling back to $fallback.") + fallback.estimateMemory(info) + } +} + +object MemoryEstimatorStepStrategy extends FlowStepStrategy[JobConf] { + + private val LOG = LoggerFactory.getLogger(this.getClass) + + implicit val estimatorMonoid: Monoid[MemoryEstimator] = new Monoid[MemoryEstimator] { + override def zero: MemoryEstimator = new MemoryEstimator { + override def estimateMemory(info: FlowStrategyInfo): Option[MemoryEstimate] = None + } + + override def plus(l: MemoryEstimator, r: MemoryEstimator): MemoryEstimator = + FallbackMemoryEstimator(l, r) + } + + /** + * Make memory estimate, possibly overriding explicitly-set memory settings, + * and save useful info (such as the default & estimate) in JobConf for + * later consumption. + * + * Called by Cascading at the start of each job step. + */ + final override def apply(flow: Flow[JobConf], + preds: JList[FlowStep[JobConf]], + step: FlowStep[JobConf]): Unit = { + + if (skipMemoryEstimation(step)) { + LOG.info(s"Skipping memory estimation as ${Config.MemoryEstimators} is not set ") + } else { + estimate(flow, preds, step) + } + } + + private def skipMemoryEstimation(step: FlowStep[JobConf]): Boolean = + step.getConfig.get(Config.MemoryEstimators) == null + + private def estimate(flow: Flow[JobConf], + preds: JList[FlowStep[JobConf]], + step: FlowStep[JobConf]): Unit = { + val conf = step.getConfig + + Option(conf.get(Config.MemoryEstimators)).foreach { clsNames => + + val clsLoader = Thread.currentThread.getContextClassLoader + + val estimators = StringUtility.fastSplit(clsNames, ",") + .map(clsLoader.loadClass(_).newInstance.asInstanceOf[MemoryEstimator]) + val combinedEstimator = Monoid.sum(estimators) + + val info = FlowStrategyInfo(flow, preds.asScala, step) + + // get memory estimate + val memoryEstimate: Option[MemoryEstimate] = combinedEstimator.estimateMemory(info) + + memoryEstimate match { + case Some(MemoryEstimate(Some(mapMem), Some(reduceMem))) => + LOG.info(s"Overriding map Xmx memory to: $mapMem and reduce Xmx memory to $reduceMem") + setMapMemory(mapMem, conf) + setReduceMemory(reduceMem, conf) + case Some(MemoryEstimate(Some(mapMem), _)) => + LOG.info(s"Overriding only map Xmx memory to: $mapMem") + setMapMemory(mapMem, conf) + case Some(MemoryEstimate(_, Some(reduceMem))) => + LOG.info(s"Overriding only reduce Xmx memory to $reduceMem") + setReduceMemory(reduceMem, conf) + case _ => LOG.info("Memory estimators didn't calculate any value. Skipping setting memory overrides") + } + } + } + + private def setMapMemory(mapMem: Long, conf: Configuration): Unit = { + conf.setLong(MemoryEstimatorConfig.originalMapMemory, conf.getLong(Config.MapMemory, 0L)) + + val mapContainerMem = mapMem * MemoryEstimatorConfig.XmxToMemoryScaleFactor + val schedulerIncrement = conf.getInt(MemoryEstimatorConfig.yarnSchedulerIncrementAllocationMB, 512) + val roundedMapContainerMem = roundUp(mapContainerMem, schedulerIncrement).toLong + + conf.setLong(Config.MapMemory, roundedMapContainerMem) + + val mapOpts = conf.get(Config.MapJavaOpts, "") + //remove existing xmx / xms + val mapOptsWithoutXm = mapOpts.split(" ").filterNot(s => s.startsWith("-Xmx") || s.startsWith("-Xms")).mkString(" ") + + conf.set(Config.MapJavaOpts, mapOptsWithoutXm + s" -Xmx${mapMem}M") + } + + private def setReduceMemory(reduceMem: Long, conf: Configuration): Unit = { + conf.setLong(MemoryEstimatorConfig.originalReduceMemory, conf.getLong(Config.ReduceMemory, 0L)) + + val reduceContainerMem = reduceMem * MemoryEstimatorConfig.XmxToMemoryScaleFactor + val schedulerIncrement = conf.getInt(MemoryEstimatorConfig.yarnSchedulerIncrementAllocationMB, 512) + val roundedReduceContainerMem = roundUp(reduceContainerMem, schedulerIncrement).toLong + conf.setLong(Config.ReduceMemory, roundedReduceContainerMem) + + val reduceOpts = conf.get(Config.ReduceJavaOpts, "") + //remove existing xmx / xms + val reduceOptsWithoutXm = reduceOpts.split(" ").filterNot(s => s.startsWith("-Xmx") || s.startsWith("-Xms")).mkString(" ") + + conf.set(Config.ReduceJavaOpts, reduceOptsWithoutXm + s" -Xmx${reduceMem}M") + } + + //Round up value to a multiple of block + private def roundUp(value: Double, block: Double): Double = Math.ceil(value / block) * block +} diff --git a/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/memory_estimation/HRavenMemoryService.scala b/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/memory_estimation/HRavenMemoryService.scala new file mode 100644 index 0000000000..c77d4f502d --- /dev/null +++ b/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/memory_estimation/HRavenMemoryService.scala @@ -0,0 +1,232 @@ +package com.twitter.scalding.hraven.memory_estimation + +import java.io.IOException +import cascading.flow.FlowStep +import com.twitter.hraven.{ Flow, JobDetails } +import com.twitter.hraven.rest.client.HRavenRestClient +import org.apache.hadoop.mapred.JobConf +import org.slf4j.LoggerFactory +import scala.collection.JavaConverters._ +import com.twitter.hraven.JobDescFactory.{ JOBTRACKER_KEY, RESOURCE_MANAGER_KEY } +import com.twitter.scalding.memory_estimation.{ FlowStepMemoryHistory, HistoryMemoryEstimator, MemoryService, Task } +import com.twitter.scalding.reducer_estimation.{ FlowStepKeys, FlowStrategyInfo } +import scala.util.{ Failure, Success, Try } + +object HRavenClient { + import HRavenMemoryService.jobConfToRichConfig + + val apiHostnameKey = "hraven.api.hostname" // hraven.devel.vchannapattan.service.smf1.twitter.com + val clientConnectTimeoutKey = "hraven.client.connect.timeout" + val clientReadTimeoutKey = "hraven.client.read.timeout" + + private final val clientConnectTimeoutDefault = 120000 + private final val clientReadTimeoutDefault = 120000 + + def apply(conf: JobConf): Try[HRavenRestClient] = + conf.getFirstKey(apiHostnameKey) + .map(new HRavenRestClient(_, + conf.getInt(clientConnectTimeoutKey, clientConnectTimeoutDefault), + conf.getInt(clientReadTimeoutKey, clientReadTimeoutDefault))) +} + +/** + * Mixin for MemoryEstimators to give them the ability to query hRaven for + * info about past runs. + */ +object HRavenMemoryService extends MemoryService { + + private val LOG = LoggerFactory.getLogger(this.getClass) + + private val TaskCounterGroup = "org.apache.hadoop.mapreduce.TaskCounter" + private val CommittedHeapBytes = "COMMITTED_HEAP_BYTES" + private val CpuMs = "CPU_MILLISECONDS" + private val PhysicalMemoryBytes = "PHYSICAL_MEMORY_BYTES" + private val GCTimeMs = "GC_TIME_MILLIS" + + // List of fields that we consume from fetchTaskDetails api. + // This is sent to hraven service to filter the response data + // and avoid hitting http content length limit on hraven side. + private val TaskDetailFields = List("taskType").asJava + + private val TaskCounterFields = List( + "org.apache.hadoop.mapreduce.TaskCounter.COMMITTED_HEAP_BYTES", + "org.apache.hadoop.mapreduce.TaskCounter.PHYSICAL_MEMORY_BYTES", + "org.apache.hadoop.mapreduce.TaskCounter.GC_TIME_MILLIS", + "org.apache.hadoop.mapreduce.TaskCounter.CPU_MILLISECONDS").asJava + + val RequiredJobConfigs = Seq("cascading.flow.step.num") + + case class MissingFieldsException(fields: Seq[String]) extends Exception + + /** + * Add some helper methods to JobConf + */ + case class RichConfig(conf: JobConf) { + + val MaxFetch = "hraven.reducer.estimator.max.flow.history" + val MaxFetchDefault = 10 + + def maxFetch: Int = conf.getInt(MaxFetch, MaxFetchDefault) + + /** + * Try fields in order until one returns a value. + * Logs a warning if nothing was found. + */ + def getFirstKey(fields: String*): Try[String] = + fields.collectFirst { + case f if conf.get(f) != null => Success(conf.get(f)) + }.getOrElse { + LOG.warn("Missing required config param: " + fields.mkString(" or ")) + Failure(MissingFieldsException(fields)) + } + + } + implicit def jobConfToRichConfig(conf: JobConf): RichConfig = RichConfig(conf) + + /** + * Fetch flows until it finds one that was successful + * (using "HdfsBytesRead > 0" as a marker for successful jobs since it seems + * that this is only set on completion of jobs) + * + * TODO: query hRaven for successful jobs (first need to add ability to filter + * results in hRaven REST API) + */ + private def fetchSuccessfulFlows(client: HRavenRestClient, cluster: String, user: String, batch: String, signature: String, max: Int, nFetch: Int): Try[Seq[Flow]] = + Try(client.fetchFlowsWithConfig(cluster, user, batch, signature, nFetch, RequiredJobConfigs: _*)) + .flatMap { flows => + Try { + // Ugly mutable code to add task info to flows + flows.asScala.foreach { flow => + flow.getJobs.asScala.foreach { job => + + // client.fetchTaskDetails might throw IOException + val tasks = client.fetchTaskDetails(flow.getCluster, job.getJobId, TaskDetailFields, TaskCounterFields) + job.addTasks(tasks) + } + } + + val successfulFlows = flows.asScala.filter(_.getHdfsBytesRead > 0).take(max) + if (successfulFlows.isEmpty) { + LOG.warn("Unable to find any successful flows in the last " + nFetch + " jobs.") + } + successfulFlows + } + }.recoverWith { + case e: IOException => + LOG.error("Error making API request to hRaven. HRavenMemoryService will be disabled.") + Failure(e) + } + + /** + * Fetch info from hRaven for the last time the given JobStep ran. + * Finds the last successful complete flow and selects the corresponding + * step from it. + * + * @param step FlowStep to get info for + * @return Details about the previous successful run. + */ + def fetchPastJobDetails(step: FlowStep[JobConf], max: Int): Try[Seq[JobDetails]] = { + val conf = step.getConfig + val stepNum = step.getStepNum + + def findMatchingJobStep(pastFlow: Flow) = + pastFlow.getJobs.asScala.find { step => + try { + step.getConfiguration.get("cascading.flow.step.num").toInt == stepNum + } catch { + case _: NumberFormatException => false + } + } orElse { + LOG.warn("No matching job step in the retrieved hRaven flow.") + None + } + + def lookupClusterName(client: HRavenRestClient): Try[String] = { + // regex for case matching URL to get hostname out + val hostRegex = """(.*):\d+""".r + + // first try resource manager (for Hadoop v2), then fallback to job tracker + conf.getFirstKey(RESOURCE_MANAGER_KEY, JOBTRACKER_KEY).flatMap { + // extract hostname from hostname:port + case hostRegex(host) => + // convert hostname -> cluster name (e.g. dw2@smf1) + Try(client.getCluster(host)) + } + } + + val flowsTry = for { + // connect to hRaven REST API + client <- HRavenClient(conf) + + // lookup cluster name used by hRaven + cluster <- lookupClusterName(client) + + // get identifying info for this job + user <- conf.getFirstKey("hraven.history.user.name", "user.name") + batch <- conf.getFirstKey("batch.desc") + signature <- conf.getFirstKey("scalding.flow.class.signature") + + // query hRaven for matching flows + flows <- fetchSuccessfulFlows(client, cluster, user, batch, signature, max, conf.maxFetch) + + } yield flows + + // Find the FlowStep in the hRaven flow that corresponds to the current step + // *Note*: when hRaven says "Job" it means "FlowStep" + flowsTry.map{ flows => + val map: Seq[JobDetails] = flows.flatMap(findMatchingJobStep) + LOG.info("Found :" + flows.length + " flows in fetchJobDetails. After finding matching job step, noJobDetails = " + map.length) + map + } + } + + override def fetchHistory(info: FlowStrategyInfo, maxHistory: Int): Try[Seq[FlowStepMemoryHistory]] = + fetchPastJobDetails(info.step, maxHistory).map { history => + for { + step <- history + keys = FlowStepKeys(step.getJobName, step.getUser, step.getPriority, step.getStatus, step.getVersion, "") + // update HRavenHistoryService.TaskDetailFields when consuming additional task fields from hraven below + tasks = step.getTasks.asScala.flatMap { + t => + //sometimes get groups with only partial data + if (t.getCounters.getGroups.isEmpty || t.getCounters.getGroup(TaskCounterGroup).size() < 4) None + else { + val group = t.getCounters.getGroup(TaskCounterGroup) + + val committedHeap = group.get(CommittedHeapBytes).getValue + val cpu = group.get(CpuMs).getValue + val phyMemory = group.get(PhysicalMemoryBytes).getValue + val gc = group.get(GCTimeMs).getValue + Some(Task(t.getType, committedHeap, phyMemory, cpu, gc)) + } + } + } yield toFlowStepMemoryHistory(keys, step, tasks) + } + + private def toFlowStepMemoryHistory(keys: FlowStepKeys, step: JobDetails, tasks: Seq[Task]) = + FlowStepMemoryHistory( + keys = keys, + submitTime = step.getSubmitTime, + launchTime = step.getLaunchTime, + finishTime = step.getFinishTime, + totalMaps = step.getTotalMaps, + totalReduces = step.getTotalReduces, + finishedMaps = step.getFinishedMaps, + finishedReduces = step.getFinishedReduces, + failedMaps = step.getFailedMaps, + failedReduces = step.getFailedReduces, + mapFileBytesRead = step.getMapFileBytesRead, + mapFileBytesWritten = step.getMapFileBytesWritten, + reduceFileBytesRead = step.getReduceFileBytesRead, + hdfsBytesRead = step.getHdfsBytesRead, + hdfsBytesWritten = step.getHdfsBytesWritten, + mapperTimeMillis = step.getMapSlotMillis, + reducerTimeMillis = step.getReduceSlotMillis, + reduceShuffleBytes = step.getReduceShuffleBytes, + cost = 0, + tasks = tasks) +} + +class HRavenMemoryEstimator extends HistoryMemoryEstimator { + override val memoryService = HRavenMemoryService +} From c97446a3a8bc4e902e2664bf8ce80e4d89a26da7 Mon Sep 17 00:00:00 2001 From: Anton Panasenko Date: Mon, 17 Apr 2017 18:26:03 -0700 Subject: [PATCH 2/3] Refactoring memory and reducer estimator --- .../twitter/scalding/ExecutionContext.scala | 6 +- .../twitter/scalding/estimation/Common.scala | 36 + .../scalding/estimation/Estimator.scala | 67 + .../scalding/estimation/HistoryService.scala | 50 + .../memory/MemoryEstimatorConfig.scala | 47 + .../memory/MemoryEstimatorStepStrategy.scala | 92 + .../SmoothedHistoryMemoryEstimator.scala | 117 + .../scalding/memory_estimation/Common.scala | 318 -- .../scalding/reducer_estimation/Common.scala | 278 -- .../InputSizeReducerEstimator.scala | 5 +- .../RatioBasedEstimator.scala | 79 +- .../ReducerEstimatorConfig.scala | 33 + .../ReducerEstimatorStepStrategy.scala | 108 + .../ReducerHistoryEstimator.scala | 20 + .../RuntimeReducerEstimator.scala | 164 +- .../MemoryEstimatorStepStrategyTest.scala | 37 + .../SmoothedHistoryMemoryEstimatorTest.scala | 177 + .../memory/MemoryEstimatorTest.scala | 243 ++ .../RatioBasedEstimatorTest.scala | 28 +- .../ReducerEstimatorTest.scala | 26 +- .../RuntimeReducerEstimatorTest.scala | 9 +- .../HRavenHistoryService.scala | 132 +- .../memory/HRavenMemoryService.scala | 48 + .../HRavenMemoryService.scala | 232 -- .../HRavenBasedEstimator.scala | 38 + .../src/test/resources/flowResponse.json | 3303 +++++++++++++++++ ...jobResponse_job_1470171371859_6607542.json | 65 + ...jobResponse_job_1470171371859_6608570.json | 65 + ...jobResponse_job_1470171371859_6609558.json | 75 + .../estimation/HRavenHistoryServiceTest.scala | 153 + 30 files changed, 5022 insertions(+), 1029 deletions(-) create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/estimation/Common.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/estimation/Estimator.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/estimation/HistoryService.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorConfig.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorStepStrategy.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/estimation/memory/SmoothedHistoryMemoryEstimator.scala delete mode 100644 scalding-core/src/main/scala/com/twitter/scalding/memory_estimation/Common.scala delete mode 100644 scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorConfig.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorStepStrategy.scala create mode 100644 scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/ReducerHistoryEstimator.scala create mode 100644 scalding-core/src/test/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorStepStrategyTest.scala create mode 100644 scalding-core/src/test/scala/com/twitter/scalding/estimation/memory/SmoothedHistoryMemoryEstimatorTest.scala create mode 100644 scalding-hadoop-test/src/test/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorTest.scala rename scalding-hraven/src/main/scala/com/twitter/scalding/hraven/{reducer_estimation => estimation}/HRavenHistoryService.scala (69%) create mode 100644 scalding-hraven/src/main/scala/com/twitter/scalding/hraven/estimation/memory/HRavenMemoryService.scala delete mode 100644 scalding-hraven/src/main/scala/com/twitter/scalding/hraven/memory_estimation/HRavenMemoryService.scala create mode 100644 scalding-hraven/src/main/scala/com/twitter/scalding/hraven/reducer_estimation/HRavenBasedEstimator.scala create mode 100644 scalding-hraven/src/test/resources/flowResponse.json create mode 100644 scalding-hraven/src/test/resources/jobResponse_job_1470171371859_6607542.json create mode 100644 scalding-hraven/src/test/resources/jobResponse_job_1470171371859_6608570.json create mode 100644 scalding-hraven/src/test/resources/jobResponse_job_1470171371859_6609558.json create mode 100644 scalding-hraven/src/test/scala/com/twitter/scalding/hraven/estimation/HRavenHistoryServiceTest.scala diff --git a/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala b/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala index 4ee8ec52a9..f551b80b86 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/ExecutionContext.scala @@ -16,17 +16,17 @@ limitations under the License. package com.twitter.scalding import cascading.flow.hadoop.HadoopFlow -import cascading.flow.{ Flow, FlowDef, FlowListener, FlowStepListener, FlowStepStrategy } import cascading.flow.planner.BaseFlowStep +import cascading.flow.{ Flow, FlowDef, FlowStepStrategy } import cascading.pipe.Pipe -import com.twitter.scalding.memory_estimation.MemoryEstimatorStepStrategy +import com.twitter.scalding.estimation.memory.MemoryEstimatorStepStrategy import com.twitter.scalding.reducer_estimation.ReducerEstimatorStepStrategy import com.twitter.scalding.serialization.CascadingBinaryComparator import org.apache.hadoop.mapred.JobConf +import org.slf4j.{ Logger, LoggerFactory } import scala.collection.JavaConverters._ import scala.concurrent.Future import scala.util.{ Failure, Success, Try } -import org.slf4j.{ Logger, LoggerFactory } /* * This has all the state needed to build a single flow diff --git a/scalding-core/src/main/scala/com/twitter/scalding/estimation/Common.scala b/scalding-core/src/main/scala/com/twitter/scalding/estimation/Common.scala new file mode 100644 index 0000000000..0c8da2e65e --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/estimation/Common.scala @@ -0,0 +1,36 @@ +package com.twitter.scalding.estimation + +import cascading.flow.FlowStep +import cascading.tap.hadoop.Hfs +import cascading.tap.{ CompositeTap, Tap } +import com.twitter.scalding.tap.GlobHfs +import org.apache.hadoop.mapred.JobConf +import org.slf4j.LoggerFactory +import scala.collection.JavaConverters._ + +object Common { + private[this] val LOG = LoggerFactory.getLogger(this.getClass) + + private def unrollTaps(taps: Seq[Tap[_, _, _]]): Seq[Tap[_, _, _]] = + taps.flatMap { + case multi: CompositeTap[_] => + unrollTaps(multi.getChildTaps.asScala.toSeq) + case t => Seq(t) + } + + def unrollTaps(step: FlowStep[JobConf]): Seq[Tap[_, _, _]] = + unrollTaps(step.getSources.asScala.toSeq) + + def inputSizes(step: FlowStep[JobConf]): Seq[(String, Long)] = { + val conf = step.getConfig + unrollTaps(step).flatMap { + case tap: GlobHfs => Some(tap.toString -> tap.getSize(conf)) + case tap: Hfs => Some(tap.toString -> GlobHfs.getSize(tap.getPath, conf)) + case tap => + LOG.warn("InputSizeReducerEstimator unable to calculate size: " + tap) + None + } + } + + def totalInputSize(step: FlowStep[JobConf]): Long = inputSizes(step).map(_._2).sum +} \ No newline at end of file diff --git a/scalding-core/src/main/scala/com/twitter/scalding/estimation/Estimator.scala b/scalding-core/src/main/scala/com/twitter/scalding/estimation/Estimator.scala new file mode 100644 index 0000000000..f16181071e --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/estimation/Estimator.scala @@ -0,0 +1,67 @@ +package com.twitter.scalding.estimation + +import cascading.flow.{ Flow, FlowStep } +import com.twitter.algebird.Monoid +import org.apache.hadoop.mapred.JobConf +import org.slf4j.LoggerFactory +import scala.util.{ Failure, Success } + +case class FlowStrategyInfo( + flow: Flow[JobConf], + predecessorSteps: Seq[FlowStep[JobConf]], + step: FlowStep[JobConf]) + +/** + * Trait for estimation some parameters of Job. + * @tparam T return type of estimation + */ +trait Estimator[T] { + def estimate(info: FlowStrategyInfo): Option[T] +} + +case class FallbackEstimator[T](first: Estimator[T], fallback: Estimator[T]) extends Estimator[T] { + private val LOG = LoggerFactory.getLogger(this.getClass) + + override def estimate(info: FlowStrategyInfo): Option[T] = { + first.estimate(info).orElse { + LOG.warn(s"$first estimator failed. Falling back to $fallback.") + fallback.estimate(info) + } + } +} + +class FallbackEstimatorMonoid[T] extends Monoid[Estimator[T]] { + override def zero: Estimator[T] = new Estimator[T] { + override def estimate(info: FlowStrategyInfo): Option[T] = None + } + + override def plus(l: Estimator[T], r: Estimator[T]): Estimator[T] = FallbackEstimator(l, r) +} + +trait HistoryEstimator[T] extends Estimator[T] { + private val LOG = LoggerFactory.getLogger(this.getClass) + + def maxHistoryItems(conf: JobConf): Int + + def historyService: HistoryService + + override def estimate(info: FlowStrategyInfo): Option[T] = { + val conf = info.step.getConfig + + historyService.fetchHistory(info, maxHistoryItems(conf)) match { + case Success(history) if history.isEmpty => + LOG.warn(s"No matching history found for $info") + None + case Success(history) => + LOG.info(s"${history.length} history entries found for $info") + val estimation = estimate(info, conf, history) + LOG.info(s"$getClass estimate: $estimation") + estimation + case Failure(f) => + LOG.warn(s"Unable to fetch history in $getClass", f) + None + } + } + + protected def estimate(info: FlowStrategyInfo, conf: JobConf, history: Seq[FlowStepHistory]): Option[T] +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/estimation/HistoryService.scala b/scalding-core/src/main/scala/com/twitter/scalding/estimation/HistoryService.scala new file mode 100644 index 0000000000..488b45b280 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/estimation/HistoryService.scala @@ -0,0 +1,50 @@ +package com.twitter.scalding.estimation + +import scala.util.Try + +/** + * Info about a prior FlowStep, provided by implementers of HistoryService + */ +final case class FlowStepHistory( + keys: FlowStepKeys, + submitTimeMillis: Long, + launchTimeMillis: Long, + finishTimeMillis: Long, + totalMaps: Long, + totalReduces: Long, + finishedMaps: Long, + finishedReduces: Long, + failedMaps: Long, + failedReduces: Long, + mapFileBytesRead: Long, + mapFileBytesWritten: Long, + mapOutputBytes: Long, + reduceFileBytesRead: Long, + hdfsBytesRead: Long, + hdfsBytesWritten: Long, + mapperTimeMillis: Long, + reducerTimeMillis: Long, + reduceShuffleBytes: Long, + cost: Double, + tasks: Seq[Task]) + +final case class FlowStepKeys( + jobName: String, + user: String, + priority: String, + status: String, + version: String, + queue: String) + +final case class Task(details: Map[String, Any], counters: Map[String, Long]) { + def taskType: Option[String] = details.get(Task.TaskType).map(_.asInstanceOf[String]) +} + +object Task { + val TaskType = "taskType" +} + +trait HistoryService { + def fetchHistory(info: FlowStrategyInfo, maxHistory: Int): Try[Seq[FlowStepHistory]] +} + diff --git a/scalding-core/src/main/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorConfig.scala b/scalding-core/src/main/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorConfig.scala new file mode 100644 index 0000000000..55f67e4fd2 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorConfig.scala @@ -0,0 +1,47 @@ +package com.twitter.scalding.estimation.memory + +import org.apache.hadoop.mapred.JobConf + +object MemoryEstimatorConfig { + /** Output param: what the original job map memory was. */ + val originalMapMemory = "scalding.map.memory.estimator.original" + + /** Output param: what the original job reduce memory was. */ + val originalReduceMemory = "scalding.reduce.memory.estimator.original" + + /** + * Value of alpha for exponential smoothing. + * Lower values ensure more smoothing and less importance to newer data + * Higher values provide lesser smoothing and more importance to newer data + */ + val alphaKey = "scalding.memory.estimator.alpha" + + /** Indicates how much to scale the memory estimate after it's calculated */ + val memoryScaleFactor = "scalding.memory.estimator.scale.factor" + + val XmxToMemoryScaleFactorKey = "scalding.memory.estimator.xmx.scale.factor" + + val maxContainerMemoryKey = "scalding.memory.estimator.container.max" + + val minContainerMemoryKey = "scalding.memory.estimator.container.min" + + /** yarn allocates in increments. So we might as well round up our container ask **/ + val yarnSchedulerIncrementAllocationMB = "yarn.scheduler.increment-allocation-mb" + + /** Maximum number of history items to use for memory estimation. */ + val maxHistoryKey = "scalding.memory.estimator.max.history" + + def getMaxContainerMemory(conf: JobConf): Long = conf.getLong(maxContainerMemoryKey, 8 * 1024) + + def getMinContainerMemory(conf: JobConf): Long = conf.getLong(minContainerMemoryKey, 1 * 1024) + + def getAlpha(conf: JobConf): Double = conf.getDouble(alphaKey, 1.0) + + def getScaleFactor(conf: JobConf): Double = conf.getDouble(memoryScaleFactor, 1.2) + + def getXmxScaleFactor(conf: JobConf): Double = conf.getDouble(XmxToMemoryScaleFactorKey, 1.25) + + def getYarnSchedulerIncrement(conf: JobConf): Int = conf.getInt(yarnSchedulerIncrementAllocationMB, 512) + + def getMaxHistory(conf: JobConf): Int = conf.getInt(maxHistoryKey, 5) +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorStepStrategy.scala b/scalding-core/src/main/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorStepStrategy.scala new file mode 100644 index 0000000000..540775e7b9 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorStepStrategy.scala @@ -0,0 +1,92 @@ +package com.twitter.scalding.estimation.memory + +import cascading.flow.{ Flow, FlowStep, FlowStepStrategy } +import com.twitter.algebird.Monoid +import com.twitter.scalding.estimation.{ Estimator, FallbackEstimatorMonoid, FlowStrategyInfo } +import com.twitter.scalding.{ Config, StringUtility } +import java.util.{ List => JList } +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapred.JobConf +import org.slf4j.LoggerFactory +import scala.collection.JavaConverters._ + +object MemoryEstimatorStepStrategy extends FlowStepStrategy[JobConf] { + + private val LOG = LoggerFactory.getLogger(this.getClass) + + implicit val estimatorMonoid = new FallbackEstimatorMonoid[MemoryEstimate] + + /** + * Make memory estimate, possibly overriding explicitly-set memory settings, + * and save useful info (such as the original & estimate value of memory settings) + * in JobConf for later consumption. + * + * Called by Cascading at the start of each job step. + */ + final override def apply( + flow: Flow[JobConf], + preds: JList[FlowStep[JobConf]], + step: FlowStep[JobConf]): Unit = { + + if (skipMemoryEstimation(step)) { + LOG.info(s"Skipping memory estimation as ${Config.MemoryEstimators} is not set ") + } else { + estimate(flow, preds.asScala, step) + } + } + + private[estimation] def skipMemoryEstimation(step: FlowStep[JobConf]): Boolean = + step.getConfig.get(Config.MemoryEstimators, "").isEmpty + + private[estimation] def estimate( + flow: Flow[JobConf], + preds: Seq[FlowStep[JobConf]], + step: FlowStep[JobConf]): Unit = { + val conf = step.getConfig + + Option(conf.get(Config.MemoryEstimators)).foreach { clsNames => + + val clsLoader = Thread.currentThread.getContextClassLoader + + val estimators = StringUtility.fastSplit(clsNames, ",") + .map(clsLoader.loadClass(_).newInstance.asInstanceOf[Estimator[MemoryEstimate]]) + val combinedEstimator = Monoid.sum(estimators) + + val info = FlowStrategyInfo(flow, preds, step) + + // get memory estimate + val memoryEstimate: Option[MemoryEstimate] = combinedEstimator.estimate(info) + + memoryEstimate match { + case Some(MemoryEstimate(Some(mapMem), Some(reduceMem))) => + LOG.info(s"Overriding map memory to: $mapMem in Mb and reduce memory to: $reduceMem in Mb") + setMemory(mapMem, (Config.MapJavaOpts, Config.MapMemory), conf) + setMemory(reduceMem, (Config.ReduceJavaOpts, Config.ReduceMemory), conf) + case Some(MemoryEstimate(Some(mapMem), _)) => + LOG.info(s"Overriding only map memory to: $mapMem in Mb") + setMemory(mapMem, (Config.MapJavaOpts, Config.MapMemory), conf) + case Some(MemoryEstimate(_, Some(reduceMem))) => + LOG.info(s"Overriding only reduce memory to: $reduceMem in Mb") + setMemory(reduceMem, (Config.ReduceJavaOpts, Config.ReduceMemory), conf) + case _ => LOG.info("Memory estimators didn't calculate any value. Skipping setting memory overrides") + } + } + } + + private[estimation] def setMemory(memorySettings: (Long, Long), keys: (String, String), conf: JobConf): Unit = { + val (xmxMemory, containerMemory) = memorySettings + val (xmxKey, containerKey) = keys + + conf.setLong(containerKey, containerMemory) + + setXmxMemory(xmxKey, xmxMemory, conf) + } + + private[estimation] def setXmxMemory(xmxKey: String, xmxMemory: Long, conf: JobConf): Unit = { + val xmxOpts = conf.get(xmxKey, "") + //remove existing xmx / xms + val xmxOptsWithoutXm = xmxOpts.split(" ").filterNot(s => s.startsWith("-Xmx") || s.startsWith("-Xms")).mkString(" ") + + conf.set(xmxKey, xmxOptsWithoutXm + s" -Xmx${xmxMemory}m") + } +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/estimation/memory/SmoothedHistoryMemoryEstimator.scala b/scalding-core/src/main/scala/com/twitter/scalding/estimation/memory/SmoothedHistoryMemoryEstimator.scala new file mode 100644 index 0000000000..8d28749ad9 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/estimation/memory/SmoothedHistoryMemoryEstimator.scala @@ -0,0 +1,117 @@ +package com.twitter.scalding.estimation.memory + +import com.twitter.scalding.estimation.{ FlowStepHistory, FlowStrategyInfo, HistoryEstimator, Task } +import org.apache.hadoop.mapred.JobConf +import org.slf4j.LoggerFactory + +// Tuple(MapMemory in MB for java process and container, ReduceMemory in MB for java process and container), +// or None to keep the default. +case class MemoryEstimate(mapMemoryInMB: Option[(Long, Long)], reduceMemoryInMB: Option[(Long, Long)]) + +object SmoothedHistoryMemoryEstimator { + val CommittedHeapBytes = "COMMITTED_HEAP_BYTES" + val CpuMs = "CPU_MILLISECONDS" + val PhysicalMemoryBytes = "PHYSICAL_MEMORY_BYTES" + val GCTimeMs = "GC_TIME_MILLIS" + + implicit class MemoryRichTask(val task: Task) extends AnyVal { + def committedHeapBytes: Option[Long] = task.counters.get(CommittedHeapBytes) + } +} + +trait SmoothedHistoryMemoryEstimator extends HistoryEstimator[MemoryEstimate] { + import SmoothedHistoryMemoryEstimator.MemoryRichTask + + private val LOG = LoggerFactory.getLogger(this.getClass) + + override def maxHistoryItems(conf: JobConf): Int = MemoryEstimatorConfig.getMaxHistory(conf) + + override protected def estimate(info: FlowStrategyInfo, conf: JobConf, history: Seq[FlowStepHistory]): Option[MemoryEstimate] = { + // iterate over mem history + // collect: for maps, list of max memory in past runs + // for reduce, list of max memory in past runs + // compute smoothed memory est + // multiple by scale factor + // return + val maxMemory = history.map(historyMemory) + + val xmxMemoryOfMapper = xmxMemory(maxMemory.flatMap(_._1), conf) + val xmxMemoryOfReducer = xmxMemory(maxMemory.flatMap(_._2), conf) + + val containerMemoryOfMapper = containerMemory(xmxMemoryOfMapper, conf) + val containerMemoryOfReducer = containerMemory(xmxMemoryOfReducer, conf) + + Some(MemoryEstimate( + cappedMemory(containerMemoryOfMapper, conf), + cappedMemory(containerMemoryOfReducer, conf))) + } + + private def xmxMemory(historyMemory: Seq[Long], conf: JobConf): Double = { + val scaleFactor = MemoryEstimatorConfig.getScaleFactor(conf) + val alpha = MemoryEstimatorConfig.getAlpha(conf) + + val smoothEstimation = smoothedAverage(historyMemory, alpha) + val scaledEstimation = smoothEstimation * scaleFactor + + //TODO handle gc + + LOG.info(s"Calculated xmx memory for: $historyMemory smoothAvg = $smoothEstimation, scaled: $scaledEstimation") + + scaledEstimation / (1024L * 1024) + } + + private def containerMemory(xmxMemory: Double, conf: JobConf): Double = { + xmxMemory * MemoryEstimatorConfig.getXmxScaleFactor(conf) + } + + private def cappedMemory(containerMemory: Double, conf: JobConf): Option[(Long, Long)] = { + val schedulerIncrement = MemoryEstimatorConfig.getYarnSchedulerIncrement(conf) + val roundedContainerMemory = roundUp(containerMemory, schedulerIncrement) + + val maxContainerMemory = MemoryEstimatorConfig.getMaxContainerMemory(conf) + val minContainerMemory = MemoryEstimatorConfig.getMinContainerMemory(conf) + val scaleFactor = MemoryEstimatorConfig.getXmxScaleFactor(conf) + + if (roundedContainerMemory == 0) { + None + } else if (roundedContainerMemory > maxContainerMemory) { + Some(((maxContainerMemory / scaleFactor).toLong, maxContainerMemory)) + } else if (roundedContainerMemory < minContainerMemory) { + Some(((minContainerMemory / scaleFactor).toLong, minContainerMemory)) + } else { + Some((roundedContainerMemory / scaleFactor).toLong, roundedContainerMemory) + } + } + + private def historyMemory(history: FlowStepHistory): (Option[Long], Option[Long]) = { + LOG.debug(s"Processing tasks: ${history.tasks}") + val reduceTasks: Seq[Task] = history.tasks.filter { t => t.taskType.contains("REDUCE") } + val mapTasks: Seq[Task] = history.tasks.filter { t => t.taskType.contains("MAP") } + + // handle empty task list due to either no task history / lack of reducers + val maxReduceCommittedHeap: Option[Long] = + if (reduceTasks.isEmpty) + None + else + Some(reduceTasks.flatMap(_.committedHeapBytes).max) + + val maxMapCommittedHeap: Option[Long] = + if (mapTasks.isEmpty) + None + else + Some(mapTasks.flatMap(_.committedHeapBytes).max) + + LOG.info(s"Calculated max committed heap for job: ${history.keys}, map: $maxMapCommittedHeap reduce: $maxReduceCommittedHeap") + (maxMapCommittedHeap, maxReduceCommittedHeap) + } + + // memoryEstimate = (currentMemoryValue * alpha) + (1 - alpha) * oldEstimate + private def smoothedAverage(memoryList: Seq[Long], alpha: Double): Double = + memoryList + .foldLeft(0.0) { (oldEstimate, currentVal) => + (currentVal * alpha) + (1 - alpha) * oldEstimate + } + + private def roundUp(value: Double, block: Double): Long = + (Math.ceil(value / block) * block).toLong +} \ No newline at end of file diff --git a/scalding-core/src/main/scala/com/twitter/scalding/memory_estimation/Common.scala b/scalding-core/src/main/scala/com/twitter/scalding/memory_estimation/Common.scala deleted file mode 100644 index adfdde7934..0000000000 --- a/scalding-core/src/main/scala/com/twitter/scalding/memory_estimation/Common.scala +++ /dev/null @@ -1,318 +0,0 @@ -package com.twitter.scalding.memory_estimation - -import cascading.flow.{ Flow, FlowStep, FlowStepStrategy } -import com.twitter.algebird.Monoid -import com.twitter.scalding.{ Config, StringUtility } -import com.twitter.scalding.reducer_estimation.{ FlowStepKeys, FlowStrategyInfo } -import java.util.{ List => JList } -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.mapred.JobConf -import org.slf4j.LoggerFactory -import scala.collection.JavaConverters._ -import scala.util.{ Failure, Success, Try } - -/** - * Info about a prior FlowStep, provided by implementers of HistoryService - */ -final case class FlowStepMemoryHistory(keys: FlowStepKeys, - submitTime: Long, - launchTime: Long, - finishTime: Long, - totalMaps: Long, - totalReduces: Long, - finishedMaps: Long, - finishedReduces: Long, - failedMaps: Long, - failedReduces: Long, - mapFileBytesRead: Long, - mapFileBytesWritten: Long, - reduceFileBytesRead: Long, - hdfsBytesRead: Long, - hdfsBytesWritten: Long, - mapperTimeMillis: Long, - reducerTimeMillis: Long, - reduceShuffleBytes: Long, - cost: Double, - tasks: Seq[Task]) - -final case class Task( - taskType: String, - committedHeapBytes: Long, - physicalMemoryBytes: Long, - cpuMilliSeconds: Long, - gcTimeMillis: Long) - -/** - * Provider of information about prior runs. - */ -trait MemoryService { - def fetchHistory(info: FlowStrategyInfo, maxHistory: Int): Try[Seq[FlowStepMemoryHistory]] -} - -object MemoryEstimatorConfig { - - /** Output param: what the original job map memory was. */ - val originalMapMemory = "scalding.map.memory.estimator.original" - - /** Output param: what the original job map memory was. */ - val originalReduceMemory = "scalding.reduce.memory.estimator.original" - - /** Maximum number of history items to use for memory estimation. */ - val maxHistoryKey = "scalding.memory.estimator.max.history" - - /** - * Value of alpha for exponential smoothing. - * Lower values ensure more smoothing and less importance to newer data - * Higher values provide lesser smoothing and more importance to newer data - */ - val alphaKey = "scalding.memory.estimator.alpha" - - /** Indicates how much to scale the memory estimate after it's calculated */ - val memoryScaleFactor = "scalding.memory.scale.factor" - - //yarn allocates in increments. So we might as well round up our container ask - val yarnSchedulerIncrementAllocationMB: String = "yarn.scheduler.increment-allocation-mb" - - def getMaxHistory(conf: JobConf): Int = conf.getInt(maxHistoryKey, 5) - - def getAlpha(conf: JobConf): Double = conf.getDouble(alphaKey, 1.0) - - def getScaleFactor(conf: JobConf): Double = conf.getDouble(memoryScaleFactor, 1.2) - - def getYarnSchedulerIncrement(conf: JobConf): Int = conf.getInt(yarnSchedulerIncrementAllocationMB, 512) - - // max container is 8GB, given XmxToMemoryScaleFactor we want to ensure this - // estimate when multiplied by that stays below 8GB - val maxMemoryEstimate: Double = 6500.0 * 1024 * 1024 - - // min container is 1G, multiplying by XmxToMemoryScaleFactor keeps us under the - // min container size - val minMemoryEstimate: Double = 800.0 * 1024 * 1024 - - val XmxToMemoryScaleFactor: Double = 1.25 - -} - -// Tuple(MapMemory in MB, ReduceMemory in MB), or None to keep the default. -case class MemoryEstimate(mapMemory: Option[Long], reduceMemory: Option[Long]) - -trait MemoryEstimator { - /** - * Estimate Map / Reduce memory settings. Called for each FlowStep before - * it is scheduled. Custom memory estimators should override this rather than - * apply() directly. - * - * @param info Holds information about the overall flow (.flow), - * previously-run steps (.predecessorSteps), - * and the current step (.step). - * @return MemoryEstimate. - */ - def estimateMemory(info: FlowStrategyInfo): Option[MemoryEstimate] -} - -trait HistoryMemoryEstimator extends MemoryEstimator { - - private val LOG = LoggerFactory.getLogger(this.getClass) - - def memoryService: MemoryService - - override def estimateMemory(info: FlowStrategyInfo): Option[MemoryEstimate] = { - val conf = info.step.getConfig - val maxHistory = MemoryEstimatorConfig.getMaxHistory(conf) - val alpha = MemoryEstimatorConfig.getAlpha(conf) - val scaleFactor = MemoryEstimatorConfig.getScaleFactor(conf) - val yarnSchedulerIncrementMB = MemoryEstimatorConfig.getYarnSchedulerIncrement(conf) - - LOG.info(s"Attempting to estimate memory settings with maxHistory: $maxHistory, alpha: $alpha, scaleFactor: $scaleFactor, schedulerIncrement: $yarnSchedulerIncrementMB") - - memoryService.fetchHistory(info, maxHistory) match { - case Success(h) if h.isEmpty => - LOG.warn("No matching history found.") - None - case Success(h) => - LOG.info(s"${h.length} history entries found.") - val estimate = estimateMemory(info, h, alpha, scaleFactor) - LOG.info(s"Memory estimate: $estimate") - Some(estimate) - case Failure(f) => - LOG.warn(s"Unable to fetch history in $getClass", f) - None - } - } - - private def estimateMemory(info: FlowStrategyInfo, history: Seq[FlowStepMemoryHistory], alpha: Double, scaleFactor: Double): MemoryEstimate = { - // iterate over mem history - // collect: for maps, list of max memory in past runs - // for reduce, list of max memory in past runs - // compute smoothed memory est - // multiple by scale factor - // cap estimate to max size if needed - // handle gc - // return - val maxMemoryList: Seq[(Option[Long], Option[Long])] = history.map{ h => getMapReduceMemory(h) } - val maxMapList: Seq[Long] = maxMemoryList.flatMap(_._1) - val maxReduceList: Seq[Long] = maxMemoryList.flatMap(_._2) - - val mapSmoothEst: Double = smoothedAverage(maxMapList, alpha) - val reduceSmoothEst: Double = smoothedAverage(maxReduceList, alpha) - - val mapScaledEst = mapSmoothEst * scaleFactor - val reduceScaledEst = reduceSmoothEst * scaleFactor - - //todo handle gc values - val cappedMapEst = cappedMemoryEstimateMB(mapScaledEst) - val cappedReduceEst = cappedMemoryEstimateMB(reduceScaledEst) - - LOG.info(s"Calculated map val for: $maxMapList smoothAvg = $mapSmoothEst, scaled: $mapScaledEst, capped: $cappedMapEst") - LOG.info(s"Calculated reduce val for: $maxReduceList smoothAvg = $reduceSmoothEst, scaled: $reduceScaledEst, capped: $cappedReduceEst") - - MemoryEstimate(cappedMapEst, cappedReduceEst) - } - - private def getMapReduceMemory(history: FlowStepMemoryHistory): (Option[Long], Option[Long]) = { - LOG.info(s"Processing tasks: ${history.tasks}") - val reduceTasks: Seq[Task] = history.tasks.filter { t => t.taskType == "REDUCE" } - val mapTasks: Seq[Task] = history.tasks.filter { t => t.taskType == "MAP" } - - // handle empty task list due to either no task history / lack of reducers - val maxReduceCommittedHeap: Option[Long] = if (reduceTasks.isEmpty) None else Some(reduceTasks.map(_.committedHeapBytes).max) - val maxMapCommittedHeap: Option[Long] = if (mapTasks.isEmpty) None else Some(mapTasks.map(_.committedHeapBytes).max) - - LOG.info(s"Calculated max committed heap for job: ${history.keys}, map: $maxMapCommittedHeap reduce: $maxReduceCommittedHeap") - (maxMapCommittedHeap, maxReduceCommittedHeap) - } - - // memoryEstimate = (currentMemoryValue * alpha) + (1 - alpha) * oldEstimate - private def smoothedAverage(memoryList: Seq[Long], alpha: Double): Double = { - memoryList.foldLeft(0.0){ (oldEstimate, currentVal) => (currentVal * alpha) + (1 - alpha) * oldEstimate } - } - - // calculate the capped Xmx memory estimate - private def cappedMemoryEstimateMB(memoryEst: Double): Option[Long] = { - val memoryEstimateBytes: Option[Double] = - if (memoryEst > MemoryEstimatorConfig.maxMemoryEstimate) - Some(MemoryEstimatorConfig.maxMemoryEstimate) - else if (memoryEst == 0) - None - else if (memoryEst < MemoryEstimatorConfig.minMemoryEstimate) - Some(MemoryEstimatorConfig.minMemoryEstimate) - else - Some(memoryEst) - - memoryEstimateBytes.map{ est => (est / (1024 * 1024)).toLong } - } - -} - -case class FallbackMemoryEstimator(first: MemoryEstimator, fallback: MemoryEstimator) extends MemoryEstimator { - private val LOG = LoggerFactory.getLogger(this.getClass) - - override def estimateMemory(info: FlowStrategyInfo): Option[MemoryEstimate] = - first.estimateMemory(info).orElse { - LOG.warn(s"$first estimator failed. Falling back to $fallback.") - fallback.estimateMemory(info) - } -} - -object MemoryEstimatorStepStrategy extends FlowStepStrategy[JobConf] { - - private val LOG = LoggerFactory.getLogger(this.getClass) - - implicit val estimatorMonoid: Monoid[MemoryEstimator] = new Monoid[MemoryEstimator] { - override def zero: MemoryEstimator = new MemoryEstimator { - override def estimateMemory(info: FlowStrategyInfo): Option[MemoryEstimate] = None - } - - override def plus(l: MemoryEstimator, r: MemoryEstimator): MemoryEstimator = - FallbackMemoryEstimator(l, r) - } - - /** - * Make memory estimate, possibly overriding explicitly-set memory settings, - * and save useful info (such as the default & estimate) in JobConf for - * later consumption. - * - * Called by Cascading at the start of each job step. - */ - final override def apply(flow: Flow[JobConf], - preds: JList[FlowStep[JobConf]], - step: FlowStep[JobConf]): Unit = { - - if (skipMemoryEstimation(step)) { - LOG.info(s"Skipping memory estimation as ${Config.MemoryEstimators} is not set ") - } else { - estimate(flow, preds, step) - } - } - - private def skipMemoryEstimation(step: FlowStep[JobConf]): Boolean = - step.getConfig.get(Config.MemoryEstimators) == null - - private def estimate(flow: Flow[JobConf], - preds: JList[FlowStep[JobConf]], - step: FlowStep[JobConf]): Unit = { - val conf = step.getConfig - - Option(conf.get(Config.MemoryEstimators)).foreach { clsNames => - - val clsLoader = Thread.currentThread.getContextClassLoader - - val estimators = StringUtility.fastSplit(clsNames, ",") - .map(clsLoader.loadClass(_).newInstance.asInstanceOf[MemoryEstimator]) - val combinedEstimator = Monoid.sum(estimators) - - val info = FlowStrategyInfo(flow, preds.asScala, step) - - // get memory estimate - val memoryEstimate: Option[MemoryEstimate] = combinedEstimator.estimateMemory(info) - - memoryEstimate match { - case Some(MemoryEstimate(Some(mapMem), Some(reduceMem))) => - LOG.info(s"Overriding map Xmx memory to: $mapMem and reduce Xmx memory to $reduceMem") - setMapMemory(mapMem, conf) - setReduceMemory(reduceMem, conf) - case Some(MemoryEstimate(Some(mapMem), _)) => - LOG.info(s"Overriding only map Xmx memory to: $mapMem") - setMapMemory(mapMem, conf) - case Some(MemoryEstimate(_, Some(reduceMem))) => - LOG.info(s"Overriding only reduce Xmx memory to $reduceMem") - setReduceMemory(reduceMem, conf) - case _ => LOG.info("Memory estimators didn't calculate any value. Skipping setting memory overrides") - } - } - } - - private def setMapMemory(mapMem: Long, conf: Configuration): Unit = { - conf.setLong(MemoryEstimatorConfig.originalMapMemory, conf.getLong(Config.MapMemory, 0L)) - - val mapContainerMem = mapMem * MemoryEstimatorConfig.XmxToMemoryScaleFactor - val schedulerIncrement = conf.getInt(MemoryEstimatorConfig.yarnSchedulerIncrementAllocationMB, 512) - val roundedMapContainerMem = roundUp(mapContainerMem, schedulerIncrement).toLong - - conf.setLong(Config.MapMemory, roundedMapContainerMem) - - val mapOpts = conf.get(Config.MapJavaOpts, "") - //remove existing xmx / xms - val mapOptsWithoutXm = mapOpts.split(" ").filterNot(s => s.startsWith("-Xmx") || s.startsWith("-Xms")).mkString(" ") - - conf.set(Config.MapJavaOpts, mapOptsWithoutXm + s" -Xmx${mapMem}M") - } - - private def setReduceMemory(reduceMem: Long, conf: Configuration): Unit = { - conf.setLong(MemoryEstimatorConfig.originalReduceMemory, conf.getLong(Config.ReduceMemory, 0L)) - - val reduceContainerMem = reduceMem * MemoryEstimatorConfig.XmxToMemoryScaleFactor - val schedulerIncrement = conf.getInt(MemoryEstimatorConfig.yarnSchedulerIncrementAllocationMB, 512) - val roundedReduceContainerMem = roundUp(reduceContainerMem, schedulerIncrement).toLong - conf.setLong(Config.ReduceMemory, roundedReduceContainerMem) - - val reduceOpts = conf.get(Config.ReduceJavaOpts, "") - //remove existing xmx / xms - val reduceOptsWithoutXm = reduceOpts.split(" ").filterNot(s => s.startsWith("-Xmx") || s.startsWith("-Xms")).mkString(" ") - - conf.set(Config.ReduceJavaOpts, reduceOptsWithoutXm + s" -Xmx${reduceMem}M") - } - - //Round up value to a multiple of block - private def roundUp(value: Double, block: Double): Double = Math.ceil(value / block) * block -} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala deleted file mode 100644 index c190731e2c..0000000000 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/Common.scala +++ /dev/null @@ -1,278 +0,0 @@ -package com.twitter.scalding.reducer_estimation - -import cascading.flow.{ Flow, FlowStep, FlowStepStrategy } -import cascading.tap.hadoop.Hfs -import cascading.tap.{ CompositeTap, Tap } -import com.twitter.algebird.Monoid -import com.twitter.scalding.tap.GlobHfs -import com.twitter.scalding.{ Config, StringUtility } -import java.util.{ List => JList } -import org.apache.hadoop.mapred.JobConf -import org.slf4j.LoggerFactory -import scala.collection.JavaConverters._ -import scala.util.{ Failure, Success, Try } - -object EstimatorConfig { - - /** Output param: what the Reducer Estimator recommended, regardless of if it was used. */ - val estimatedNumReducers = "scalding.reducer.estimator.result" - - /** - * Output param: same as estimatedNumReducers but with the cap specified by maxEstimatedReducersKey - * applied. Can be used to determine whether a cap was applied to the estimated number of reducers - * and potentially to trigger alerting / logging. - */ - val cappedEstimatedNumReducersKey = "scalding.reducer.estimator.result.capped" - - /** Output param: what the original job config was. */ - val originalNumReducers = "scalding.reducer.estimator.original.mapred.reduce.tasks" - - /** Maximum number of history items to use for reducer estimation. */ - val maxHistoryKey = "scalding.reducer.estimator.max.history" - - /** - * If we estimate more than this number of reducers, - * we will use this number instead of the estimated value - */ - val maxEstimatedReducersKey = "scalding.reducer.estimator.max.estimated.reducers" - - /* fairly arbitrary choice here -- you will probably want to configure this in your cluster defaults */ - val defaultMaxEstimatedReducers = 5000 - - def getMaxHistory(conf: JobConf): Int = conf.getInt(maxHistoryKey, 1) - -} - -object Common { - private[this] val LOG = LoggerFactory.getLogger(this.getClass) - - private def unrollTaps(taps: Seq[Tap[_, _, _]]): Seq[Tap[_, _, _]] = - taps.flatMap { - case multi: CompositeTap[_] => - unrollTaps(multi.getChildTaps.asScala.toSeq) - case t => Seq(t) - } - - def unrollTaps(step: FlowStep[JobConf]): Seq[Tap[_, _, _]] = - unrollTaps(step.getSources.asScala.toSeq) - - def inputSizes(step: FlowStep[JobConf]): Seq[(String, Long)] = { - val conf = step.getConfig - unrollTaps(step).flatMap { - case tap: GlobHfs => Some(tap.toString -> tap.getSize(conf)) - case tap: Hfs => Some(tap.toString -> GlobHfs.getSize(tap.getPath, conf)) - case tap => - LOG.warn("InputSizeReducerEstimator unable to calculate size: " + tap) - None - } - } - - def totalInputSize(step: FlowStep[JobConf]): Long = inputSizes(step).map(_._2).sum - -} - -case class FlowStrategyInfo( - flow: Flow[JobConf], - predecessorSteps: Seq[FlowStep[JobConf]], - step: FlowStep[JobConf]) - -trait ReducerEstimator { - /** - * Estimate how many reducers should be used. Called for each FlowStep before - * it is scheduled. Custom reducer estimators should override this rather than - * apply() directly. - * - * @param info Holds information about the overall flow (.flow), - * previously-run steps (.predecessorSteps), - * and the current step (.step). - * @return Number of reducers recommended by the estimator, or None to keep the default. - */ - def estimateReducers(info: FlowStrategyInfo): Option[Int] - -} - -trait HistoryReducerEstimator extends ReducerEstimator { - - private val LOG = LoggerFactory.getLogger(this.getClass) - - def historyService: HistoryService - - override def estimateReducers(info: FlowStrategyInfo): Option[Int] = { - val conf = info.step.getConfig - val maxHistory = EstimatorConfig.getMaxHistory(conf) - - historyService.fetchHistory(info, maxHistory) match { - case Success(h) if h.isEmpty => - LOG.warn("No matching history found.") - None - case Success(h) => - LOG.info(s"${h.length} history entries found.") - val estimate = estimateReducers(info, h) - LOG.info(s"Reducer estimate: $estimate") - estimate - case Failure(f) => - LOG.warn(s"Unable to fetch history in $getClass", f) - None - } - } - - protected def estimateReducers(info: FlowStrategyInfo, history: Seq[FlowStepHistory]): Option[Int] -} - -case class FallbackEstimator(first: ReducerEstimator, fallback: ReducerEstimator) extends ReducerEstimator { - private val LOG = LoggerFactory.getLogger(this.getClass) - - override def estimateReducers(info: FlowStrategyInfo): Option[Int] = - first.estimateReducers(info).orElse { - LOG.warn(s"$first estimator failed. Falling back to $fallback.") - fallback.estimateReducers(info) - } -} - -object ReducerEstimatorStepStrategy extends FlowStepStrategy[JobConf] { - - private val LOG = LoggerFactory.getLogger(this.getClass) - - implicit val estimatorMonoid: Monoid[ReducerEstimator] = new Monoid[ReducerEstimator] { - override def zero: ReducerEstimator = new ReducerEstimator { - override def estimateReducers(info: FlowStrategyInfo) = None - } - - override def plus(l: ReducerEstimator, r: ReducerEstimator): ReducerEstimator = - FallbackEstimator(l, r) - } - - /** - * Make reducer estimate, possibly overriding explicitly-set numReducers, - * and save useful info (such as the default & estimate) in JobConf for - * later consumption. - * - * Called by Cascading at the start of each job step. - */ - final override def apply(flow: Flow[JobConf], - preds: JList[FlowStep[JobConf]], - step: FlowStep[JobConf]): Unit = { - - val conf = step.getConfig - // for steps with reduce phase, mapred.reduce.tasks is set in the jobconf at this point - // so we check that to determine if this is a map-only step. - conf.getNumReduceTasks match { - case 0 => LOG.info(s"${flow.getName} is a map-only step. Skipping reducer estimation.") - case _ => - if (skipReducerEstimation(step)) { - LOG.info( - s""" - |Flow step ${step.getName} was configured with reducers - |set explicitly (${Config.WithReducersSetExplicitly}=true) and the estimator - |explicit override turned off (${Config.ReducerEstimatorOverride}=false). Skipping - |reducer estimation. - """.stripMargin) - } else { - estimate(flow, preds, step) - } - } - } - - // whether the reducers have been set explicitly with `withReducers` - private def reducersSetExplicitly(step: FlowStep[JobConf]) = - step.getConfig.getBoolean(Config.WithReducersSetExplicitly, false) - - // whether we should override explicitly-specified numReducers - private def overrideExplicitReducers(step: FlowStep[JobConf]) = - step.getConfig.getBoolean(Config.ReducerEstimatorOverride, false) - - private def skipReducerEstimation(step: FlowStep[JobConf]) = - reducersSetExplicitly(step) && !overrideExplicitReducers(step) - - private def estimate(flow: Flow[JobConf], - preds: JList[FlowStep[JobConf]], - step: FlowStep[JobConf]): Unit = { - val conf = step.getConfig - - val stepNumReducers = conf.get(Config.HadoopNumReducers) - Option(conf.get(Config.ReducerEstimators)).foreach { clsNames => - - val clsLoader = Thread.currentThread.getContextClassLoader - - val estimators = StringUtility.fastSplit(clsNames, ",") - .map(clsLoader.loadClass(_).newInstance.asInstanceOf[ReducerEstimator]) - val combinedEstimator = Monoid.sum(estimators) - - val info = FlowStrategyInfo(flow, preds.asScala, step) - - // get estimate - val estimatedNumReducers = combinedEstimator.estimateReducers(info) - - // apply cap if needed - val cappedNumReducers = estimatedNumReducers.map { n => - val configuredMax = conf.getInt(EstimatorConfig.maxEstimatedReducersKey, EstimatorConfig.defaultMaxEstimatedReducers) - - if (n > configuredMax) { - LOG.warn( - s""" - |Reducer estimator estimated $n reducers, which is more than the configured maximum of $configuredMax. - |Will use $configuredMax instead. - """.stripMargin) - configuredMax - } else { - n - } - } - - // save the estimate and capped estimate in the JobConf which should be saved by hRaven - conf.setInt(EstimatorConfig.estimatedNumReducers, estimatedNumReducers.getOrElse(-1)) - conf.setInt(EstimatorConfig.cappedEstimatedNumReducersKey, cappedNumReducers.getOrElse(-1)) - // set number of reducers - cappedNumReducers.foreach(conf.setNumReduceTasks) - // log in JobConf what was explicitly set by 'withReducers' - if (reducersSetExplicitly(step)) { - conf.set(EstimatorConfig.originalNumReducers, stepNumReducers) - } - } - } -} - -/** - * Info about a prior FlowStep, provided by implementers of HistoryService - */ -final case class FlowStepHistory(keys: FlowStepKeys, - submitTime: Long, - launchTime: Long, - finishTime: Long, - totalMaps: Long, - totalReduces: Long, - finishedMaps: Long, - finishedReduces: Long, - failedMaps: Long, - failedReduces: Long, - mapFileBytesRead: Long, - mapFileBytesWritten: Long, - mapOutputBytes: Long, - reduceFileBytesRead: Long, - hdfsBytesRead: Long, - hdfsBytesWritten: Long, - mapperTimeMillis: Long, - reducerTimeMillis: Long, - reduceShuffleBytes: Long, - cost: Double, - tasks: Seq[Task]) - -final case class FlowStepKeys(jobName: String, - user: String, - priority: String, - status: String, - version: String, - queue: String) - -final case class Task( - taskType: String, - status: String, - startTime: Long, - finishTime: Long) - -/** - * Provider of information about prior runs. - */ -trait HistoryService { - def fetchHistory(info: FlowStrategyInfo, maxHistory: Int): Try[Seq[FlowStepHistory]] -} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/InputSizeReducerEstimator.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/InputSizeReducerEstimator.scala index 3f68cbdbee..45e8cda0ce 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/InputSizeReducerEstimator.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/InputSizeReducerEstimator.scala @@ -1,6 +1,7 @@ package com.twitter.scalding.reducer_estimation import cascading.tap.hadoop.Hfs +import com.twitter.scalding.estimation.{ Common, Estimator, FlowStrategyInfo } import org.apache.hadoop.mapred.JobConf import org.slf4j.LoggerFactory @@ -63,9 +64,9 @@ object InputSizeReducerEstimator { * * Bytes per reducer can be configured with configuration parameter, defaults to 4 GB. */ -class InputSizeReducerEstimator extends ReducerEstimator { +class InputSizeReducerEstimator extends Estimator[Int] { import InputSizeReducerEstimator._ - override def estimateReducers(info: FlowStrategyInfo): Option[Int] = + override def estimate(info: FlowStrategyInfo): Option[Int] = estimateReducersWithoutRounding(info).map { _.ceil.toInt.max(1) } } \ No newline at end of file diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimator.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimator.scala index a895257472..abf76278b2 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimator.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimator.scala @@ -1,10 +1,9 @@ package com.twitter.scalding.reducer_estimation +import com.twitter.scalding.estimation.{Common, FlowStepHistory, FlowStrategyInfo} import org.apache.hadoop.mapred.JobConf import org.slf4j.LoggerFactory -import scala.util.{ Failure, Success } - object RatioBasedEstimator { /** * RatioBasedEstimator optionally ignores history items whose input size is @@ -16,10 +15,7 @@ object RatioBasedEstimator { def getInputRatioThreshold(conf: JobConf) = conf.getFloat(inputRatioThresholdKey, 0.10f) } -abstract class RatioBasedEstimator extends ReducerEstimator { - - def historyService: HistoryService - +abstract class RatioBasedEstimator extends ReducerHistoryEstimator { private val LOG = LoggerFactory.getLogger(this.getClass) /** @@ -42,51 +38,44 @@ abstract class RatioBasedEstimator extends ReducerEstimator { * Compute the average ratio of mapper bytes to reducer bytes and use that to * scale the estimate produced by InputSizeReducerEstimator. */ - override def estimateReducers(info: FlowStrategyInfo): Option[Int] = { - val conf = info.step.getConfig - val maxHistory = EstimatorConfig.getMaxHistory(conf) + override protected def estimate( + info: FlowStrategyInfo, + conf: JobConf, + history: Seq[FlowStepHistory] + ): Option[Int] = { val threshold = RatioBasedEstimator.getInputRatioThreshold(conf) + val inputBytes = Common.totalInputSize(info.step) - historyService.fetchHistory(info, maxHistory) match { - case Success(h) if h.isEmpty => - LOG.warn("No matching history found.") - None - case Success(history) => - val inputBytes = Common.totalInputSize(info.step) - - if (inputBytes == 0) { - LOG.warn("No input detected.") - None - } else { - val ratios = for { - h <- history - if h.mapOutputBytes > 0 - if acceptableInputRatio(inputBytes, h.hdfsBytesRead, threshold) - } yield h.mapOutputBytes / h.hdfsBytesRead.toDouble + if (inputBytes == 0) { + LOG.warn("No input detected.") + None + } else { + val ratios = for { + h <- history + if h.mapOutputBytes > 0 + if acceptableInputRatio(inputBytes, h.hdfsBytesRead, threshold) + } yield { + h.mapOutputBytes / h.hdfsBytesRead.toDouble + } - if (ratios.isEmpty) { - LOG.warn(s"No matching history found within input ratio threshold: $threshold") - None - } else { - val reducerRatio = ratios.sum / ratios.length - LOG.info("Getting base estimate from InputSizeReducerEstimator") - val inputSizeBasedEstimate = InputSizeReducerEstimator.estimateReducersWithoutRounding(info) - inputSizeBasedEstimate.map { baseEstimate => - // scale reducer estimate based on the historical input ratio - val e = (baseEstimate * reducerRatio).ceil.toInt.max(1) + if (ratios.isEmpty) { + LOG.warn(s"No matching history found within input ratio threshold: $threshold") + None + } else { + val reducerRatio = ratios.sum / ratios.length + LOG.info("Getting base estimate from InputSizeReducerEstimator") + val inputSizeBasedEstimate = InputSizeReducerEstimator.estimateReducersWithoutRounding(info) + inputSizeBasedEstimate.map { baseEstimate => + // scale reducer estimate based on the historical input ratio + val e = (baseEstimate * reducerRatio).ceil.toInt.max(1) - LOG.info("\nRatioBasedEstimator" - + "\n - past reducer ratio: " + reducerRatio - + "\n - reducer estimate: " + e) + LOG.info("\nRatioBasedEstimator" + + "\n - past reducer ratio: " + reducerRatio + + "\n - reducer estimate: " + e) - e - } - } + e } - case Failure(e) => - LOG.warn("Unable to fetch history. Disabling RatioBasedEstimator.", e) - None + } } } - } diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorConfig.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorConfig.scala new file mode 100644 index 0000000000..a8153e71d4 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorConfig.scala @@ -0,0 +1,33 @@ +package com.twitter.scalding.reducer_estimation + +import org.apache.hadoop.mapred.JobConf + +object ReducerEstimatorConfig { + + /** Output param: what the Reducer Estimator recommended, regardless of if it was used. */ + val estimatedNumReducers = "scalding.reducer.estimator.result" + + /** + * Output param: same as estimatedNumReducers but with the cap specified by maxEstimatedReducersKey + * applied. Can be used to determine whether a cap was applied to the estimated number of reducers + * and potentially to trigger alerting / logging. + */ + val cappedEstimatedNumReducersKey = "scalding.reducer.estimator.result.capped" + + /** Output param: what the original job config was. */ + val originalNumReducers = "scalding.reducer.estimator.original.mapred.reduce.tasks" + + /** + * If we estimate more than this number of reducers, + * we will use this number instead of the estimated value + */ + val maxEstimatedReducersKey = "scalding.reducer.estimator.max.estimated.reducers" + + /* fairly arbitrary choice here -- you will probably want to configure this in your cluster defaults */ + val defaultMaxEstimatedReducers = 5000 + + /** Maximum number of history items to use for reducer estimation. */ + val maxHistoryKey = "scalding.reducer.estimator.max.history" + + def getMaxHistory(conf: JobConf): Int = conf.getInt(maxHistoryKey, 1) +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorStepStrategy.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorStepStrategy.scala new file mode 100644 index 0000000000..61581587a0 --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorStepStrategy.scala @@ -0,0 +1,108 @@ +package com.twitter.scalding.reducer_estimation + +import cascading.flow.{ Flow, FlowStep, FlowStepStrategy } +import com.twitter.algebird.Monoid +import com.twitter.scalding.estimation.{ Estimator, FallbackEstimatorMonoid, FlowStrategyInfo } +import com.twitter.scalding.{ Config, StringUtility } +import java.util.{ List => JList } +import org.apache.hadoop.mapred.JobConf +import org.slf4j.LoggerFactory +import scala.collection.JavaConverters._ + +object ReducerEstimatorStepStrategy extends FlowStepStrategy[JobConf] { + + private val LOG = LoggerFactory.getLogger(this.getClass) + + implicit val estimatorMonoid = new FallbackEstimatorMonoid[Int] + + /** + * Make reducer estimate, possibly overriding explicitly-set numReducers, + * and save useful info (such as the default & estimate) in JobConf for + * later consumption. + * + * Called by Cascading at the start of each job step. + */ + final override def apply( + flow: Flow[JobConf], + preds: JList[FlowStep[JobConf]], + step: FlowStep[JobConf] + ): Unit = { + val conf = step.getConfig + // for steps with reduce phase, mapred.reduce.tasks is set in the jobconf at this point + // so we check that to determine if this is a map-only step. + conf.getNumReduceTasks match { + case 0 => LOG.info(s"${ flow.getName } is a map-only step. Skipping reducer estimation.") + case _ => + if (skipReducerEstimation(step)) { + LOG.info( + s""" + |Flow step ${ step.getName } was configured with reducers + |set explicitly (${ Config.WithReducersSetExplicitly }=true) and the estimator + |explicit override turned off (${ Config.ReducerEstimatorOverride }=false). Skipping + |reducer estimation. + """.stripMargin) + } else { + estimate(flow, preds.asScala, step) + } + } + } + + // whether the reducers have been set explicitly with `withReducers` + private def reducersSetExplicitly(step: FlowStep[JobConf]) = + step.getConfig.getBoolean(Config.WithReducersSetExplicitly, false) + + // whether we should override explicitly-specified numReducers + private def overrideExplicitReducers(step: FlowStep[JobConf]) = + step.getConfig.getBoolean(Config.ReducerEstimatorOverride, false) + + private def skipReducerEstimation(step: FlowStep[JobConf]) = + reducersSetExplicitly(step) && !overrideExplicitReducers(step) + + private def estimate( + flow: Flow[JobConf], + preds: Seq[FlowStep[JobConf]], + step: FlowStep[JobConf]): Unit = { + val conf = step.getConfig + + val stepNumReducers = conf.get(Config.HadoopNumReducers) + Option(conf.get(Config.ReducerEstimators)).foreach { clsNames => + + val clsLoader = Thread.currentThread.getContextClassLoader + + val estimators = StringUtility.fastSplit(clsNames, ",") + .map(clsLoader.loadClass(_).newInstance.asInstanceOf[Estimator[Int]]) + val combinedEstimator = Monoid.sum(estimators) + + val info = FlowStrategyInfo(flow, preds, step) + + // get estimate + val estimatedNumReducers = combinedEstimator.estimate(info) + + // apply cap if needed + val cappedNumReducers = estimatedNumReducers.map { n => + val configuredMax = conf.getInt(ReducerEstimatorConfig.maxEstimatedReducersKey, ReducerEstimatorConfig.defaultMaxEstimatedReducers) + + if (n > configuredMax) { + LOG.warn( + s""" + |Reducer estimator estimated $n reducers, which is more than the configured maximum of $configuredMax. + |Will use $configuredMax instead. + """.stripMargin) + configuredMax + } else { + n + } + } + + // save the estimate and capped estimate in the JobConf which should be saved by hRaven + conf.setInt(ReducerEstimatorConfig.estimatedNumReducers, estimatedNumReducers.getOrElse(-1)) + conf.setInt(ReducerEstimatorConfig.cappedEstimatedNumReducersKey, cappedNumReducers.getOrElse(-1)) + // set number of reducers + cappedNumReducers.foreach(conf.setNumReduceTasks) + // log in JobConf what was explicitly set by 'withReducers' + if (reducersSetExplicitly(step)) { + conf.set(ReducerEstimatorConfig.originalNumReducers, stepNumReducers) + } + } + } +} \ No newline at end of file diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/ReducerHistoryEstimator.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/ReducerHistoryEstimator.scala new file mode 100644 index 0000000000..4cffffee9c --- /dev/null +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/ReducerHistoryEstimator.scala @@ -0,0 +1,20 @@ +package com.twitter.scalding.reducer_estimation + +import com.twitter.scalding.estimation.{ HistoryEstimator, Task } +import org.apache.hadoop.mapred.JobConf + +object ReducerHistoryEstimator { + val Status = "status" + val StartTime = "startTime" + val FinishTime = "finishTime" + + implicit class ReducerRichTask(val task: Task) { + def status: Option[String] = task.details.get(Status).map(_.asInstanceOf[String]) + def startTime: Option[Long] = task.details.get(StartTime).map(_.asInstanceOf[Long]) + def finishTime: Option[Long] = task.details.get(FinishTime).map(_.asInstanceOf[Long]) + } +} + +trait ReducerHistoryEstimator extends HistoryEstimator[Int] { + override def maxHistoryItems(conf: JobConf): Int = ReducerEstimatorConfig.getMaxHistory(conf) +} diff --git a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimator.scala b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimator.scala index 15bf5a1bd6..c516434164 100644 --- a/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimator.scala +++ b/scalding-core/src/main/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimator.scala @@ -1,9 +1,45 @@ package com.twitter.scalding.reducer_estimation +import com.twitter.scalding.estimation.{ Common, FlowStepHistory, FlowStrategyInfo, HistoryEstimator } import org.apache.hadoop.mapred.JobConf import org.slf4j.LoggerFactory +/** + * Estimator that uses the input size and a fixed "bytesPerReducer" target. + * + * Bytes per reducer can be configured with configuration parameter, defaults to 1 GB. + */ +trait RuntimeEstimationScheme { + + /** + * Given a list of times that each reducer took in a certain FlowStep, + * aggregates these times into a single estimate of the time that + * a "typical" reducer took. + * Suggested implementation: mean or median. + */ + def estimateTaskTime(times: Seq[Double]): Option[Double] + + /** + * Given a list of "typical" times observed in a series of jobs of + * the same FlowStep, aggregates these times into a single estimate of + * the time that a "typical" reducer took in a "typical" job. + * Suggested implementation: mean or median. + */ + def estimateJobTime(times: Seq[Double]): Option[Double] +} + +object MedianEstimationScheme extends RuntimeEstimationScheme { + def estimateJobTime(times: Seq[Double]) = median(times) + def estimateTaskTime(times: Seq[Double]) = median(times) +} + +object MeanEstimationScheme extends RuntimeEstimationScheme { + def estimateJobTime(times: Seq[Double]) = mean(times) + def estimateTaskTime(times: Seq[Double]) = mean(times) +} + object RuntimeReducerEstimator { + import ReducerHistoryEstimator.ReducerRichTask val RuntimePerReducer = "scalding.reducer.estimator.runtime.per.reducer" val EstimationScheme = "scalding.reducer.estimator.runtime.estimation.scheme" @@ -44,54 +80,45 @@ object RuntimeReducerEstimator { def getReduceTimes(history: Seq[FlowStepHistory]): Seq[Seq[Double]] = history.map { h => h.tasks - .filter { t => t.taskType == "REDUCE" && t.status == "SUCCEEDED" && t.finishTime > t.startTime } - .map { t => (t.finishTime - t.startTime).toDouble } + .filter { t => t.taskType.contains("REDUCE") && t.status.contains("SUCCEEDED") } + .flatMap { t => + t.finishTime.zip(t.startTime) + .filter { + case (finishedTime, startTime) => + finishedTime > startTime + } + .map { + case (finishedTime, startTime) => + (finishedTime - startTime).toDouble + } + } } } -/** - * Estimator that uses the input size and a fixed "bytesPerReducer" target. - * - * Bytes per reducer can be configured with configuration parameter, defaults to 1 GB. - */ -trait RuntimeEstimationScheme { - - /** - * Given a list of times that each reducer took in a certain FlowStep, - * aggregates these times into a single estimate of the time that - * a "typical" reducer took. - * Suggested implementation: mean or median. - */ - def estimateTaskTime(times: Seq[Double]): Option[Double] - - /** - * Given a list of "typical" times observed in a series of jobs of - * the same FlowStep, aggregates these times into a single estimate of - * the time that a "typical" reducer took in a "typical" job. - * Suggested implementation: mean or median. - */ - def estimateJobTime(times: Seq[Double]): Option[Double] -} - -trait BasicRuntimeReducerEstimator extends HistoryReducerEstimator { +trait BasicRuntimeReducerEstimator extends ReducerHistoryEstimator { import RuntimeReducerEstimator._ private val LOG = LoggerFactory.getLogger(this.getClass) def runtimeEstimationScheme: RuntimeEstimationScheme - def estimateReducers(info: FlowStrategyInfo, history: Seq[FlowStepHistory]): Option[Int] = { + override protected def estimate( + info: FlowStrategyInfo, + conf: JobConf, + history: Seq[FlowStepHistory] + ): Option[Int] = { val reduceTimes: Seq[Seq[Double]] = getReduceTimes(history) LOG.info( s"""| |History items have the following numbers of tasks: - | ${history.map(_.tasks.length)}, + | ${ history.map(_.tasks.length) }, |and the following numbers of tasks have valid task histories: - | ${reduceTimes.map(_.length)}""".stripMargin) + | ${ reduceTimes.map(_.length) }""".stripMargin) // total time taken in the step = time per reducer * number of reducers - val jobTimes: Seq[Option[Double]] = reduceTimes.map { xs => runtimeEstimationScheme.estimateTaskTime(xs).map(_ * xs.length) } + val jobTimes: Seq[Option[Double]] = reduceTimes + .map { xs => runtimeEstimationScheme.estimateTaskTime(xs).map(_ * xs.length) } // time per step, averaged over all the steps val typicalJobTime: Option[Double] = runtimeEstimationScheme.estimateJobTime(jobTimes.flatten) @@ -100,43 +127,52 @@ trait BasicRuntimeReducerEstimator extends HistoryReducerEstimator { val estimate = typicalJobTime.map { t: Double => (t / desiredRuntime).ceil.toInt } - LOG.info(s""" - | - Typical job time: $typicalJobTime - | - Desired runtime: $desiredRuntime - | - Estimate: $estimate + LOG.info( + s""" + | - Typical job time: $typicalJobTime + | - Desired runtime: $desiredRuntime + | - Estimate: $estimate """.stripMargin) estimate } } -trait InputScaledRuntimeReducerEstimator extends HistoryReducerEstimator { +trait InputScaledRuntimeReducerEstimator extends ReducerHistoryEstimator { import RuntimeReducerEstimator._ private val LOG = LoggerFactory.getLogger(this.getClass) def runtimeEstimationScheme: RuntimeEstimationScheme - def estimateReducers(info: FlowStrategyInfo, history: Seq[FlowStepHistory]): Option[Int] = { + override protected def estimate( + info: FlowStrategyInfo, + conf: JobConf, + history: Seq[FlowStepHistory] + ): Option[Int] = { val reduceTimes: Seq[Seq[Double]] = getReduceTimes(history) LOG.info( s"""| |History items have the following numbers of tasks: - | ${history.map(_.tasks.length)}, + | ${ history.map(_.tasks.length) }, |and the following numbers of tasks have valid task histories: - | ${reduceTimes.map(_.length)}""".stripMargin) + | ${ reduceTimes.map(_.length) }""".stripMargin) // total time taken in the step = time per reducer * number of reducers - val jobTimes: Seq[Option[Double]] = reduceTimes.map { xs => runtimeEstimationScheme.estimateTaskTime(xs).map(_ * xs.length) } + val jobTimes: Seq[Option[Double]] = reduceTimes + .map { xs => runtimeEstimationScheme.estimateTaskTime(xs).map(_ * xs.length) } // time-to-byte ratio for a step = time per reducer * number of reducers / number of bytes - val timeToByteRatios: Seq[Double] = - jobTimes.zip { history.map(_.hdfsBytesRead) } - .collect { case (Some(time), bytes) => time / bytes } + val timeToByteRatios: Seq[Double] = jobTimes + .zip { + history.map(_.hdfsBytesRead) + } + .collect { case (Some(time), bytes) => time / bytes } // time-to-byte ratio, averaged over all the steps - val typicalTimeToByteRatio: Option[Double] = runtimeEstimationScheme.estimateJobTime(timeToByteRatios) + val typicalTimeToByteRatio: Option[Double] = runtimeEstimationScheme + .estimateJobTime(timeToByteRatios) val desiredRuntime = getRuntimePerReducer(info.step.getConfig) val inputBytes = Common.totalInputSize(info.step) @@ -150,28 +186,30 @@ trait InputScaledRuntimeReducerEstimator extends HistoryReducerEstimator { (t * inputBytes / desiredRuntime).ceil.toInt } - LOG.info(s""" - | - HDFS bytes read: ${history.map(_.hdfsBytesRead)} - | - Time-to-byte-ratios: $timeToByteRatios - | - Typical type-to-byte-ratio: $typicalTimeToByteRatio - | - Desired runtime: $desiredRuntime - | - Input bytes: $inputBytes - | - Estimate: $estimate + LOG.info( + s""" + | - HDFS bytes read: ${ history.map(_.hdfsBytesRead) } + | - Time-to-byte-ratios: $timeToByteRatios + | - Typical type-to-byte-ratio: $typicalTimeToByteRatio + | - Desired runtime: $desiredRuntime + | - Input bytes: $inputBytes + | - Estimate: $estimate """.stripMargin) estimate } } } -trait RuntimeReducerEstimator extends HistoryReducerEstimator { - def estimateReducers(info: FlowStrategyInfo, history: Seq[FlowStepHistory]): Option[Int] = { +trait RuntimeReducerEstimator extends ReducerHistoryEstimator { + override def estimate(info: FlowStrategyInfo): Option[Int] = { val estimationScheme = RuntimeReducerEstimator.getRuntimeEstimationScheme(info.step.getConfig) - val history = historyService + val runtimeHistoryService = historyService val basicEstimator = new BasicRuntimeReducerEstimator { def runtimeEstimationScheme = estimationScheme - def historyService = history + + def historyService = runtimeHistoryService } val combinedEstimator = if (RuntimeReducerEstimator.getRuntimeIgnoreInputSize(info.step.getConfig)) { @@ -179,21 +217,15 @@ trait RuntimeReducerEstimator extends HistoryReducerEstimator { } else { val inputScaledEstimator = new InputScaledRuntimeReducerEstimator { def runtimeEstimationScheme = estimationScheme - def historyService = history + + def historyService = runtimeHistoryService } ReducerEstimatorStepStrategy.estimatorMonoid.plus(inputScaledEstimator, basicEstimator) } - combinedEstimator.estimateReducers(info) + combinedEstimator.estimate(info) } -} -object MedianEstimationScheme extends RuntimeEstimationScheme { - def estimateJobTime(times: Seq[Double]) = median(times) - def estimateTaskTime(times: Seq[Double]) = median(times) -} - -object MeanEstimationScheme extends RuntimeEstimationScheme { - def estimateJobTime(times: Seq[Double]) = mean(times) - def estimateTaskTime(times: Seq[Double]) = mean(times) + override protected def estimate(info: FlowStrategyInfo, conf: JobConf, history: Seq[FlowStepHistory]): Option[Int] = + estimate(info) } diff --git a/scalding-core/src/test/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorStepStrategyTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorStepStrategyTest.scala new file mode 100644 index 0000000000..0b8ce6e6ec --- /dev/null +++ b/scalding-core/src/test/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorStepStrategyTest.scala @@ -0,0 +1,37 @@ +package com.twitter.scalding.estimation.memory + +import org.apache.hadoop.mapred.JobConf +import org.scalatest.{Matchers, WordSpec} + +class MemoryEstimatorStepStrategyTest extends WordSpec with Matchers { + "A Memory estimator step strategy" should { + "set xmx settings correctly" in { + val conf = confWith("test.opts", "-Xmx3500m -Djava.net.preferIPv4Stack=true -Xms34m") + + MemoryEstimatorStepStrategy.setXmxMemory("test.opts", 1024, conf) + + conf.get("test.opts") shouldBe "-Djava.net.preferIPv4Stack=true -Xmx1024m" + } + + "set xmx settings correctly with empty original config" in { + val conf = confWith(Map.empty) + + MemoryEstimatorStepStrategy.setXmxMemory("test.opts", 1024, conf) + + conf.get("test.opts") shouldBe " -Xmx1024m" + } + } + + def confWith(key: String, value: String): JobConf = + confWith(Map(key -> value)) + + def confWith(values: Map[String, String]): JobConf = { + val conf = new JobConf(false) + + values.foreach { case (k, v) => + conf.set(k, v) + } + + conf + } +} diff --git a/scalding-core/src/test/scala/com/twitter/scalding/estimation/memory/SmoothedHistoryMemoryEstimatorTest.scala b/scalding-core/src/test/scala/com/twitter/scalding/estimation/memory/SmoothedHistoryMemoryEstimatorTest.scala new file mode 100644 index 0000000000..8184e1d369 --- /dev/null +++ b/scalding-core/src/test/scala/com/twitter/scalding/estimation/memory/SmoothedHistoryMemoryEstimatorTest.scala @@ -0,0 +1,177 @@ +package com.twitter.scalding.estimation.memory + +import cascading.flow.FlowStep +import com.twitter.scalding.estimation.{FlowStepHistory, FlowStrategyInfo, HistoryService, Task} +import org.apache.hadoop.mapred.JobConf +import org.mockito.Mockito._ +import org.mockito.Matchers._ +import org.scalatest.{Matchers, WordSpec} +import scala.util.{Success, Try} + +class SmoothedHistoryMemoryEstimatorTest extends WordSpec with Matchers { + import Utils._ + + "A memory history estimator" should { + "return None without history" in { + SmoothedMemoryEstimator.empty.estimate(TestFlowStrategyInfo.dummy) shouldBe None + } + + "estimate correct numbers for only reducers" in { + val estimation = SmoothedMemoryEstimator + .makeHistory(Seq( + "REDUCE" -> 1024.megabytes + )) + .estimate(TestFlowStrategyInfo.dummy) + + estimation shouldBe reduceEstimate((1228, 1536)) + } + + "estimate correct numbers for only mappers" in { + val estimation = SmoothedMemoryEstimator + .makeHistory(Seq( + "MAP" -> 1024.megabytes + )) + .estimate(TestFlowStrategyInfo.dummy) + + estimation shouldBe mapEstimate((1228, 1536)) + } + + "estimate correct numbers" in { + val estimation = SmoothedMemoryEstimator + .makeHistory(Seq( + "MAP" -> 800.megabytes, + "REDUCE" -> 800.megabytes, + "MAP" -> 1024.megabytes, + "REDUCE" -> 1024.megabytes, + "MAP" -> 1300.megabytes, + "REDUCE" -> 1300.megabytes, + "MAP" -> 723.megabytes, + "REDUCE" -> 723.megabytes + )) + .estimate(TestFlowStrategyInfo.dummy) + + estimation shouldBe Some(MemoryEstimate(Some((1228, 1536)), Some((1228, 1536)))) + } + + "estimate less than max cap" in { + val conf = TestFlowStrategyInfo.dummy.step.getConfig + val estimation = SmoothedMemoryEstimator + .makeHistory(Seq( + "MAP" -> (MemoryEstimatorConfig.getMaxContainerMemory(conf).megabyte + 1.gigabyte) + )) + .estimate(TestFlowStrategyInfo.dummy) + + val expectedEstimation = ( + (MemoryEstimatorConfig.getMaxContainerMemory(conf) / MemoryEstimatorConfig.getXmxScaleFactor(conf)).toLong, + MemoryEstimatorConfig.getMaxContainerMemory(conf) + ) + + estimation shouldBe mapEstimate(expectedEstimation) + } + + "estimate not less than min cap" in { + val conf = TestFlowStrategyInfo.dummy.step.getConfig + val estimation = SmoothedMemoryEstimator + .makeHistory(Seq( + "MAP" -> (MemoryEstimatorConfig.getMinContainerMemory(conf).megabyte - 500.megabyte) + )) + .estimate(TestFlowStrategyInfo.dummy) + + val expectedEstimation = ( + (MemoryEstimatorConfig.getMinContainerMemory(conf) / MemoryEstimatorConfig.getXmxScaleFactor(conf)).toLong, + MemoryEstimatorConfig.getMinContainerMemory(conf) + ) + + estimation shouldBe mapEstimate(expectedEstimation) + } + } +} + +object EmptyHistoryService extends HistoryService { + override def fetchHistory(info: FlowStrategyInfo, maxHistory: Int): Try[Seq[FlowStepHistory]] = + Success(Seq.empty) +} + +class DummyHistoryService(val history: Seq[(String, Long)]) extends HistoryService { + override def fetchHistory(info: FlowStrategyInfo, maxHistory: Int): Try[Seq[FlowStepHistory]] = { + Success(history.map { case (taskType, memory) => + val task = Task( + details = Map( + Task.TaskType -> taskType + ), + counters = Map( + SmoothedHistoryMemoryEstimator.CommittedHeapBytes -> memory + ) + ) + val tasks = Seq(task) + FlowStepHistory( + keys = null, + submitTimeMillis = 0, + launchTimeMillis = 0L, + finishTimeMillis = 0L, + totalMaps = 0L, + totalReduces = 0L, + finishedMaps = 0L, + finishedReduces = 0L, + failedMaps = 0L, + failedReduces = 0L, + mapFileBytesRead = 0L, + mapFileBytesWritten = 0L, + mapOutputBytes = 0l, + reduceFileBytesRead = 0l, + hdfsBytesRead = 0l, + hdfsBytesWritten = 0L, + mapperTimeMillis = 0L, + reducerTimeMillis = 0L, + reduceShuffleBytes = 0L, + cost = 1.1, + tasks = tasks + ) + }) + } +} + +class SmoothedMemoryEstimator(override val historyService: HistoryService) extends SmoothedHistoryMemoryEstimator + +object SmoothedMemoryEstimator { + def empty: SmoothedMemoryEstimator = new SmoothedMemoryEstimator(EmptyHistoryService) + + def makeHistory(history: Seq[(String, Long)]): SmoothedMemoryEstimator = + new SmoothedMemoryEstimator(new DummyHistoryService(history)) +} + +object TestFlowStrategyInfo { + def dummy: FlowStrategyInfo = { + val mockedConf = spy(new JobConf()) + val mockedStep = mock(classOf[FlowStep[JobConf]]) + val mockedInfo = mock(classOf[FlowStrategyInfo]) + + when(mockedConf.get(anyString())).thenReturn(null) + when(mockedStep.getConfig).thenReturn(mockedConf) + when(mockedInfo.step).thenReturn(mockedStep) + + mockedInfo + } +} + +object Utils { + implicit class StorageUnit(val wrapped: Long) extends AnyVal { + def fromMegabytes(megabytes: Long): Long = megabytes * 1024 * 1024 + def fromGigabytes(gigabytes: Long): Long = gigabytes * 1024 * 1024 * 1024 + + def megabyte: Long = megabytes + def megabytes: Long = fromMegabytes(wrapped) + def gigabyte: Long = gigabytes + def gigabytes: Long = fromGigabytes(wrapped) + + def inMegabytes: Long = wrapped / (1024L * 1024) + } + + implicit def doubleToLong(value: Double): StorageUnit = new StorageUnit(value.toLong) + + def mapEstimate(value: (Long, Long)): Some[MemoryEstimate] = + Some(MemoryEstimate(mapMemoryInMB = Some(value), None)) + + def reduceEstimate(value: (Long, Long)): Some[MemoryEstimate] = + Some(MemoryEstimate(None, reduceMemoryInMB = Some(value))) +} \ No newline at end of file diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorTest.scala new file mode 100644 index 0000000000..ee4d65b123 --- /dev/null +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorTest.scala @@ -0,0 +1,243 @@ +package com.twitter.scalding.estimation.memory + +import com.twitter.scalding.Config +import com.twitter.scalding.estimation.{ FlowStepHistory, FlowStrategyInfo, HistoryService, Task } +import com.twitter.scalding.platform.{ HadoopPlatformJobTest, HadoopSharedPlatformTest } +import com.twitter.scalding.reducer_estimation._ +import org.apache.hadoop.mapred.JobConf +import org.scalatest.{ Matchers, WordSpec } +import scala.collection.JavaConverters._ +import scala.util.{ Success, Try } + +class MemoryEstimatorTest extends WordSpec with Matchers with HadoopSharedPlatformTest { + "Single-step job with memory estimator" should { + "without history don't override memory settings" in { + val customConfig = Config.empty + + (Config.MemoryEstimators -> classOf[EmptySmoothedMemoryEstimator].getName) + + HadoopPlatformJobTest(new SimpleJob(_, customConfig), cluster) + .inspectCompletedFlow { flow => + val steps = flow.getFlowSteps.asScala + steps should have size 1 + + val conf = Config.fromHadoop(steps.head.getConfig) + + conf.get(Config.MapMemory) shouldBe None + conf.get(Config.MapJavaOpts) shouldBe None + conf.get(Config.ReduceMemory) shouldBe None + conf.get(Config.ReduceJavaOpts) shouldBe None + } + .run() + } + + "run with correct number of memory" in { + val customConfig = Config.empty + + (Config.MemoryEstimators -> classOf[SmoothedMemoryEstimatorWithData].getName) + + HadoopPlatformJobTest(new SimpleJob(_, customConfig), cluster) + .inspectCompletedFlow { flow => + val steps = flow.getFlowSteps.asScala + steps should have size 1 + + val conf = Config.fromHadoop(steps.head.getConfig) + + conf.get(Config.MapMemory) shouldBe Some("1536") + conf.get(Config.MapJavaOpts) shouldBe Some(" -Xmx1228m") + conf.get(Config.ReduceMemory) shouldBe Some("1536") + conf.get(Config.ReduceJavaOpts) shouldBe Some(" -Xmx1228m") + } + .run() + } + + "respect cap when estimated memory is above the configured max" in { + val customConfig = Config.empty + + (Config.MemoryEstimators -> classOf[SmoothedMemoryEstimatorWithMoreThanMaxCap].getName) + + HadoopPlatformJobTest(new SimpleJob(_, customConfig), cluster) + .inspectCompletedFlow { flow => + val steps = flow.getFlowSteps.asScala + steps should have size 1 + + val conf = Config.fromHadoop(steps.head.getConfig) + + conf.get(Config.MapMemory) shouldBe Some("8192") + conf.get(Config.MapJavaOpts) shouldBe Some(" -Xmx6553m") + conf.get(Config.ReduceMemory) shouldBe Some("8192") + conf.get(Config.ReduceJavaOpts) shouldBe Some(" -Xmx6553m") + } + .run() + } + + "respect cap when estimated memory is below the configured min" in { + val customConfig = Config.empty + + (Config.MemoryEstimators -> classOf[SmoothedMemoryEstimatorWithLessThanMinCap].getName) + + HadoopPlatformJobTest(new SimpleJob(_, customConfig), cluster) + .inspectCompletedFlow { flow => + val steps = flow.getFlowSteps.asScala + steps should have size 1 + + val conf = Config.fromHadoop(steps.head.getConfig) + + conf.get(Config.MapMemory) shouldBe Some("1024") + conf.get(Config.MapJavaOpts) shouldBe Some(" -Xmx819m") + conf.get(Config.ReduceMemory) shouldBe Some("1024") + conf.get(Config.ReduceJavaOpts) shouldBe Some(" -Xmx819m") + } + .run() + } + + "not set memory when error fetching history" in { + val customConfig = Config.empty + + (Config.MemoryEstimators -> classOf[ErrorHistoryBasedMemoryEstimator].getName) + + HadoopPlatformJobTest(new SimpleJobWithNoSetReducers(_, customConfig), cluster) + .inspectCompletedFlow { flow => + val steps = flow.getFlowSteps.asScala + steps should have size 1 + + val conf = Config.fromHadoop(steps.head.getConfig) + + conf.get(Config.MapMemory) shouldBe None + conf.get(Config.MapJavaOpts) shouldBe None + conf.get(Config.ReduceMemory) shouldBe None + conf.get(Config.ReduceJavaOpts) shouldBe None + + } + .run() + } + } + + "Multi-step job with memory estimator" should { + "run with correct number of memory in each step" in { + val customConfig = Config.empty + + (Config.MemoryEstimators -> classOf[SmoothedMemoryEstimatorWithData].getName) + + HadoopPlatformJobTest(new HipJob(_, customConfig), cluster) + .inspectCompletedFlow { flow => + val steps = flow.getFlowSteps.asScala + + val mapsMemory = steps.map(_.getConfig.getInt(Config.MapMemory, 0)).toList + val mapsJavaOpts = steps.map(_.getConfig.get(Config.MapJavaOpts, "")).toList + + mapsMemory shouldBe List(1536, 0, 1024) + mapsJavaOpts shouldBe List(" -Xmx1228m", "", " -Xmx819m") + + val reducersMemory = steps.map(_.getConfig.getInt(Config.ReduceMemory, 0)).toList + val reducersJavaOpts = steps.map(_.getConfig.get(Config.ReduceJavaOpts, "")).toList + + reducersMemory shouldBe List(1536, 0, 1024) + reducersJavaOpts shouldBe List(" -Xmx1228m", "", " -Xmx819m") + } + .run() + } + } +} + +object EmptyHistoryService extends HistoryService { + override def fetchHistory(info: FlowStrategyInfo, maxHistory: Int): Try[Seq[FlowStepHistory]] = + Success(Seq.empty) +} + +class CustomHistoryService(val history: JobConf => Seq[(String, Long)]) extends HistoryService { + import Utils._ + + override def fetchHistory(info: FlowStrategyInfo, maxHistory: Int): Try[Seq[FlowStepHistory]] = { + if (info.step.getStepNum == 1) { + makeHistory(info.step.getConfig, history) + } else if (info.step.getStepNum == 2) { + Success(Nil) + } else { + makeHistory(info.step.getConfig, _ => Seq( + "MAP" -> 512.megabyte, + "REDUCE" -> 512.megabyte)) + } + } + + def makeHistory(conf: JobConf, history: JobConf => Seq[(String, Long)]): Success[Seq[FlowStepHistory]] = + Success(history(conf).map { + case (taskType, memory) => + val task = Task( + details = Map( + Task.TaskType -> taskType), + counters = Map( + SmoothedHistoryMemoryEstimator.CommittedHeapBytes -> memory)) + val tasks = Seq(task) + FlowStepHistory( + keys = null, + submitTimeMillis = 0, + launchTimeMillis = 0L, + finishTimeMillis = 0L, + totalMaps = 0L, + totalReduces = 0L, + finishedMaps = 0L, + finishedReduces = 0L, + failedMaps = 0L, + failedReduces = 0L, + mapFileBytesRead = 0L, + mapFileBytesWritten = 0L, + mapOutputBytes = 0l, + reduceFileBytesRead = 0l, + hdfsBytesRead = 0l, + hdfsBytesWritten = 0L, + mapperTimeMillis = 0L, + reducerTimeMillis = 0L, + reduceShuffleBytes = 0L, + cost = 1.1, + tasks = tasks) + }) +} + +class EmptySmoothedMemoryEstimator extends SmoothedHistoryMemoryEstimator { + override def historyService: HistoryService = EmptyHistoryService +} + +class SmoothedMemoryEstimatorWithData extends SmoothedHistoryMemoryEstimator { + import Utils._ + + override def historyService: HistoryService = new CustomHistoryService(_ => Seq( + "MAP" -> 800.megabytes, + "REDUCE" -> 800.megabytes, + "MAP" -> 1024.megabytes, + "REDUCE" -> 1024.megabytes, + "MAP" -> 1300.megabytes, + "REDUCE" -> 1300.megabytes, + "MAP" -> 723.megabytes, + "REDUCE" -> 723.megabytes)) +} + +class SmoothedMemoryEstimatorWithMoreThanMaxCap extends SmoothedHistoryMemoryEstimator { + import Utils._ + + override def historyService: HistoryService = new CustomHistoryService(conf => Seq( + "MAP" -> (MemoryEstimatorConfig.getMaxContainerMemory(conf).megabyte + 1.gigabyte), + "REDUCE" -> (MemoryEstimatorConfig.getMaxContainerMemory(conf).megabyte + 1.gigabyte))) +} + +class SmoothedMemoryEstimatorWithLessThanMinCap extends SmoothedHistoryMemoryEstimator { + import Utils._ + + override def historyService: HistoryService = new CustomHistoryService(conf => Seq( + "MAP" -> (MemoryEstimatorConfig.getMinContainerMemory(conf).megabyte - 500.megabyte), + "REDUCE" -> (MemoryEstimatorConfig.getMinContainerMemory(conf).megabyte - 500.megabyte))) +} + +class ErrorHistoryBasedMemoryEstimator extends SmoothedHistoryMemoryEstimator { + override val historyService = ErrorHistoryService +} + +object Utils { + implicit class StorageUnit(val wrapped: Long) extends AnyVal { + def fromMegabytes(megabytes: Long): Long = megabytes * 1024 * 1024 + def fromGigabytes(gigabytes: Long): Long = gigabytes * 1024 * 1024 * 1024 + + def megabyte: Long = megabytes + def megabytes: Long = fromMegabytes(wrapped) + def gigabyte: Long = gigabytes + def gigabytes: Long = fromGigabytes(wrapped) + + def inMegabytes: Long = wrapped / (1024L * 1024) + } + + implicit def doubleToLong(value: Double): StorageUnit = new StorageUnit(value.toLong) +} \ No newline at end of file diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala index 1ee2d281f4..5c458cf5bc 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala @@ -1,6 +1,7 @@ package com.twitter.scalding.reducer_estimation import com.twitter.scalding._ +import com.twitter.scalding.estimation.{ FlowStepHistory, FlowStrategyInfo, HistoryService, Task } import com.twitter.scalding.platform.{ HadoopPlatformJobTest, HadoopSharedPlatformTest } import org.scalatest.{ Matchers, WordSpec } import scala.collection.JavaConverters._ @@ -24,11 +25,12 @@ object EmptyHistoryService extends HistoryService { } object ErrorHistoryService extends HistoryService { - def fetchHistory(info: FlowStrategyInfo, maxHistory: Int): Try[Seq[FlowStepHistory]] = + override def fetchHistory(info: FlowStrategyInfo, maxHistory: Int): Try[Seq[FlowStepHistory]] = Failure(new RuntimeException("Failed to fetch job history")) } object HistoryServiceWithData { + import ReducerHistoryEstimator._ // we only care about these two input size fields for RatioBasedEstimator def makeHistory(inputHdfsBytesRead: Long, mapOutputBytes: Long): FlowStepHistory = @@ -39,17 +41,19 @@ object HistoryServiceWithData { val tasks = taskRuntimes.map { time => val startTime = random.nextLong Task( - taskType = "REDUCE", - status = "SUCCEEDED", - startTime = startTime, - finishTime = startTime + time) + details = Map( + Task.TaskType -> "REDUCE", + Status -> "SUCCEEDED", + StartTime -> startTime, + FinishTime -> (startTime + time)), + Map.empty) } FlowStepHistory( keys = null, - submitTime = 0, - launchTime = 0L, - finishTime = 0L, + submitTimeMillis = 0, + launchTimeMillis = 0L, + finishTimeMillis = 0L, totalMaps = 0L, totalReduces = 0L, finishedMaps = 0L, @@ -72,9 +76,7 @@ object HistoryServiceWithData { def inputSize = HipJob.InSrcFileSize } -abstract class HistoryServiceWithData extends HistoryService - -object ValidHistoryService extends HistoryServiceWithData { +object ValidHistoryService extends HistoryService { import HistoryServiceWithData._ def fetchHistory(info: FlowStrategyInfo, maxHistory: Int): Try[Seq[FlowStepHistory]] = @@ -87,7 +89,7 @@ object ValidHistoryService extends HistoryServiceWithData { makeHistory(inputSize, inputSize / 2))) } -object SmallDataExplosionHistoryService extends HistoryServiceWithData { +object SmallDataExplosionHistoryService extends HistoryService { import HistoryServiceWithData._ def fetchHistory(info: FlowStrategyInfo, maxHistory: Int): Try[Seq[FlowStepHistory]] = { @@ -103,7 +105,7 @@ object SmallDataExplosionHistoryService extends HistoryServiceWithData { } } -object InvalidHistoryService extends HistoryServiceWithData { +object InvalidHistoryService extends HistoryService { import HistoryServiceWithData._ def fetchHistory(info: FlowStrategyInfo, maxHistory: Int): Try[Seq[FlowStepHistory]] = diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala index 6b07958f65..40d8887112 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala @@ -3,12 +3,10 @@ package com.twitter.scalding.reducer_estimation import cascading.flow.FlowException import com.twitter.scalding._ import com.twitter.scalding.platform.{ HadoopPlatformJobTest, HadoopSharedPlatformTest } +import java.io.FileNotFoundException import org.scalatest.{ Matchers, WordSpec } - import scala.collection.JavaConverters._ -import java.io.FileNotFoundException - object HipJob { val InSrcFileSize = 2496L val inPath = getClass.getResource("/hipster.txt") // file size is 2496 bytes @@ -88,8 +86,7 @@ class SimpleMemoryJob(args: Args, customConfig: Config) extends Job(args) { val inSrc = IterableSource(List( "Direct trade American Apparel squid umami tote bag. Lo-fi XOXO gluten-free meh literally, typewriter readymade wolf salvia whatever drinking vinegar organic. Four loko literally bicycle rights drinking vinegar Cosby sweater hella stumptown. Dreamcatcher iPhone 90's organic chambray cardigan, wolf fixie gluten-free Brooklyn four loko. Mumblecore ennui twee, 8-bit food truck sustainable tote bag Williamsburg mixtape biodiesel. Semiotics Helvetica put a bird on it, roof party fashion axe organic post-ironic readymade Wes Anderson Pinterest keffiyeh. Craft beer meggings sartorial, butcher Marfa kitsch art party mustache Brooklyn vinyl.", - "Wolf flannel before they sold out vinyl, selfies four loko Bushwick Banksy Odd Future. Chillwave banh mi iPhone, Truffaut shabby chic craft beer keytar DIY. Scenester selvage deep v YOLO paleo blog photo booth fap. Sustainable wolf mixtape small batch skateboard, pop-up brunch asymmetrical seitan butcher Thundercats disrupt twee Etsy. You probably haven't heard of them freegan skateboard before they sold out, mlkshk pour-over Echo Park keytar retro farm-to-table. Tattooed sustainable beard, Helvetica Wes Anderson pickled vinyl yr pop-up Vice. Wolf bespoke lomo photo booth ethnic cliche." - )) + "Wolf flannel before they sold out vinyl, selfies four loko Bushwick Banksy Odd Future. Chillwave banh mi iPhone, Truffaut shabby chic craft beer keytar DIY. Scenester selvage deep v YOLO paleo blog photo booth fap. Sustainable wolf mixtape small batch skateboard, pop-up brunch asymmetrical seitan butcher Thundercats disrupt twee Etsy. You probably haven't heard of them freegan skateboard before they sold out, mlkshk pour-over Echo Park keytar retro farm-to-table. Tattooed sustainable beard, Helvetica Wes Anderson pickled vinyl yr pop-up Vice. Wolf bespoke lomo photo booth ethnic cliche.")) override def config = super.config ++ customConfig.toMap.toMap @@ -159,7 +156,7 @@ class ReducerEstimatorTest extends WordSpec with Matchers with HadoopSharedPlatf val conf = Config.fromHadoop(steps.head.getConfig) conf.getNumReducers should contain (2) - conf.get(EstimatorConfig.originalNumReducers) should be (None) + conf.get(ReducerEstimatorConfig.originalNumReducers) should be (None) } .run() } @@ -176,7 +173,7 @@ class ReducerEstimatorTest extends WordSpec with Matchers with HadoopSharedPlatf val conf = Config.fromHadoop(steps.head.getConfig) conf.getNumReducers should contain (3) - conf.get(EstimatorConfig.originalNumReducers) should contain ("2") + conf.get(ReducerEstimatorConfig.originalNumReducers) should contain ("2") } .run() } @@ -193,7 +190,7 @@ class ReducerEstimatorTest extends WordSpec with Matchers with HadoopSharedPlatf val conf = Config.fromHadoop(steps.head.getConfig) conf.getNumReducers should contain (3) - conf.get(EstimatorConfig.originalNumReducers) should contain ("2") + conf.get(ReducerEstimatorConfig.originalNumReducers) should contain ("2") } .run() } @@ -203,7 +200,7 @@ class ReducerEstimatorTest extends WordSpec with Matchers with HadoopSharedPlatf (Config.ReducerEstimatorOverride -> "true") + // 1 reducer per byte, should give us a large number (InputSizeReducerEstimator.BytesPerReducer -> 1.toString) + - (EstimatorConfig.maxEstimatedReducersKey -> 10.toString) + (ReducerEstimatorConfig.maxEstimatedReducersKey -> 10.toString) HadoopPlatformJobTest(new SimpleJob(_, customConfig), cluster) .inspectCompletedFlow { flow => @@ -211,8 +208,8 @@ class ReducerEstimatorTest extends WordSpec with Matchers with HadoopSharedPlatf steps should have size 1 val conf = Config.fromHadoop(steps.head.getConfig) - conf.get(EstimatorConfig.estimatedNumReducers) should contain ("2496") - conf.get(EstimatorConfig.cappedEstimatedNumReducersKey) should contain ("10") + conf.get(ReducerEstimatorConfig.estimatedNumReducers) should contain ("2496") + conf.get(ReducerEstimatorConfig.cappedEstimatedNumReducersKey) should contain ("10") conf.getNumReducers should contain (10) } .run() @@ -230,7 +227,7 @@ class ReducerEstimatorTest extends WordSpec with Matchers with HadoopSharedPlatf val conf = Config.fromHadoop(steps.head.getConfig) conf.getNumReducers should contain (2) - conf.get(EstimatorConfig.originalNumReducers) should contain ("2") + conf.get(ReducerEstimatorConfig.originalNumReducers) should contain ("2") } .run() } @@ -241,8 +238,9 @@ class ReducerEstimatorTest extends WordSpec with Matchers with HadoopSharedPlatf (Config.ReducerEstimatorOverride -> "true") HadoopPlatformJobTest(new SimpleFileNotFoundJob(_, customConfig), cluster) - .runExpectFailure { case error: FlowException => - error.getCause.getClass should be(classOf[FileNotFoundException]) + .runExpectFailure { + case error: FlowException => + error.getCause.getClass should be(classOf[FileNotFoundException]) } } } diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala index d911272f50..b41cd95b7e 100644 --- a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala +++ b/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala @@ -1,13 +1,14 @@ package com.twitter.scalding.reducer_estimation import com.twitter.scalding._ -import com.twitter.scalding.reducer_estimation.RuntimeReducerEstimator.{ RuntimePerReducer, EstimationScheme, IgnoreInputSize } +import RuntimeReducerEstimator.{ EstimationScheme, IgnoreInputSize, RuntimePerReducer } +import com.twitter.scalding.estimation.{ Estimator, FlowStepHistory, FlowStrategyInfo, HistoryService } import com.twitter.scalding.platform.{ HadoopPlatformJobTest, HadoopSharedPlatformTest } import org.scalatest.{ Matchers, WordSpec } import scala.collection.JavaConverters._ import scala.util.{ Success, Try } -object HistoryService1 extends HistoryServiceWithData { +object HistoryService1 extends HistoryService { import HistoryServiceWithData._ def fetchHistory(info: FlowStrategyInfo, maxHistory: Int): Try[Seq[FlowStepHistory]] = @@ -30,8 +31,8 @@ class ErrorRuntimeEstimator extends RatioBasedEstimator { override val historyService = ErrorHistoryService } -class DummyEstimator extends ReducerEstimator { - def estimateReducers(info: FlowStrategyInfo) = Some(42) +class DummyEstimator extends Estimator[Int] { + override def estimate(info: FlowStrategyInfo): Option[Int] = Some(42) } class RuntimeReducerEstimatorTest extends WordSpec with Matchers with HadoopSharedPlatformTest { diff --git a/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/reducer_estimation/HRavenHistoryService.scala b/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/estimation/HRavenHistoryService.scala similarity index 69% rename from scalding-hraven/src/main/scala/com/twitter/scalding/hraven/reducer_estimation/HRavenHistoryService.scala rename to scalding-hraven/src/main/scala/com/twitter/scalding/hraven/estimation/HRavenHistoryService.scala index 3b9d2779b9..bfd74af7af 100644 --- a/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/reducer_estimation/HRavenHistoryService.scala +++ b/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/estimation/HRavenHistoryService.scala @@ -1,14 +1,14 @@ -package com.twitter.scalding.hraven.reducer_estimation +package com.twitter.scalding.hraven.estimation -import java.io.IOException import cascading.flow.FlowStep -import com.twitter.hraven.{ Constants, CounterMap, Flow, HistoryFileType, JobDetails } +import com.twitter.hraven.JobDescFactory.{ JOBTRACKER_KEY, RESOURCE_MANAGER_KEY } import com.twitter.hraven.rest.client.HRavenRestClient -import com.twitter.scalding.reducer_estimation._ +import com.twitter.hraven.{ Constants, CounterMap, Flow, HadoopVersion, JobDetails, TaskDetails } +import com.twitter.scalding.estimation.{ FlowStepHistory, FlowStepKeys, FlowStrategyInfo, HistoryService, Task } +import java.io.IOException import org.apache.hadoop.mapred.JobConf import org.slf4j.LoggerFactory import scala.collection.JavaConverters._ -import com.twitter.hraven.JobDescFactory.{ JOBTRACKER_KEY, RESOURCE_MANAGER_KEY } import scala.util.{ Failure, Success, Try } object HRavenClient { @@ -28,26 +28,9 @@ object HRavenClient { conf.getInt(clientReadTimeoutKey, clientReadTimeoutDefault))) } -/** - * Mixin for ReducerEstimators to give them the ability to query hRaven for - * info about past runs. - */ -object HRavenHistoryService extends HistoryService { - +object HRavenHistoryService { private val LOG = LoggerFactory.getLogger(this.getClass) - // List of fields that we consume from fetchTaskDetails api. - // This is sent to hraven service to filter the response data - // and avoid hitting http content length limit on hraven side. - private val TaskDetailFields = List( - "taskType", - "status", - "startTime", - "finishTime").asJava - private val MapOutputBytesKey = "MAP_OUTPUT_BYTES" - - val RequiredJobConfigs = Seq("cascading.flow.step.num") - case class MissingFieldsException(fields: Seq[String]) extends Exception /** @@ -55,10 +38,18 @@ object HRavenHistoryService extends HistoryService { */ case class RichConfig(conf: JobConf) { - val MaxFetch = "hraven.reducer.estimator.max.flow.history" + val OldMaxFetch = "hraven.reducer.estimator.max.flow.history" + val MaxFetch = "hraven.estimator.max.flow.history" val MaxFetchDefault = 8 - def maxFetch: Int = conf.getInt(MaxFetch, MaxFetchDefault) + def maxFetch: Int = { + val max = conf.getInt(MaxFetch, -1) + if (max == -1) { + conf.getInt(OldMaxFetch, MaxFetchDefault) + } else { + max + } + } /** * Try fields in order until one returns a value. @@ -71,9 +62,28 @@ object HRavenHistoryService extends HistoryService { LOG.warn("Missing required config param: " + fields.mkString(" or ")) Failure(MissingFieldsException(fields)) } - } + implicit def jobConfToRichConfig(conf: JobConf): RichConfig = RichConfig(conf) +} + +/** + * History Service that gives ability to query hRaven for info about past runs. + */ +trait HRavenHistoryService extends HistoryService { + import HRavenHistoryService.jobConfToRichConfig + + private val LOG = LoggerFactory.getLogger(this.getClass) + private val RequiredJobConfigs = Seq("cascading.flow.step.num") + private val MapOutputBytesKey = "MAP_OUTPUT_BYTES" + + protected val detailFields: List[String] + protected val counterFields: List[String] + + protected def details(taskDetails: TaskDetails): Option[Map[String, Any]] + protected def counters(taskCounters: CounterMap): Option[Map[String, Long]] + + def hRavenClient(conf: JobConf): Try[HRavenRestClient] = HRavenClient(conf) /** * Fetch flows until it finds one that was successful @@ -83,18 +93,35 @@ object HRavenHistoryService extends HistoryService { * TODO: query hRaven for successful jobs (first need to add ability to filter * results in hRaven REST API) */ - private def fetchSuccessfulFlows(client: HRavenRestClient, cluster: String, user: String, batch: String, signature: String, max: Int, nFetch: Int): Try[Seq[Flow]] = - Try(client.fetchFlowsWithConfig(cluster, user, batch, signature, nFetch, RequiredJobConfigs: _*)) + private def fetchSuccessfulFlows( + client: HRavenRestClient, + cluster: String, + user: String, + batch: String, + signature: String, + stepNum: Int, + max: Int, + nFetch: Int + ): Try[Seq[Flow]] = + Try(client + .fetchFlowsWithConfig(cluster, user, batch, signature, nFetch, RequiredJobConfigs: _*)) .flatMap { flows => Try { // Ugly mutable code to add task info to flows flows.asScala.foreach { flow => - flow.getJobs.asScala.foreach { job => - - // client.fetchTaskDetails might throw IOException - val tasks = client.fetchTaskDetails(flow.getCluster, job.getJobId, TaskDetailFields) - job.addTasks(tasks) - } + flow.getJobs.asScala + .filter { step => + step.getConfiguration.get("cascading.flow.step.num").toInt == stepNum + } + .foreach { job => + // client.fetchTaskDetails might throw IOException + val tasks = if (counterFields.isEmpty) { + client.fetchTaskDetails(flow.getCluster, job.getJobId, detailFields.asJava) + } else { + client.fetchTaskDetails(flow.getCluster, job.getJobId, detailFields.asJava, counterFields.asJava) + } + job.addTasks(tasks) + } } val successfulFlows = flows.asScala.filter(_.getHdfsBytesRead > 0).take(max) @@ -104,10 +131,10 @@ object HRavenHistoryService extends HistoryService { successfulFlows } }.recoverWith { - case e: IOException => - LOG.error("Error making API request to hRaven. HRavenHistoryService will be disabled.") - Failure(e) - } + case e: IOException => + LOG.error("Error making API request to hRaven. HRavenHistoryService will be disabled.") + Failure(e) + } /** * Fetch info from hRaven for the last time the given JobStep ran. @@ -148,7 +175,7 @@ object HRavenHistoryService extends HistoryService { val flowsTry = for { // connect to hRaven REST API - client <- HRavenClient(conf) + client <- hRavenClient(conf) // lookup cluster name used by hRaven cluster <- lookupClusterName(client) @@ -159,7 +186,7 @@ object HRavenHistoryService extends HistoryService { signature <- conf.getFirstKey("scalding.flow.class.signature") // query hRaven for matching flows - flows <- fetchSuccessfulFlows(client, cluster, user, batch, signature, max, conf.maxFetch) + flows <- fetchSuccessfulFlows(client, cluster, user, batch, signature, stepNum, max, conf.maxFetch) } yield flows @@ -174,16 +201,21 @@ object HRavenHistoryService extends HistoryService { step <- history keys = FlowStepKeys(step.getJobName, step.getUser, step.getPriority, step.getStatus, step.getVersion, "") // update HRavenHistoryService.TaskDetailFields when consuming additional task fields from hraven below - tasks = step.getTasks.asScala.map { t => Task(t.getType, t.getStatus, t.getStartTime, t.getFinishTime) } + tasks = step.getTasks.asScala.flatMap { taskDetails => + details(taskDetails).zip(counters(taskDetails.getCounters)).map { + case (details, counters) => + Task(details, counters) + } + } } yield toFlowStepHistory(keys, step, tasks) } private def toFlowStepHistory(keys: FlowStepKeys, step: JobDetails, tasks: Seq[Task]) = { FlowStepHistory( keys = keys, - submitTime = step.getSubmitTime, - launchTime = step.getLaunchTime, - finishTime = step.getFinishTime, + submitTimeMillis = step.getSubmitTime, + launchTimeMillis = step.getLaunchTime, + finishTimeMillis = step.getFinishTime, totalMaps = step.getTotalMaps, totalReduces = step.getTotalReduces, finishedMaps = step.getFinishedMaps, @@ -204,7 +236,7 @@ object HRavenHistoryService extends HistoryService { } private def mapOutputBytes(step: JobDetails): Long = { - if (step.getHistoryFileType == HistoryFileType.TWO) { + if (step.getHadoopVersion == HadoopVersion.TWO) { getCounterValueAsLong(step.getMapCounters, Constants.TASK_COUNTER_HADOOP2, MapOutputBytesKey) } else { getCounterValueAsLong(step.getMapCounters, Constants.TASK_COUNTER, MapOutputBytesKey) @@ -215,12 +247,4 @@ object HRavenHistoryService extends HistoryService { val counter = counters.getCounter(counterGroupName, counterName) if (counter != null) counter.getValue else 0L } -} - -class HRavenRatioBasedEstimator extends RatioBasedEstimator { - override val historyService = HRavenHistoryService -} - -class HRavenRuntimeBasedEstimator extends RuntimeReducerEstimator { - override val historyService = HRavenHistoryService -} +} \ No newline at end of file diff --git a/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/estimation/memory/HRavenMemoryService.scala b/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/estimation/memory/HRavenMemoryService.scala new file mode 100644 index 0000000000..f7eafed942 --- /dev/null +++ b/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/estimation/memory/HRavenMemoryService.scala @@ -0,0 +1,48 @@ +package com.twitter.scalding.hraven.estimation.memory + +import com.twitter.hraven.{ CounterMap, TaskDetails } +import com.twitter.scalding.estimation.Task +import com.twitter.scalding.estimation.memory.SmoothedHistoryMemoryEstimator +import com.twitter.scalding.hraven.estimation.HRavenHistoryService + +trait HRavenMemoryHistoryService extends HRavenHistoryService { + import SmoothedHistoryMemoryEstimator._ + + private val TaskCounterGroup = "org.apache.hadoop.mapreduce.TaskCounter" + + override protected val detailFields: List[String] = List(Task.TaskType) + override protected val counterFields: List[String] = List( + "org.apache.hadoop.mapreduce.TaskCounter.COMMITTED_HEAP_BYTES", + "org.apache.hadoop.mapreduce.TaskCounter.PHYSICAL_MEMORY_BYTES", + "org.apache.hadoop.mapreduce.TaskCounter.GC_TIME_MILLIS", + "org.apache.hadoop.mapreduce.TaskCounter.CPU_MILLISECONDS") + + override protected def details(taskDetails: TaskDetails): Option[Map[String, Any]] = { + if (taskDetails.getType.nonEmpty) { + Some(Map(Task.TaskType -> taskDetails.getType)) + } else { + None + } + } + + override protected def counters(taskCounters: CounterMap): Option[Map[String, Long]] = { + //sometimes get groups with only partial data + if (taskCounters.getGroups.isEmpty || taskCounters.getGroup(TaskCounterGroup).size() < 4) { + None + } else { + val group = taskCounters.getGroup(TaskCounterGroup) + + Some(Map( + CommittedHeapBytes -> group.get(CommittedHeapBytes).getValue, + CpuMs -> group.get(CpuMs).getValue, + PhysicalMemoryBytes -> group.get(PhysicalMemoryBytes).getValue, + GCTimeMs -> group.get(GCTimeMs).getValue)) + } + } +} + +object HRavenMemoryHistoryService extends HRavenMemoryHistoryService + +class HRavenSmoothedMemoryEstimator extends SmoothedHistoryMemoryEstimator { + override def historyService = HRavenMemoryHistoryService +} diff --git a/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/memory_estimation/HRavenMemoryService.scala b/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/memory_estimation/HRavenMemoryService.scala deleted file mode 100644 index c77d4f502d..0000000000 --- a/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/memory_estimation/HRavenMemoryService.scala +++ /dev/null @@ -1,232 +0,0 @@ -package com.twitter.scalding.hraven.memory_estimation - -import java.io.IOException -import cascading.flow.FlowStep -import com.twitter.hraven.{ Flow, JobDetails } -import com.twitter.hraven.rest.client.HRavenRestClient -import org.apache.hadoop.mapred.JobConf -import org.slf4j.LoggerFactory -import scala.collection.JavaConverters._ -import com.twitter.hraven.JobDescFactory.{ JOBTRACKER_KEY, RESOURCE_MANAGER_KEY } -import com.twitter.scalding.memory_estimation.{ FlowStepMemoryHistory, HistoryMemoryEstimator, MemoryService, Task } -import com.twitter.scalding.reducer_estimation.{ FlowStepKeys, FlowStrategyInfo } -import scala.util.{ Failure, Success, Try } - -object HRavenClient { - import HRavenMemoryService.jobConfToRichConfig - - val apiHostnameKey = "hraven.api.hostname" // hraven.devel.vchannapattan.service.smf1.twitter.com - val clientConnectTimeoutKey = "hraven.client.connect.timeout" - val clientReadTimeoutKey = "hraven.client.read.timeout" - - private final val clientConnectTimeoutDefault = 120000 - private final val clientReadTimeoutDefault = 120000 - - def apply(conf: JobConf): Try[HRavenRestClient] = - conf.getFirstKey(apiHostnameKey) - .map(new HRavenRestClient(_, - conf.getInt(clientConnectTimeoutKey, clientConnectTimeoutDefault), - conf.getInt(clientReadTimeoutKey, clientReadTimeoutDefault))) -} - -/** - * Mixin for MemoryEstimators to give them the ability to query hRaven for - * info about past runs. - */ -object HRavenMemoryService extends MemoryService { - - private val LOG = LoggerFactory.getLogger(this.getClass) - - private val TaskCounterGroup = "org.apache.hadoop.mapreduce.TaskCounter" - private val CommittedHeapBytes = "COMMITTED_HEAP_BYTES" - private val CpuMs = "CPU_MILLISECONDS" - private val PhysicalMemoryBytes = "PHYSICAL_MEMORY_BYTES" - private val GCTimeMs = "GC_TIME_MILLIS" - - // List of fields that we consume from fetchTaskDetails api. - // This is sent to hraven service to filter the response data - // and avoid hitting http content length limit on hraven side. - private val TaskDetailFields = List("taskType").asJava - - private val TaskCounterFields = List( - "org.apache.hadoop.mapreduce.TaskCounter.COMMITTED_HEAP_BYTES", - "org.apache.hadoop.mapreduce.TaskCounter.PHYSICAL_MEMORY_BYTES", - "org.apache.hadoop.mapreduce.TaskCounter.GC_TIME_MILLIS", - "org.apache.hadoop.mapreduce.TaskCounter.CPU_MILLISECONDS").asJava - - val RequiredJobConfigs = Seq("cascading.flow.step.num") - - case class MissingFieldsException(fields: Seq[String]) extends Exception - - /** - * Add some helper methods to JobConf - */ - case class RichConfig(conf: JobConf) { - - val MaxFetch = "hraven.reducer.estimator.max.flow.history" - val MaxFetchDefault = 10 - - def maxFetch: Int = conf.getInt(MaxFetch, MaxFetchDefault) - - /** - * Try fields in order until one returns a value. - * Logs a warning if nothing was found. - */ - def getFirstKey(fields: String*): Try[String] = - fields.collectFirst { - case f if conf.get(f) != null => Success(conf.get(f)) - }.getOrElse { - LOG.warn("Missing required config param: " + fields.mkString(" or ")) - Failure(MissingFieldsException(fields)) - } - - } - implicit def jobConfToRichConfig(conf: JobConf): RichConfig = RichConfig(conf) - - /** - * Fetch flows until it finds one that was successful - * (using "HdfsBytesRead > 0" as a marker for successful jobs since it seems - * that this is only set on completion of jobs) - * - * TODO: query hRaven for successful jobs (first need to add ability to filter - * results in hRaven REST API) - */ - private def fetchSuccessfulFlows(client: HRavenRestClient, cluster: String, user: String, batch: String, signature: String, max: Int, nFetch: Int): Try[Seq[Flow]] = - Try(client.fetchFlowsWithConfig(cluster, user, batch, signature, nFetch, RequiredJobConfigs: _*)) - .flatMap { flows => - Try { - // Ugly mutable code to add task info to flows - flows.asScala.foreach { flow => - flow.getJobs.asScala.foreach { job => - - // client.fetchTaskDetails might throw IOException - val tasks = client.fetchTaskDetails(flow.getCluster, job.getJobId, TaskDetailFields, TaskCounterFields) - job.addTasks(tasks) - } - } - - val successfulFlows = flows.asScala.filter(_.getHdfsBytesRead > 0).take(max) - if (successfulFlows.isEmpty) { - LOG.warn("Unable to find any successful flows in the last " + nFetch + " jobs.") - } - successfulFlows - } - }.recoverWith { - case e: IOException => - LOG.error("Error making API request to hRaven. HRavenMemoryService will be disabled.") - Failure(e) - } - - /** - * Fetch info from hRaven for the last time the given JobStep ran. - * Finds the last successful complete flow and selects the corresponding - * step from it. - * - * @param step FlowStep to get info for - * @return Details about the previous successful run. - */ - def fetchPastJobDetails(step: FlowStep[JobConf], max: Int): Try[Seq[JobDetails]] = { - val conf = step.getConfig - val stepNum = step.getStepNum - - def findMatchingJobStep(pastFlow: Flow) = - pastFlow.getJobs.asScala.find { step => - try { - step.getConfiguration.get("cascading.flow.step.num").toInt == stepNum - } catch { - case _: NumberFormatException => false - } - } orElse { - LOG.warn("No matching job step in the retrieved hRaven flow.") - None - } - - def lookupClusterName(client: HRavenRestClient): Try[String] = { - // regex for case matching URL to get hostname out - val hostRegex = """(.*):\d+""".r - - // first try resource manager (for Hadoop v2), then fallback to job tracker - conf.getFirstKey(RESOURCE_MANAGER_KEY, JOBTRACKER_KEY).flatMap { - // extract hostname from hostname:port - case hostRegex(host) => - // convert hostname -> cluster name (e.g. dw2@smf1) - Try(client.getCluster(host)) - } - } - - val flowsTry = for { - // connect to hRaven REST API - client <- HRavenClient(conf) - - // lookup cluster name used by hRaven - cluster <- lookupClusterName(client) - - // get identifying info for this job - user <- conf.getFirstKey("hraven.history.user.name", "user.name") - batch <- conf.getFirstKey("batch.desc") - signature <- conf.getFirstKey("scalding.flow.class.signature") - - // query hRaven for matching flows - flows <- fetchSuccessfulFlows(client, cluster, user, batch, signature, max, conf.maxFetch) - - } yield flows - - // Find the FlowStep in the hRaven flow that corresponds to the current step - // *Note*: when hRaven says "Job" it means "FlowStep" - flowsTry.map{ flows => - val map: Seq[JobDetails] = flows.flatMap(findMatchingJobStep) - LOG.info("Found :" + flows.length + " flows in fetchJobDetails. After finding matching job step, noJobDetails = " + map.length) - map - } - } - - override def fetchHistory(info: FlowStrategyInfo, maxHistory: Int): Try[Seq[FlowStepMemoryHistory]] = - fetchPastJobDetails(info.step, maxHistory).map { history => - for { - step <- history - keys = FlowStepKeys(step.getJobName, step.getUser, step.getPriority, step.getStatus, step.getVersion, "") - // update HRavenHistoryService.TaskDetailFields when consuming additional task fields from hraven below - tasks = step.getTasks.asScala.flatMap { - t => - //sometimes get groups with only partial data - if (t.getCounters.getGroups.isEmpty || t.getCounters.getGroup(TaskCounterGroup).size() < 4) None - else { - val group = t.getCounters.getGroup(TaskCounterGroup) - - val committedHeap = group.get(CommittedHeapBytes).getValue - val cpu = group.get(CpuMs).getValue - val phyMemory = group.get(PhysicalMemoryBytes).getValue - val gc = group.get(GCTimeMs).getValue - Some(Task(t.getType, committedHeap, phyMemory, cpu, gc)) - } - } - } yield toFlowStepMemoryHistory(keys, step, tasks) - } - - private def toFlowStepMemoryHistory(keys: FlowStepKeys, step: JobDetails, tasks: Seq[Task]) = - FlowStepMemoryHistory( - keys = keys, - submitTime = step.getSubmitTime, - launchTime = step.getLaunchTime, - finishTime = step.getFinishTime, - totalMaps = step.getTotalMaps, - totalReduces = step.getTotalReduces, - finishedMaps = step.getFinishedMaps, - finishedReduces = step.getFinishedReduces, - failedMaps = step.getFailedMaps, - failedReduces = step.getFailedReduces, - mapFileBytesRead = step.getMapFileBytesRead, - mapFileBytesWritten = step.getMapFileBytesWritten, - reduceFileBytesRead = step.getReduceFileBytesRead, - hdfsBytesRead = step.getHdfsBytesRead, - hdfsBytesWritten = step.getHdfsBytesWritten, - mapperTimeMillis = step.getMapSlotMillis, - reducerTimeMillis = step.getReduceSlotMillis, - reduceShuffleBytes = step.getReduceShuffleBytes, - cost = 0, - tasks = tasks) -} - -class HRavenMemoryEstimator extends HistoryMemoryEstimator { - override val memoryService = HRavenMemoryService -} diff --git a/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/reducer_estimation/HRavenBasedEstimator.scala b/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/reducer_estimation/HRavenBasedEstimator.scala new file mode 100644 index 0000000000..c12801e170 --- /dev/null +++ b/scalding-hraven/src/main/scala/com/twitter/scalding/hraven/reducer_estimation/HRavenBasedEstimator.scala @@ -0,0 +1,38 @@ +package com.twitter.scalding.hraven.reducer_estimation + +import com.twitter.hraven.{ CounterMap, TaskDetails } +import com.twitter.scalding.estimation.Task +import com.twitter.scalding.hraven.estimation.HRavenHistoryService +import com.twitter.scalding.reducer_estimation.{ RatioBasedEstimator, RuntimeReducerEstimator } + +trait HRavenReducerHistoryService extends HRavenHistoryService { + override protected val counterFields: List[String] = List() + override protected val detailFields: List[String] = List( + Task.TaskType, + "status", + "startTime", + "finishTime") + + override protected def counters(taskCounters: CounterMap): Option[Map[String, Long]] = Some(Map.empty) + + override protected def details(taskDetails: TaskDetails): Option[Map[String, Any]] = + if (taskDetails.getType.nonEmpty) { + Some(Map( + Task.TaskType -> taskDetails.getType, + "status" -> taskDetails.getStatus, + "startTime" -> taskDetails.getStartTime, + "finishTime" -> taskDetails.getFinishTime)) + } else { + None + } +} + +object HRavenReducerHistoryService extends HRavenReducerHistoryService + +class HRavenRatioBasedEstimator extends RatioBasedEstimator { + override val historyService = HRavenReducerHistoryService +} + +class HRavenRuntimeBasedEstimator extends RuntimeReducerEstimator { + override val historyService = HRavenReducerHistoryService +} diff --git a/scalding-hraven/src/test/resources/flowResponse.json b/scalding-hraven/src/test/resources/flowResponse.json new file mode 100644 index 0000000000..1a90f3740a --- /dev/null +++ b/scalding-hraven/src/test/resources/flowResponse.json @@ -0,0 +1,3303 @@ +[ { + "flowKey" : { + "cluster" : "test@cluster", + "userName" : "testuser", + "appId" : "somegoodjob", + "runId" : 1492636627835, + "encodedRunId" : 9223370544218147972 + }, + "flowName" : "somegoodjob", + "userName" : "testuser", + "jobCount" : 3, + "totalMaps" : 368, + "totalReduces" : 136, + "mapFileBytesRead" : 111415465732, + "mapFileBytesWritten" : 222939745150, + "reduceFileBytesRead" : 159412950342, + "hdfsBytesRead" : 261872610033, + "hdfsBytesWritten" : 101710652089, + "mapSlotMillis" : 26673895, + "reduceSlotMillis" : 16718566, + "megabyteMillis" : 140725068288, + "cost" : 0.15269647166666667, + "reduceShuffleBytes" : 111314400214, + "duration" : 1278809, + "wallClockTime" : 1292275, + "cluster" : "test@cluster", + "appId" : "somegoodjob", + "runId" : 1492636627835, + "version" : "02CFBD0A94AD5E297C2E4D6665B3B6F0", + "hadoopVersion" : "TWO", + "submitTime" : 1492636719034, + "launchTime" : 1492636732500, + "finishTime" : 1492638011309, + "queue" : "testuser", + "counters" : { + "Scalding Custom" : { + "test_user" : 749 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 47654313, + "Tuples_Read" : 183443467 + }, + "evictions" : { + "MapsideReduce" : 12588706 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "REDUCE_INPUT_RECORDS" : 66501607, + "SPILLED_RECORDS" : 212145628, + "VIRTUAL_MEMORY_BYTES" : 2050469523456, + "LOCALIZED_NANOS" : 523231295546, + "SPLIT_RAW_BYTES" : 772454, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 379508682777, + "REDUCE_SHUFFLE_BYTES" : 111314400214, + "PHYSICAL_MEMORY_BYTES" : 446650638336, + "COMBINE_OUTPUT_RECORDS" : 0, + "LOCALIZED_FILES_CACHED" : 508618, + "CPU_MILLISECONDS" : 32777850, + "LOCALIZED_FILES_MISSED" : 65239, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 111314400214, + "MERGED_MAP_OUTPUTS" : 19834, + "MAP_INPUT_RECORDS" : 183443467, + "LOCALIZED_BYTES_MISSED" : 76410918436, + "GC_TIME_MILLIS" : 3666842, + "REDUCE_INPUT_GROUPS" : 49999258, + "SHUFFLED_MAPS" : 19834, + "REDUCE_OUTPUT_RECORDS" : 47654313, + "LOCALIZED_BYTES_CACHED" : 464339205620, + "MAP_OUTPUT_RECORDS" : 66501607, + "COMBINE_INPUT_RECORDS" : 0, + "COMMITTED_HEAP_BYTES" : 457269559296 + }, + "hits" : { + "MapsideReduce" : 1545 + }, + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 382535497232, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 1890, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 261872610033, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 270828416074, + "HDFS_WRITE_OPS" : 292, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 101710652089 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 114155920, + "Tuples_Read" : 249945074, + "Read_Duration" : 15736600, + "Write_Duration" : 5645584, + "Process_Begin_Time" : 752289113507899, + "Process_End_Time" : 752289136140472 + }, + "LzoBlocks of com.twitter.tweetypie.thriftscala.TweetEvent" : { + "Errors" : 0, + "Records Read" : 151396662, + "Records Skipped" : 0 + }, + "misses" : { + "MapsideReduce" : 16467158 + }, + "org.apache.hadoop.mapreduce.JobCounter" : { + "TOTAL_LAUNCHED_MAPS" : 383, + "VCORES_MILLIS_REDUCES" : 14628747, + "TOTAL_LAUNCHED_REDUCES" : 137, + "NUM_KILLED_MAPS" : 15, + "OTHER_LOCAL_MAPS" : 193, + "DATA_LOCAL_MAPS" : 14, + "NUM_KILLED_REDUCES" : 1, + "MB_MILLIS_MAPS" : 83649341440, + "SLOTS_MILLIS_REDUCES" : 16718566, + "VCORES_MILLIS_MAPS" : 23339660, + "MB_MILLIS_REDUCES" : 52429429248, + "SLOTS_MILLIS_MAPS" : 26673895, + "RACK_LOCAL_MAPS" : 178, + "MILLIS_REDUCES" : 14628747, + "MILLIS_MAPS" : 23339660 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + } + }, + "mapCounters" : { + "Scalding Custom" : { + "test_user" : 749 + }, + "hits" : { + "MapsideReduce" : 1545 + }, + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 222939745150, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 1472, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 261872610033, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 111415465732, + "HDFS_WRITE_OPS" : 0, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Read" : 183443467 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 66501607, + "Tuples_Read" : 183443467, + "Read_Duration" : 12255275, + "Write_Duration" : 1936172, + "Process_Begin_Time" : 549290456528594, + "Process_End_Time" : 549290471470429 + }, + "LzoBlocks of com.twitter.tweetypie.thriftscala.TweetEvent" : { + "Errors" : 0, + "Records Read" : 151396662, + "Records Skipped" : 0 + }, + "misses" : { + "MapsideReduce" : 16467158 + }, + "evictions" : { + "MapsideReduce" : 12588706 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 50274, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 111314400214, + "MERGED_MAP_OUTPUTS" : 0, + "SPILLED_RECORDS" : 114990966, + "VIRTUAL_MEMORY_BYTES" : 1488733134848, + "MAP_INPUT_RECORDS" : 183443467, + "LOCALIZED_NANOS" : 404720635591, + "SPLIT_RAW_BYTES" : 772454, + "LOCALIZED_BYTES_MISSED" : 58694615852, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 379508682777, + "PHYSICAL_MEMORY_BYTES" : 384967135232, + "GC_TIME_MILLIS" : 3588158, + "LOCALIZED_FILES_CACHED" : 366240, + "LOCALIZED_BYTES_CACHED" : 333801489356, + "MAP_OUTPUT_RECORDS" : 66501607, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 21975550, + "COMMITTED_HEAP_BYTES" : 388319395840 + } + }, + "reduceCounters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 159595752082, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 418, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 0, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 159412950342, + "HDFS_WRITE_OPS" : 292, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 101710652089 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 47654313 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 47654313, + "Tuples_Read" : 66501607, + "Read_Duration" : 3481325, + "Write_Duration" : 3709412, + "Process_Begin_Time" : 202998656979305, + "Process_End_Time" : 202998664670043 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 14954, + "MERGED_MAP_OUTPUTS" : 19834, + "REDUCE_INPUT_RECORDS" : 66501607, + "SPILLED_RECORDS" : 97154662, + "VIRTUAL_MEMORY_BYTES" : 561736388608, + "LOCALIZED_NANOS" : 117077260352, + "LOCALIZED_BYTES_MISSED" : 17711987894, + "FAILED_SHUFFLE" : 0, + "REDUCE_SHUFFLE_BYTES" : 111314400214, + "PHYSICAL_MEMORY_BYTES" : 61683503104, + "GC_TIME_MILLIS" : 78684, + "REDUCE_INPUT_GROUPS" : 49999258, + "COMBINE_OUTPUT_RECORDS" : 0, + "SHUFFLED_MAPS" : 19834, + "LOCALIZED_FILES_CACHED" : 138988, + "REDUCE_OUTPUT_RECORDS" : 47654313, + "LOCALIZED_BYTES_CACHED" : 127341550620, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 10802300, + "COMMITTED_HEAP_BYTES" : 68950163456 + } + }, + "jobs" : [ { + "jobKey" : { + "cluster" : "test@cluster", + "userName" : "testuser", + "appId" : "somegoodjob", + "runId" : 1492636627835, + "jobId" : { + "cluster" : "test@cluster", + "jobEpoch" : 1470171371859, + "jobSequence" : 6609466, + "jobIdString" : "job_1470171371859_6609466" + }, + "qualifiedJobId" : { + "cluster" : "test@cluster", + "jobEpoch" : 1470171371859, + "jobSequence" : 6609466, + "jobIdString" : "job_1470171371859_6609466" + }, + "encodedRunId" : 9223370544218147972 + }, + "jobId" : "job_1470171371859_6609466", + "jobName" : "[757B3B58BB294F3F90AF5F83B8580C9E/A5F2F3029B964D4687A37F67670F8243] com.twitter.testuser.forward.testuserUnhydratedBatchJob/(2/3)", + "user" : "testuser", + "priority" : "", + "status" : "SUCCEEDED", + "version" : "02CFBD0A94AD5E297C2E4D6665B3B6F0", + "hadoopVersion" : "TWO", + "queue" : "testuser", + "submitTime" : 1492636719034, + "launchTime" : 1492636732500, + "finishTime" : 1492637099729, + "totalMaps" : 121, + "totalReduces" : 64, + "finishedMaps" : 121, + "finishedReduces" : 64, + "failedMaps" : 0, + "failedReduces" : 0, + "mapFileBytesRead" : 37795618712, + "mapFileBytesWritten" : 75485125090, + "reduceFileBytesRead" : 37635771874, + "hdfsBytesRead" : 94267826741, + "hdfsBytesWritten" : 37796931657, + "mapSlotMillis" : 13579749, + "reduceSlotMillis" : 7677084, + "reduceShuffleBytes" : 37660545208, + "megabyteMillis" : 68165602304, + "cost" : 0.07396441222222222, + "counters" : { + "Scalding Custom" : { + "test_user" : 749 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 16439297, + "Tuples_Read" : 75698331 + }, + "evictions" : { + "MapsideReduce" : 12588706 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "REDUCE_INPUT_RECORDS" : 16467158, + "SPILLED_RECORDS" : 49363208, + "VIRTUAL_MEMORY_BYTES" : 753780228096, + "LOCALIZED_NANOS" : 211444954699, + "SPLIT_RAW_BYTES" : 321497, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 129012446460, + "REDUCE_SHUFFLE_BYTES" : 37660545208, + "PHYSICAL_MEMORY_BYTES" : 222611759104, + "COMBINE_OUTPUT_RECORDS" : 0, + "LOCALIZED_FILES_CACHED" : 185293, + "CPU_MILLISECONDS" : 14149680, + "LOCALIZED_FILES_MISSED" : 25261, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 37660545208, + "MERGED_MAP_OUTPUTS" : 7744, + "MAP_INPUT_RECORDS" : 75698331, + "LOCALIZED_BYTES_MISSED" : 29807994052, + "GC_TIME_MILLIS" : 3528882, + "REDUCE_INPUT_GROUPS" : 16439297, + "SHUFFLED_MAPS" : 7744, + "REDUCE_OUTPUT_RECORDS" : 16439297, + "LOCALIZED_BYTES_CACHED" : 168574833664, + "MAP_OUTPUT_RECORDS" : 16467158, + "COMBINE_INPUT_RECORDS" : 0, + "COMMITTED_HEAP_BYTES" : 219938238464 + }, + "hits" : { + "MapsideReduce" : 1545 + }, + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 113206590143, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 676, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 94267826741, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 75431390586, + "HDFS_WRITE_OPS" : 128, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 37796931657 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 32906455, + "Tuples_Read" : 92165489, + "Read_Duration" : 7976477, + "Write_Duration" : 1878377, + "Process_Begin_Time" : 276137816396411, + "Process_End_Time" : 276137827048673 + }, + "LzoBlocks of com.twitter.tweetypie.thriftscala.TweetEvent" : { + "Errors" : 0, + "Records Read" : 75698331, + "Records Skipped" : 0 + }, + "misses" : { + "MapsideReduce" : 16467158 + }, + "org.apache.hadoop.mapreduce.JobCounter" : { + "TOTAL_LAUNCHED_MAPS" : 132, + "VCORES_MILLIS_REDUCES" : 6717449, + "TOTAL_LAUNCHED_REDUCES" : 64, + "NUM_KILLED_MAPS" : 11, + "OTHER_LOCAL_MAPS" : 75, + "DATA_LOCAL_MAPS" : 1, + "MB_MILLIS_MAPS" : 42586095104, + "SLOTS_MILLIS_REDUCES" : 7677084, + "VCORES_MILLIS_MAPS" : 11882281, + "MB_MILLIS_REDUCES" : 24075337216, + "SLOTS_MILLIS_MAPS" : 13579749, + "RACK_LOCAL_MAPS" : 58, + "MILLIS_REDUCES" : 6717449, + "MILLIS_MAPS" : 11882281 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + } + }, + "mapCounters" : { + "Scalding Custom" : { + "test_user" : 749 + }, + "hits" : { + "MapsideReduce" : 1545 + }, + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 75485125090, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 484, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 94267826741, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 37795618712, + "HDFS_WRITE_OPS" : 0, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Read" : 75698331 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 16467158, + "Tuples_Read" : 75698331, + "Read_Duration" : 6583101, + "Write_Duration" : 1230218, + "Process_Begin_Time" : 180609047914354, + "Process_End_Time" : 180609056298876 + }, + "LzoBlocks of com.twitter.tweetypie.thriftscala.TweetEvent" : { + "Errors" : 0, + "Records Read" : 75698331, + "Records Skipped" : 0 + }, + "misses" : { + "MapsideReduce" : 16467158 + }, + "evictions" : { + "MapsideReduce" : 12588706 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 17197, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 37660545208, + "MERGED_MAP_OUTPUTS" : 0, + "SPILLED_RECORDS" : 32909712, + "VIRTUAL_MEMORY_BYTES" : 489125904384, + "MAP_INPUT_RECORDS" : 75698331, + "LOCALIZED_NANOS" : 150246438277, + "SPLIT_RAW_BYTES" : 321497, + "LOCALIZED_BYTES_MISSED" : 20416784175, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 129012446460, + "PHYSICAL_MEMORY_BYTES" : 193147375616, + "GC_TIME_MILLIS" : 3495443, + "LOCALIZED_FILES_CACHED" : 119775, + "LOCALIZED_BYTES_CACHED" : 108638492318, + "MAP_OUTPUT_RECORDS" : 16467158, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 10839880, + "COMMITTED_HEAP_BYTES" : 187491102720 + } + }, + "reduceCounters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 37721465053, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 192, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 0, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 37635771874, + "HDFS_WRITE_OPS" : 128, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 37796931657 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 16439297 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 16439297, + "Tuples_Read" : 16467158, + "Read_Duration" : 1393376, + "Write_Duration" : 648159, + "Process_Begin_Time" : 95528768482057, + "Process_End_Time" : 95528770749797 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 8060, + "MERGED_MAP_OUTPUTS" : 7744, + "REDUCE_INPUT_RECORDS" : 16467158, + "SPILLED_RECORDS" : 16453496, + "VIRTUAL_MEMORY_BYTES" : 264654323712, + "LOCALIZED_NANOS" : 60713180069, + "LOCALIZED_BYTES_MISSED" : 9389689314, + "FAILED_SHUFFLE" : 0, + "REDUCE_SHUFFLE_BYTES" : 37660545208, + "PHYSICAL_MEMORY_BYTES" : 29464383488, + "GC_TIME_MILLIS" : 33439, + "REDUCE_INPUT_GROUPS" : 16439297, + "COMBINE_OUTPUT_RECORDS" : 0, + "SHUFFLED_MAPS" : 7744, + "LOCALIZED_FILES_CACHED" : 64388, + "REDUCE_OUTPUT_RECORDS" : 16439297, + "LOCALIZED_BYTES_CACHED" : 58870952798, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 3309800, + "COMMITTED_HEAP_BYTES" : 32447135744 + } + }, + "tasks" : [ ], + "configuration" : { + "cascading.flow.step.num" : "2" + }, + "submitDate" : 1492636719034, + "launchDate" : 1492636732500, + "finishDate" : 1492637099729, + "runTime" : 367229 + }, { + "jobKey" : { + "cluster" : "test@cluster", + "userName" : "testuser", + "appId" : "somegoodjob", + "runId" : 1492636627835, + "jobId" : { + "cluster" : "test@cluster", + "jobEpoch" : 1470171371859, + "jobSequence" : 6609558, + "jobIdString" : "job_1470171371859_6609558" + }, + "qualifiedJobId" : { + "cluster" : "test@cluster", + "jobEpoch" : 1470171371859, + "jobSequence" : 6609558, + "jobIdString" : "job_1470171371859_6609558" + }, + "encodedRunId" : 9223370544218147972 + }, + "jobId" : "job_1470171371859_6609558", + "jobName" : "[757B3B58BB294F3F90AF5F83B8580C9E/CE20226849374E27A88A88828634B0A1] com.twitter.testuser.forward.testuserUnhydratedBatchJob/(1/3)", + "user" : "testuser", + "priority" : "", + "status" : "SUCCEEDED", + "version" : "02CFBD0A94AD5E297C2E4D6665B3B6F0", + "hadoopVersion" : "TWO", + "queue" : "testuser", + "submitTime" : 1492637189747, + "launchTime" : 1492637201324, + "finishTime" : 1492637387288, + "totalMaps" : 185, + "totalReduces" : 62, + "finishedMaps" : 185, + "finishedReduces" : 62, + "failedMaps" : 0, + "failedReduces" : 0, + "mapFileBytesRead" : 38087140565, + "mapFileBytesWritten" : 76339257791, + "reduceFileBytesRead" : 63401214784, + "hdfsBytesRead" : 132064823998, + "hdfsBytesWritten" : 35539895434, + "mapSlotMillis" : 9895558, + "reduceSlotMillis" : 4403484, + "reduceShuffleBytes" : 38141261549, + "megabyteMillis" : 45603508736, + "cost" : 0.049482973888888886, + "counters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 139823447713, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 926, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 132064823998, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 101488355349, + "HDFS_WRITE_OPS" : 124, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 35539895434 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 15607508, + "Tuples_Read" : 92137628 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 50034449, + "Tuples_Read" : 126564569, + "Read_Duration" : 5795589, + "Write_Duration" : 930624, + "Process_Begin_Time" : 368681399803019, + "Process_End_Time" : 368681406897953 + }, + "LzoBlocks of com.twitter.tweetypie.thriftscala.TweetEvent" : { + "Errors" : 0, + "Records Read" : 75698331, + "Records Skipped" : 0 + }, + "org.apache.hadoop.mapreduce.JobCounter" : { + "TOTAL_LAUNCHED_MAPS" : 188, + "VCORES_MILLIS_REDUCES" : 3853049, + "TOTAL_LAUNCHED_REDUCES" : 63, + "NUM_KILLED_MAPS" : 3, + "OTHER_LOCAL_MAPS" : 77, + "DATA_LOCAL_MAPS" : 11, + "NUM_KILLED_REDUCES" : 1, + "MB_MILLIS_MAPS" : 31032472576, + "SLOTS_MILLIS_REDUCES" : 4403484, + "VCORES_MILLIS_MAPS" : 8658614, + "MB_MILLIS_REDUCES" : 13809327616, + "SLOTS_MILLIS_MAPS" : 9895558, + "RACK_LOCAL_MAPS" : 100, + "MILLIS_REDUCES" : 3853049, + "MILLIS_MAPS" : 8658614 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "REDUCE_INPUT_RECORDS" : 34426941, + "SPILLED_RECORDS" : 105895650, + "VIRTUAL_MEMORY_BYTES" : 1004495097856, + "LOCALIZED_NANOS" : 246352867221, + "SPLIT_RAW_BYTES" : 387097, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 129571356432, + "REDUCE_SHUFFLE_BYTES" : 38141261549, + "PHYSICAL_MEMORY_BYTES" : 170776510464, + "COMBINE_OUTPUT_RECORDS" : 0, + "LOCALIZED_FILES_CACHED" : 248625, + "CPU_MILLISECONDS" : 11651100, + "LOCALIZED_FILES_MISSED" : 32113, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 38141261549, + "MERGED_MAP_OUTPUTS" : 11470, + "MAP_INPUT_RECORDS" : 92137628, + "LOCALIZED_BYTES_MISSED" : 37651200580, + "GC_TIME_MILLIS" : 100047, + "REDUCE_INPUT_GROUPS" : 33559951, + "SHUFFLED_MAPS" : 11470, + "REDUCE_OUTPUT_RECORDS" : 15607508, + "LOCALIZED_BYTES_CACHED" : 226859897415, + "MAP_OUTPUT_RECORDS" : 34426941, + "COMBINE_INPUT_RECORDS" : 0, + "COMMITTED_HEAP_BYTES" : 181911969792 + } + }, + "mapCounters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 76339257791, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 740, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 132064823998, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 38087140565, + "HDFS_WRITE_OPS" : 0, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Read" : 92137628 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 34426941, + "Tuples_Read" : 92137628, + "Read_Duration" : 4571858, + "Write_Duration" : 430227, + "Process_Begin_Time" : 276137886793246, + "Process_End_Time" : 276137891951274 + }, + "LzoBlocks of com.twitter.tweetypie.thriftscala.TweetEvent" : { + "Errors" : 0, + "Records Read" : 75698331, + "Records Skipped" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 25224, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 38141261549, + "MERGED_MAP_OUTPUTS" : 0, + "SPILLED_RECORDS" : 50866238, + "VIRTUAL_MEMORY_BYTES" : 748630978560, + "MAP_INPUT_RECORDS" : 92137628, + "LOCALIZED_NANOS" : 193724866170, + "SPLIT_RAW_BYTES" : 387097, + "LOCALIZED_BYTES_MISSED" : 29337513318, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 129571356432, + "PHYSICAL_MEMORY_BYTES" : 142990761984, + "GC_TIME_MILLIS" : 68431, + "LOCALIZED_FILES_CACHED" : 184196, + "LOCALIZED_BYTES_CACHED" : 167978930317, + "MAP_OUTPUT_RECORDS" : 34426941, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 8247540, + "COMMITTED_HEAP_BYTES" : 150478807040 + } + }, + "reduceCounters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 63484189922, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 186, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 0, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 63401214784, + "HDFS_WRITE_OPS" : 124, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 35539895434 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 15607508 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 15607508, + "Tuples_Read" : 34426941, + "Read_Duration" : 1223731, + "Write_Duration" : 500397, + "Process_Begin_Time" : 92543513009773, + "Process_End_Time" : 92543514946679 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 6885, + "MERGED_MAP_OUTPUTS" : 11470, + "REDUCE_INPUT_RECORDS" : 34426941, + "SPILLED_RECORDS" : 55029412, + "VIRTUAL_MEMORY_BYTES" : 255864119296, + "LOCALIZED_NANOS" : 51994187544, + "LOCALIZED_BYTES_MISSED" : 8312094452, + "FAILED_SHUFFLE" : 0, + "REDUCE_SHUFFLE_BYTES" : 38141261549, + "PHYSICAL_MEMORY_BYTES" : 27785748480, + "GC_TIME_MILLIS" : 31616, + "REDUCE_INPUT_GROUPS" : 33559951, + "COMBINE_OUTPUT_RECORDS" : 0, + "SHUFFLED_MAPS" : 11470, + "LOCALIZED_FILES_CACHED" : 63299, + "REDUCE_OUTPUT_RECORDS" : 15607508, + "LOCALIZED_BYTES_CACHED" : 57815578550, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 3403560, + "COMMITTED_HEAP_BYTES" : 31433162752 + } + }, + "tasks" : [ ], + "configuration" : { + "cascading.flow.step.num" : "1" + }, + "submitDate" : 1492637189747, + "launchDate" : 1492637201324, + "finishDate" : 1492637387288, + "runTime" : 185964 + }, { + "jobKey" : { + "cluster" : "test@cluster", + "userName" : "testuser", + "appId" : "somegoodjob", + "runId" : 1492636627835, + "jobId" : { + "cluster" : "test@cluster", + "jobEpoch" : 1470171371859, + "jobSequence" : 6609614, + "jobIdString" : "job_1470171371859_6609614" + }, + "qualifiedJobId" : { + "cluster" : "test@cluster", + "jobEpoch" : 1470171371859, + "jobSequence" : 6609614, + "jobIdString" : "job_1470171371859_6609614" + }, + "encodedRunId" : 9223370544218147972 + }, + "jobId" : "job_1470171371859_6609614", + "jobName" : "[757B3B58BB294F3F90AF5F83B8580C9E/8B29C94927F245C5A2D7CA117AFC2565] com.twitter.testuser.forward.testuserUnhydratedBatchJob/(3/3) .../unhydrated/2017/04/19/19", + "user" : "testuser", + "priority" : "", + "status" : "SUCCEEDED", + "version" : "02CFBD0A94AD5E297C2E4D6665B3B6F0", + "hadoopVersion" : "TWO", + "queue" : "testuser", + "submitTime" : 1492637423760, + "launchTime" : 1492637430152, + "finishTime" : 1492638011309, + "totalMaps" : 62, + "totalReduces" : 10, + "finishedMaps" : 62, + "finishedReduces" : 10, + "failedMaps" : 0, + "failedReduces" : 0, + "mapFileBytesRead" : 35532706455, + "mapFileBytesWritten" : 71115362269, + "reduceFileBytesRead" : 58375963684, + "hdfsBytesRead" : 35539959294, + "hdfsBytesWritten" : 28373824998, + "mapSlotMillis" : 3198588, + "reduceSlotMillis" : 4637998, + "reduceShuffleBytes" : 35512593457, + "megabyteMillis" : 26955957248, + "cost" : 0.029249085555555554, + "counters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 129505459376, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 288, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 35539959294, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 93908670139, + "HDFS_WRITE_OPS" : 40, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 28373824998 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 15607508, + "Tuples_Read" : 15607508 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 31215016, + "Tuples_Read" : 31215016, + "Read_Duration" : 1964534, + "Write_Duration" : 2836583, + "Process_Begin_Time" : 107469897308469, + "Process_End_Time" : 107469902193846 + }, + "org.apache.hadoop.mapreduce.JobCounter" : { + "TOTAL_LAUNCHED_MAPS" : 63, + "VCORES_MILLIS_REDUCES" : 4058249, + "TOTAL_LAUNCHED_REDUCES" : 10, + "NUM_KILLED_MAPS" : 1, + "OTHER_LOCAL_MAPS" : 41, + "DATA_LOCAL_MAPS" : 2, + "MB_MILLIS_MAPS" : 10030773760, + "SLOTS_MILLIS_REDUCES" : 4637998, + "VCORES_MILLIS_MAPS" : 2798765, + "MB_MILLIS_REDUCES" : 14544764416, + "SLOTS_MILLIS_MAPS" : 3198588, + "RACK_LOCAL_MAPS" : 20, + "MILLIS_REDUCES" : 4058249, + "MILLIS_MAPS" : 2798765 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "REDUCE_INPUT_RECORDS" : 15607508, + "SPILLED_RECORDS" : 56886770, + "VIRTUAL_MEMORY_BYTES" : 292194197504, + "LOCALIZED_NANOS" : 65433473626, + "SPLIT_RAW_BYTES" : 63860, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 120924879885, + "REDUCE_SHUFFLE_BYTES" : 35512593457, + "PHYSICAL_MEMORY_BYTES" : 53262368768, + "COMBINE_OUTPUT_RECORDS" : 0, + "LOCALIZED_FILES_CACHED" : 74700, + "CPU_MILLISECONDS" : 6977070, + "LOCALIZED_FILES_MISSED" : 7865, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 35512593457, + "MERGED_MAP_OUTPUTS" : 620, + "MAP_INPUT_RECORDS" : 15607508, + "LOCALIZED_BYTES_MISSED" : 8951723804, + "GC_TIME_MILLIS" : 37913, + "REDUCE_INPUT_GROUPS" : 10, + "SHUFFLED_MAPS" : 620, + "REDUCE_OUTPUT_RECORDS" : 15607508, + "LOCALIZED_BYTES_CACHED" : 68904474541, + "MAP_OUTPUT_RECORDS" : 15607508, + "COMBINE_INPUT_RECORDS" : 0, + "COMMITTED_HEAP_BYTES" : 55419351040 + } + }, + "mapCounters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 71115362269, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 248, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 35539959294, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 35532706455, + "HDFS_WRITE_OPS" : 0, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Read" : 15607508 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 15607508, + "Tuples_Read" : 15607508, + "Read_Duration" : 1100316, + "Write_Duration" : 275727, + "Process_Begin_Time" : 92543521820994, + "Process_End_Time" : 92543523220279 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 7853, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 35512593457, + "MERGED_MAP_OUTPUTS" : 0, + "SPILLED_RECORDS" : 31215016, + "VIRTUAL_MEMORY_BYTES" : 250976251904, + "MAP_INPUT_RECORDS" : 15607508, + "LOCALIZED_NANOS" : 60749331144, + "SPLIT_RAW_BYTES" : 63860, + "LOCALIZED_BYTES_MISSED" : 8940318359, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 120924879885, + "PHYSICAL_MEMORY_BYTES" : 48828997632, + "GC_TIME_MILLIS" : 24284, + "LOCALIZED_FILES_CACHED" : 62269, + "LOCALIZED_BYTES_CACHED" : 57184066721, + "MAP_OUTPUT_RECORDS" : 15607508, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 2888130, + "COMMITTED_HEAP_BYTES" : 50349486080 + } + }, + "reduceCounters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 58390097107, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 40, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 0, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 58375963684, + "HDFS_WRITE_OPS" : 40, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 28373824998 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 15607508 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 15607508, + "Tuples_Read" : 15607508, + "Read_Duration" : 864218, + "Write_Duration" : 2560856, + "Process_Begin_Time" : 14926375487475, + "Process_End_Time" : 14926378973567 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 9, + "MERGED_MAP_OUTPUTS" : 620, + "REDUCE_INPUT_RECORDS" : 15607508, + "SPILLED_RECORDS" : 25671754, + "VIRTUAL_MEMORY_BYTES" : 41217945600, + "LOCALIZED_NANOS" : 4369892739, + "LOCALIZED_BYTES_MISSED" : 10204128, + "FAILED_SHUFFLE" : 0, + "REDUCE_SHUFFLE_BYTES" : 35512593457, + "PHYSICAL_MEMORY_BYTES" : 4433371136, + "GC_TIME_MILLIS" : 13629, + "REDUCE_INPUT_GROUPS" : 10, + "COMBINE_OUTPUT_RECORDS" : 0, + "SHUFFLED_MAPS" : 620, + "LOCALIZED_FILES_CACHED" : 11301, + "REDUCE_OUTPUT_RECORDS" : 15607508, + "LOCALIZED_BYTES_CACHED" : 10655019272, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 4088940, + "COMMITTED_HEAP_BYTES" : 5069864960 + } + }, + "tasks" : [ ], + "configuration" : { + "cascading.flow.step.num" : "3" + }, + "submitDate" : 1492637423760, + "launchDate" : 1492637430152, + "finishDate" : 1492638011309, + "runTime" : 581157 + } ] +}, { + "flowKey" : { + "cluster" : "test@cluster", + "userName" : "testuser", + "appId" : "somegoodjob", + "runId" : 1492632715606, + "encodedRunId" : 9223370544222060201 + }, + "flowName" : "somegoodjob", + "userName" : "testuser", + "jobCount" : 3, + "totalMaps" : 385, + "totalReduces" : 137, + "mapFileBytesRead" : 113172163234, + "mapFileBytesWritten" : 226544957183, + "reduceFileBytesRead" : 163250996657, + "hdfsBytesRead" : 266627900618, + "hdfsBytesWritten" : 103607443641, + "mapSlotMillis" : 29331332, + "reduceSlotMillis" : 17119024, + "megabyteMillis" : 150597505536, + "cost" : 0.16340875166666663, + "reduceShuffleBytes" : 113144899105, + "duration" : 1355698, + "wallClockTime" : 1368561, + "cluster" : "test@cluster", + "appId" : "somegoodjob", + "runId" : 1492632715606, + "version" : "02CFBD0A94AD5E297C2E4D6665B3B6F0", + "hadoopVersion" : "TWO", + "submitTime" : 1492632797525, + "launchTime" : 1492632810388, + "finishTime" : 1492634166086, + "queue" : "testuser", + "counters" : { + "Scalding Custom" : { + "test_user" : 810 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 48800236, + "Tuples_Read" : 195996831 + }, + "evictions" : { + "MapsideReduce" : 12771518 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "REDUCE_INPUT_RECORDS" : 72741055, + "SPILLED_RECORDS" : 231774506, + "VIRTUAL_MEMORY_BYTES" : 2122960666624, + "LOCALIZED_NANOS" : 682258492993, + "SPLIT_RAW_BYTES" : 815480, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 387016458073, + "REDUCE_SHUFFLE_BYTES" : 113144899105, + "PHYSICAL_MEMORY_BYTES" : 464332615680, + "COMBINE_OUTPUT_RECORDS" : 0, + "LOCALIZED_FILES_CACHED" : 526479, + "CPU_MILLISECONDS" : 36129990, + "LOCALIZED_FILES_MISSED" : 67753, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 113144899105, + "MERGED_MAP_OUTPUTS" : 21045, + "MAP_INPUT_RECORDS" : 195996831, + "LOCALIZED_BYTES_MISSED" : 79988792697, + "GC_TIME_MILLIS" : 3804321, + "REDUCE_INPUT_GROUPS" : 55895910, + "SHUFFLED_MAPS" : 21045, + "REDUCE_OUTPUT_RECORDS" : 48800236, + "LOCALIZED_BYTES_CACHED" : 479959047536, + "MAP_OUTPUT_RECORDS" : 72741055, + "COMBINE_INPUT_RECORDS" : 0, + "COMMITTED_HEAP_BYTES" : 474627133440 + }, + "hits" : { + "MapsideReduce" : 1603 + }, + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 389979938079, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 1961, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 266627900618, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 276423159891, + "HDFS_WRITE_OPS" : 294, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 103607443641 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 121541291, + "Tuples_Read" : 268737886, + "Read_Duration" : 17360974, + "Write_Duration" : 6210507, + "Process_Begin_Time" : 779154533017338, + "Process_End_Time" : 779154558057047 + }, + "LzoBlocks of com.twitter.tweetypie.thriftscala.TweetEvent" : { + "Errors" : 0, + "Records Read" : 163208962, + "Records Skipped" : 0 + }, + "misses" : { + "MapsideReduce" : 16803506 + }, + "org.apache.hadoop.mapreduce.JobCounter" : { + "TOTAL_LAUNCHED_MAPS" : 403, + "VCORES_MILLIS_REDUCES" : 14979147, + "TOTAL_LAUNCHED_REDUCES" : 141, + "NUM_KILLED_MAPS" : 17, + "OTHER_LOCAL_MAPS" : 205, + "DATA_LOCAL_MAPS" : 19, + "NUM_KILLED_REDUCES" : 4, + "MB_MILLIS_MAPS" : 91983058944, + "SLOTS_MILLIS_REDUCES" : 17119024, + "VCORES_MILLIS_MAPS" : 25664916, + "NUM_FAILED_MAPS" : 1, + "MB_MILLIS_REDUCES" : 53685262848, + "SLOTS_MILLIS_MAPS" : 29331332, + "RACK_LOCAL_MAPS" : 180, + "MILLIS_REDUCES" : 14979147, + "MILLIS_MAPS" : 25664916 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + } + }, + "mapCounters" : { + "Scalding Custom" : { + "test_user" : 810 + }, + "hits" : { + "MapsideReduce" : 1603 + }, + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 226544957183, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 1540, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 266627900618, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 113172163234, + "HDFS_WRITE_OPS" : 0, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Read" : 195996831 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 72741055, + "Tuples_Read" : 195996831, + "Read_Duration" : 13299362, + "Write_Duration" : 2170007, + "Process_Begin_Time" : 574663776002832, + "Process_End_Time" : 574663792332928 + }, + "LzoBlocks of com.twitter.tweetypie.thriftscala.TweetEvent" : { + "Errors" : 0, + "Records Read" : 163208962, + "Records Skipped" : 0 + }, + "misses" : { + "MapsideReduce" : 16803506 + }, + "evictions" : { + "MapsideReduce" : 12771518 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 51827, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 113144899105, + "MERGED_MAP_OUTPUTS" : 0, + "SPILLED_RECORDS" : 122300442, + "VIRTUAL_MEMORY_BYTES" : 1557522055168, + "MAP_INPUT_RECORDS" : 195996831, + "LOCALIZED_NANOS" : 526011367517, + "SPLIT_RAW_BYTES" : 815480, + "LOCALIZED_BYTES_MISSED" : 61496078904, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 387016458073, + "PHYSICAL_MEMORY_BYTES" : 402146570240, + "GC_TIME_MILLIS" : 3714743, + "LOCALIZED_FILES_CACHED" : 383930, + "LOCALIZED_BYTES_CACHED" : 349131284361, + "MAP_OUTPUT_RECORDS" : 72741055, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 23949430, + "COMMITTED_HEAP_BYTES" : 405168017408 + } + }, + "reduceCounters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 163434980896, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 421, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 0, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 163250996657, + "HDFS_WRITE_OPS" : 294, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 103607443641 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 48800236 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 48800236, + "Tuples_Read" : 72741055, + "Read_Duration" : 4061612, + "Write_Duration" : 4040500, + "Process_Begin_Time" : 204490757014506, + "Process_End_Time" : 204490765724119 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 15914, + "MERGED_MAP_OUTPUTS" : 21045, + "REDUCE_INPUT_RECORDS" : 72741055, + "SPILLED_RECORDS" : 109474064, + "VIRTUAL_MEMORY_BYTES" : 565438611456, + "LOCALIZED_NANOS" : 154690352172, + "LOCALIZED_BYTES_MISSED" : 18484109247, + "FAILED_SHUFFLE" : 0, + "REDUCE_SHUFFLE_BYTES" : 113144899105, + "PHYSICAL_MEMORY_BYTES" : 62186045440, + "GC_TIME_MILLIS" : 89578, + "REDUCE_INPUT_GROUPS" : 55895910, + "COMBINE_OUTPUT_RECORDS" : 0, + "SHUFFLED_MAPS" : 21045, + "LOCALIZED_FILES_CACHED" : 139160, + "REDUCE_OUTPUT_RECORDS" : 48800236, + "LOCALIZED_BYTES_CACHED" : 127635845787, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 12180560, + "COMMITTED_HEAP_BYTES" : 69459116032 + } + }, + "jobs" : [ { + "jobKey" : { + "cluster" : "test@cluster", + "userName" : "testuser", + "appId" : "somegoodjob", + "runId" : 1492632715606, + "jobId" : { + "cluster" : "test@cluster", + "jobEpoch" : 1470171371859, + "jobSequence" : 6608374, + "jobIdString" : "job_1470171371859_6608374" + }, + "qualifiedJobId" : { + "cluster" : "test@cluster", + "jobEpoch" : 1470171371859, + "jobSequence" : 6608374, + "jobIdString" : "job_1470171371859_6608374" + }, + "encodedRunId" : 9223370544222060201 + }, + "jobId" : "job_1470171371859_6608374", + "jobName" : "[C6D9BD40285E426AA4186394C126B551/B89F9F1DAE1B4A98B710A4D32EC55873] com.twitter.testuser.forward.testuserUnhydratedBatchJob/(2/3)", + "user" : "testuser", + "priority" : "", + "status" : "SUCCEEDED", + "version" : "02CFBD0A94AD5E297C2E4D6665B3B6F0", + "hadoopVersion" : "TWO", + "queue" : "testuser", + "submitTime" : 1492632797525, + "launchTime" : 1492632810388, + "finishTime" : 1492633163993, + "totalMaps" : 129, + "totalReduces" : 64, + "finishedMaps" : 129, + "finishedReduces" : 64, + "failedMaps" : 0, + "failedReduces" : 0, + "mapFileBytesRead" : 38269393803, + "mapFileBytesWritten" : 76454763405, + "reduceFileBytesRead" : 39043379028, + "hdfsBytesRead" : 96004737363, + "hdfsBytesWritten" : 38303868081, + "mapSlotMillis" : 14364674, + "reduceSlotMillis" : 6616707, + "reduceShuffleBytes" : 38146657114, + "megabyteMillis" : 67245979136, + "cost" : 0.07296655722222221, + "counters" : { + "Scalding Custom" : { + "test_user" : 810 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 16775502, + "Tuples_Read" : 81604481 + }, + "evictions" : { + "MapsideReduce" : 12771518 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "REDUCE_INPUT_RECORDS" : 16803506, + "SPILLED_RECORDS" : 50771480, + "VIRTUAL_MEMORY_BYTES" : 786154287104, + "LOCALIZED_NANOS" : 261150864082, + "SPLIT_RAW_BYTES" : 342753, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 131133636940, + "REDUCE_SHUFFLE_BYTES" : 38146657114, + "PHYSICAL_MEMORY_BYTES" : 232995028992, + "COMBINE_OUTPUT_RECORDS" : 0, + "LOCALIZED_FILES_CACHED" : 194946, + "CPU_MILLISECONDS" : 15268200, + "LOCALIZED_FILES_MISSED" : 24664, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 38146657114, + "MERGED_MAP_OUTPUTS" : 8256, + "MAP_INPUT_RECORDS" : 81604481, + "LOCALIZED_BYTES_MISSED" : 29607543994, + "GC_TIME_MILLIS" : 3649536, + "REDUCE_INPUT_GROUPS" : 16775502, + "SHUFFLED_MAPS" : 8256, + "REDUCE_OUTPUT_RECORDS" : 16775502, + "LOCALIZED_BYTES_CACHED" : 177307666049, + "MAP_OUTPUT_RECORDS" : 16803506, + "COMBINE_INPUT_RECORDS" : 0, + "COMMITTED_HEAP_BYTES" : 229451812864 + }, + "hits" : { + "MapsideReduce" : 1603 + }, + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 115583763634, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 708, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 96004737363, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 77312772831, + "HDFS_WRITE_OPS" : 128, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 38303868081 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 33579008, + "Tuples_Read" : 98407987, + "Read_Duration" : 8487639, + "Write_Duration" : 2047170, + "Process_Begin_Time" : 288078153269647, + "Process_End_Time" : 288078164714486 + }, + "LzoBlocks of com.twitter.tweetypie.thriftscala.TweetEvent" : { + "Errors" : 0, + "Records Read" : 81604481, + "Records Skipped" : 0 + }, + "misses" : { + "MapsideReduce" : 16803506 + }, + "org.apache.hadoop.mapreduce.JobCounter" : { + "TOTAL_LAUNCHED_MAPS" : 141, + "VCORES_MILLIS_REDUCES" : 5789619, + "TOTAL_LAUNCHED_REDUCES" : 65, + "NUM_KILLED_MAPS" : 12, + "OTHER_LOCAL_MAPS" : 77, + "DATA_LOCAL_MAPS" : 1, + "NUM_KILLED_REDUCES" : 1, + "MB_MILLIS_MAPS" : 45047618560, + "SLOTS_MILLIS_REDUCES" : 6616707, + "VCORES_MILLIS_MAPS" : 12569090, + "MB_MILLIS_REDUCES" : 20749994496, + "SLOTS_MILLIS_MAPS" : 14364674, + "RACK_LOCAL_MAPS" : 64, + "MILLIS_REDUCES" : 5789619, + "MILLIS_MAPS" : 12569090 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + } + }, + "mapCounters" : { + "Scalding Custom" : { + "test_user" : 810 + }, + "hits" : { + "MapsideReduce" : 1603 + }, + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 76454763405, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 516, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 96004737363, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 38269393803, + "HDFS_WRITE_OPS" : 0, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Read" : 81604481 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 16803506, + "Tuples_Read" : 81604481, + "Read_Duration" : 6942029, + "Write_Duration" : 1399751, + "Process_Begin_Time" : 192549636233632, + "Process_End_Time" : 192549645230601 + }, + "LzoBlocks of com.twitter.tweetypie.thriftscala.TweetEvent" : { + "Errors" : 0, + "Records Read" : 81604481, + "Records Skipped" : 0 + }, + "misses" : { + "MapsideReduce" : 16803506 + }, + "evictions" : { + "MapsideReduce" : 12771518 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 16732, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 38146657114, + "MERGED_MAP_OUTPUTS" : 0, + "SPILLED_RECORDS" : 33575024, + "VIRTUAL_MEMORY_BYTES" : 521721344000, + "MAP_INPUT_RECORDS" : 81604481, + "LOCALIZED_NANOS" : 180981945921, + "SPLIT_RAW_BYTES" : 342753, + "LOCALIZED_BYTES_MISSED" : 20331938692, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 131133636940, + "PHYSICAL_MEMORY_BYTES" : 203313250304, + "GC_TIME_MILLIS" : 3612486, + "LOCALIZED_FILES_CACHED" : 129296, + "LOCALIZED_BYTES_CACHED" : 117255771650, + "MAP_OUTPUT_RECORDS" : 16803506, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 11626970, + "COMMITTED_HEAP_BYTES" : 197002711040 + } + }, + "reduceCounters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 39129000229, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 192, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 0, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 39043379028, + "HDFS_WRITE_OPS" : 128, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 38303868081 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 16775502 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 16775502, + "Tuples_Read" : 16803506, + "Read_Duration" : 1545610, + "Write_Duration" : 647419, + "Process_Begin_Time" : 95528517036015, + "Process_End_Time" : 95528519483885 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 7927, + "MERGED_MAP_OUTPUTS" : 8256, + "REDUCE_INPUT_RECORDS" : 16803506, + "SPILLED_RECORDS" : 17196456, + "VIRTUAL_MEMORY_BYTES" : 264432943104, + "LOCALIZED_NANOS" : 79671609404, + "LOCALIZED_BYTES_MISSED" : 9269815365, + "FAILED_SHUFFLE" : 0, + "REDUCE_SHUFFLE_BYTES" : 38146657114, + "PHYSICAL_MEMORY_BYTES" : 29681778688, + "GC_TIME_MILLIS" : 37050, + "REDUCE_INPUT_GROUPS" : 16775502, + "COMBINE_OUTPUT_RECORDS" : 0, + "SHUFFLED_MAPS" : 8256, + "LOCALIZED_FILES_CACHED" : 64521, + "REDUCE_OUTPUT_RECORDS" : 16775502, + "LOCALIZED_BYTES_CACHED" : 58990754107, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 3641230, + "COMMITTED_HEAP_BYTES" : 32449101824 + } + }, + "tasks" : [ ], + "configuration" : { + "cascading.flow.step.num" : "2" + }, + "submitDate" : 1492632797525, + "launchDate" : 1492632810388, + "finishDate" : 1492633163993, + "runTime" : 353605 + }, { + "jobKey" : { + "cluster" : "test@cluster", + "userName" : "testuser", + "appId" : "somegoodjob", + "runId" : 1492632715606, + "jobId" : { + "cluster" : "test@cluster", + "jobEpoch" : 1470171371859, + "jobSequence" : 6608570, + "jobIdString" : "job_1470171371859_6608570" + }, + "qualifiedJobId" : { + "cluster" : "test@cluster", + "jobEpoch" : 1470171371859, + "jobSequence" : 6608570, + "jobIdString" : "job_1470171371859_6608570" + }, + "encodedRunId" : 9223370544222060201 + }, + "jobId" : "job_1470171371859_6608570", + "jobName" : "[C6D9BD40285E426AA4186394C126B551/953E3620701A4D9183064DE40D3509F2] com.twitter.testuser.forward.testuserUnhydratedBatchJob/(1/3)", + "user" : "testuser", + "priority" : "", + "status" : "SUCCEEDED", + "version" : "02CFBD0A94AD5E297C2E4D6665B3B6F0", + "hadoopVersion" : "TWO", + "queue" : "testuser", + "submitTime" : 1492633250502, + "launchTime" : 1492633263806, + "finishTime" : 1492633490918, + "totalMaps" : 193, + "totalReduces" : 63, + "finishedMaps" : 193, + "finishedReduces" : 63, + "failedMaps" : 0, + "failedReduces" : 0, + "mapFileBytesRead" : 38605064146, + "mapFileBytesWritten" : 77444144858, + "reduceFileBytesRead" : 64369198442, + "hdfsBytesRead" : 134308670528, + "hdfsBytesWritten" : 36314427837, + "mapSlotMillis" : 11418369, + "reduceSlotMillis" : 5361488, + "reduceShuffleBytes" : 38721089102, + "megabyteMillis" : 53551882752, + "cost" : 0.05810751166666666, + "counters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 141897586642, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 961, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 134308670528, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 102974262588, + "HDFS_WRITE_OPS" : 126, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 36314427837 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 16012367, + "Tuples_Read" : 98379983 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 55937549, + "Tuples_Read" : 138305165, + "Read_Duration" : 6707040, + "Write_Duration" : 1074482, + "Process_Begin_Time" : 382114128351773, + "Process_End_Time" : 382114136600916 + }, + "LzoBlocks of com.twitter.tweetypie.thriftscala.TweetEvent" : { + "Errors" : 0, + "Records Read" : 81604481, + "Records Skipped" : 0 + }, + "org.apache.hadoop.mapreduce.JobCounter" : { + "TOTAL_LAUNCHED_MAPS" : 196, + "VCORES_MILLIS_REDUCES" : 4691302, + "TOTAL_LAUNCHED_REDUCES" : 66, + "NUM_KILLED_MAPS" : 3, + "OTHER_LOCAL_MAPS" : 86, + "DATA_LOCAL_MAPS" : 15, + "NUM_KILLED_REDUCES" : 3, + "MB_MILLIS_MAPS" : 35808005632, + "SLOTS_MILLIS_REDUCES" : 5361488, + "VCORES_MILLIS_MAPS" : 9991073, + "MB_MILLIS_REDUCES" : 16813626368, + "SLOTS_MILLIS_MAPS" : 11418369, + "RACK_LOCAL_MAPS" : 95, + "MILLIS_REDUCES" : 4691302, + "MILLIS_MAPS" : 9991073 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "REDUCE_INPUT_RECORDS" : 39925182, + "SPILLED_RECORDS" : 122551413, + "VIRTUAL_MEMORY_BYTES" : 1040169828352, + "LOCALIZED_NANOS" : 328012266597, + "SPLIT_RAW_BYTES" : 407837, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 131829322449, + "REDUCE_SHUFFLE_BYTES" : 38721089102, + "PHYSICAL_MEMORY_BYTES" : 177282752512, + "COMBINE_OUTPUT_RECORDS" : 0, + "LOCALIZED_FILES_CACHED" : 260456, + "CPU_MILLISECONDS" : 13393960, + "LOCALIZED_FILES_MISSED" : 30470, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 38721089102, + "MERGED_MAP_OUTPUTS" : 12159, + "MAP_INPUT_RECORDS" : 98379983, + "LOCALIZED_BYTES_MISSED" : 35659339426, + "GC_TIME_MILLIS" : 114044, + "REDUCE_INPUT_GROUPS" : 39120398, + "SHUFFLED_MAPS" : 12159, + "REDUCE_OUTPUT_RECORDS" : 16012367, + "LOCALIZED_BYTES_CACHED" : 238450653585, + "MAP_OUTPUT_RECORDS" : 39925182, + "COMBINE_INPUT_RECORDS" : 0, + "COMMITTED_HEAP_BYTES" : 188948316160 + } + }, + "mapCounters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 77444144858, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 772, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 134308670528, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 38605064146, + "HDFS_WRITE_OPS" : 0, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Read" : 98379983 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 39925182, + "Tuples_Read" : 98379983, + "Read_Duration" : 5140255, + "Write_Duration" : 483616, + "Process_Begin_Time" : 288078225244627, + "Process_End_Time" : 288078231048777 + }, + "LzoBlocks of com.twitter.tweetypie.thriftscala.TweetEvent" : { + "Errors" : 0, + "Records Read" : 81604481, + "Records Skipped" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 23153, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 38721089102, + "MERGED_MAP_OUTPUTS" : 0, + "SPILLED_RECORDS" : 56700684, + "VIRTUAL_MEMORY_BYTES" : 780523053056, + "MAP_INPUT_RECORDS" : 98379983, + "LOCALIZED_NANOS" : 259273008863, + "SPLIT_RAW_BYTES" : 407837, + "LOCALIZED_BYTES_MISSED" : 27242131551, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 131829322449, + "PHYSICAL_MEMORY_BYTES" : 149129138176, + "GC_TIME_MILLIS" : 75962, + "LOCALIZED_FILES_CACHED" : 195323, + "LOCALIZED_BYTES_CACHED" : 178606685961, + "MAP_OUTPUT_RECORDS" : 39925182, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 9260920, + "COMMITTED_HEAP_BYTES" : 157008166912 + } + }, + "reduceCounters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 64453441784, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 189, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 0, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 64369198442, + "HDFS_WRITE_OPS" : 126, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 36314427837 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 16012367 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 16012367, + "Tuples_Read" : 39925182, + "Read_Duration" : 1566785, + "Write_Duration" : 590866, + "Process_Begin_Time" : 94035903107146, + "Process_End_Time" : 94035905552139 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 7313, + "MERGED_MAP_OUTPUTS" : 12159, + "REDUCE_INPUT_RECORDS" : 39925182, + "SPILLED_RECORDS" : 65850729, + "VIRTUAL_MEMORY_BYTES" : 259646775296, + "LOCALIZED_NANOS" : 68072310662, + "LOCALIZED_BYTES_MISSED" : 8415594516, + "FAILED_SHUFFLE" : 0, + "REDUCE_SHUFFLE_BYTES" : 38721089102, + "PHYSICAL_MEMORY_BYTES" : 28153614336, + "GC_TIME_MILLIS" : 38082, + "REDUCE_INPUT_GROUPS" : 39120398, + "COMBINE_OUTPUT_RECORDS" : 0, + "SHUFFLED_MAPS" : 12159, + "LOCALIZED_FILES_CACHED" : 64003, + "REDUCE_OUTPUT_RECORDS" : 16012367, + "LOCALIZED_BYTES_CACHED" : 58778579076, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 4133040, + "COMMITTED_HEAP_BYTES" : 31940149248 + } + }, + "tasks" : [ ], + "configuration" : { + "cascading.flow.step.num" : "1" + }, + "submitDate" : 1492633250502, + "launchDate" : 1492633263806, + "finishDate" : 1492633490918, + "runTime" : 227112 + }, { + "jobKey" : { + "cluster" : "test@cluster", + "userName" : "testuser", + "appId" : "somegoodjob", + "runId" : 1492632715606, + "jobId" : { + "cluster" : "test@cluster", + "jobEpoch" : 1470171371859, + "jobSequence" : 6608659, + "jobIdString" : "job_1470171371859_6608659" + }, + "qualifiedJobId" : { + "cluster" : "test@cluster", + "jobEpoch" : 1470171371859, + "jobSequence" : 6608659, + "jobIdString" : "job_1470171371859_6608659" + }, + "encodedRunId" : 9223370544222060201 + }, + "jobId" : "job_1470171371859_6608659", + "jobName" : "[C6D9BD40285E426AA4186394C126B551/F612986BE34B4779BBC6323D3B3011F1] com.twitter.testuser.forward.testuserUnhydratedBatchJob/(3/3) .../unhydrated/2017/04/19/18", + "user" : "testuser", + "priority" : "", + "status" : "SUCCEEDED", + "version" : "02CFBD0A94AD5E297C2E4D6665B3B6F0", + "hadoopVersion" : "TWO", + "queue" : "testuser", + "submitTime" : 1492633529277, + "launchTime" : 1492633543389, + "finishTime" : 1492634166086, + "totalMaps" : 63, + "totalReduces" : 10, + "finishedMaps" : 63, + "finishedReduces" : 10, + "failedMaps" : 0, + "failedReduces" : 0, + "mapFileBytesRead" : 36297705285, + "mapFileBytesWritten" : 72646048920, + "reduceFileBytesRead" : 59838419187, + "hdfsBytesRead" : 36314492727, + "hdfsBytesWritten" : 28989147723, + "mapSlotMillis" : 3548289, + "reduceSlotMillis" : 5140829, + "reduceShuffleBytes" : 36277152889, + "megabyteMillis" : 29799643648, + "cost" : 0.032334682777777776, + "counters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 132498587803, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 292, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 36314492727, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 96136124472, + "HDFS_WRITE_OPS" : 40, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 28989147723 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 16012367, + "Tuples_Read" : 16012367 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 32024734, + "Tuples_Read" : 32024734, + "Read_Duration" : 2166295, + "Write_Duration" : 3088855, + "Process_Begin_Time" : 108962251395918, + "Process_End_Time" : 108962256741645 + }, + "org.apache.hadoop.mapreduce.JobCounter" : { + "TOTAL_LAUNCHED_MAPS" : 66, + "VCORES_MILLIS_REDUCES" : 4498226, + "TOTAL_LAUNCHED_REDUCES" : 10, + "NUM_KILLED_MAPS" : 2, + "OTHER_LOCAL_MAPS" : 42, + "DATA_LOCAL_MAPS" : 3, + "MB_MILLIS_MAPS" : 11127434752, + "SLOTS_MILLIS_REDUCES" : 5140829, + "VCORES_MILLIS_MAPS" : 3104753, + "NUM_FAILED_MAPS" : 1, + "MB_MILLIS_REDUCES" : 16121641984, + "SLOTS_MILLIS_MAPS" : 3548289, + "RACK_LOCAL_MAPS" : 21, + "MILLIS_REDUCES" : 4498226, + "MILLIS_MAPS" : 3104753 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "REDUCE_INPUT_RECORDS" : 16012367, + "SPILLED_RECORDS" : 58451613, + "VIRTUAL_MEMORY_BYTES" : 296636551168, + "LOCALIZED_NANOS" : 93095362314, + "SPLIT_RAW_BYTES" : 64890, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 124053498684, + "REDUCE_SHUFFLE_BYTES" : 36277152889, + "PHYSICAL_MEMORY_BYTES" : 54054834176, + "COMBINE_OUTPUT_RECORDS" : 0, + "LOCALIZED_FILES_CACHED" : 71077, + "CPU_MILLISECONDS" : 7467830, + "LOCALIZED_FILES_MISSED" : 12619, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 36277152889, + "MERGED_MAP_OUTPUTS" : 630, + "MAP_INPUT_RECORDS" : 16012367, + "LOCALIZED_BYTES_MISSED" : 14721909277, + "GC_TIME_MILLIS" : 40741, + "REDUCE_INPUT_GROUPS" : 10, + "SHUFFLED_MAPS" : 630, + "REDUCE_OUTPUT_RECORDS" : 16012367, + "LOCALIZED_BYTES_CACHED" : 64200727902, + "MAP_OUTPUT_RECORDS" : 16012367, + "COMBINE_INPUT_RECORDS" : 0, + "COMMITTED_HEAP_BYTES" : 56227004416 + } + }, + "mapCounters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 72646048920, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 252, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 36314492727, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 36297705285, + "HDFS_WRITE_OPS" : 0, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Read" : 16012367 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 16012367, + "Tuples_Read" : 16012367, + "Read_Duration" : 1217078, + "Write_Duration" : 286640, + "Process_Begin_Time" : 94035914524573, + "Process_End_Time" : 94035916053550 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 11942, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 36277152889, + "MERGED_MAP_OUTPUTS" : 0, + "SPILLED_RECORDS" : 32024734, + "VIRTUAL_MEMORY_BYTES" : 255277658112, + "MAP_INPUT_RECORDS" : 16012367, + "LOCALIZED_NANOS" : 85756412733, + "SPLIT_RAW_BYTES" : 64890, + "LOCALIZED_BYTES_MISSED" : 13922008661, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 124053498684, + "PHYSICAL_MEMORY_BYTES" : 49704181760, + "GC_TIME_MILLIS" : 26295, + "LOCALIZED_FILES_CACHED" : 59311, + "LOCALIZED_BYTES_CACHED" : 53268826750, + "MAP_OUTPUT_RECORDS" : 16012367, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 3061540, + "COMMITTED_HEAP_BYTES" : 51157139456 + } + }, + "reduceCounters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 59852538883, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 40, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 0, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 59838419187, + "HDFS_WRITE_OPS" : 40, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 28989147723 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 16012367 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 16012367, + "Tuples_Read" : 16012367, + "Read_Duration" : 949217, + "Write_Duration" : 2802215, + "Process_Begin_Time" : 14926336871345, + "Process_End_Time" : 14926340688095 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 674, + "MERGED_MAP_OUTPUTS" : 630, + "REDUCE_INPUT_RECORDS" : 16012367, + "SPILLED_RECORDS" : 26426879, + "VIRTUAL_MEMORY_BYTES" : 41358893056, + "LOCALIZED_NANOS" : 6946432106, + "LOCALIZED_BYTES_MISSED" : 798699366, + "FAILED_SHUFFLE" : 0, + "REDUCE_SHUFFLE_BYTES" : 36277152889, + "PHYSICAL_MEMORY_BYTES" : 4350652416, + "GC_TIME_MILLIS" : 14446, + "REDUCE_INPUT_GROUPS" : 10, + "COMBINE_OUTPUT_RECORDS" : 0, + "SHUFFLED_MAPS" : 630, + "LOCALIZED_FILES_CACHED" : 10636, + "REDUCE_OUTPUT_RECORDS" : 16012367, + "LOCALIZED_BYTES_CACHED" : 9866512604, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 4406290, + "COMMITTED_HEAP_BYTES" : 5069864960 + } + }, + "tasks" : [ ], + "configuration" : { + "cascading.flow.step.num" : "3" + }, + "submitDate" : 1492633529277, + "launchDate" : 1492633543389, + "finishDate" : 1492634166086, + "runTime" : 622697 + } ] +}, { + "flowKey" : { + "cluster" : "test@cluster", + "userName" : "testuser", + "appId" : "somegoodjob", + "runId" : 1492629120238, + "encodedRunId" : 9223370544225655569 + }, + "flowName" : "somegoodjob", + "userName" : "testuser", + "jobCount" : 3, + "totalMaps" : 415, + "totalReduces" : 141, + "mapFileBytesRead" : 119652359797, + "mapFileBytesWritten" : 239507162599, + "reduceFileBytesRead" : 184321388670, + "hdfsBytesRead" : 282024326201, + "hdfsBytesWritten" : 109915817955, + "mapSlotMillis" : 29834574, + "reduceSlotMillis" : 17818367, + "megabyteMillis" : 154376586752, + "cost" : 0.1675093172222222, + "reduceShuffleBytes" : 119607357030, + "duration" : 1386911, + "wallClockTime" : 1422000, + "cluster" : "test@cluster", + "appId" : "somegoodjob", + "runId" : 1492629120238, + "version" : "02CFBD0A94AD5E297C2E4D6665B3B6F0", + "hadoopVersion" : "TWO", + "submitTime" : 1492629217932, + "launchTime" : 1492629253021, + "finishTime" : 1492630639932, + "queue" : "testuser", + "counters" : { + "Scalding Custom" : { + "test_user" : 805 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 51711261, + "Tuples_Read" : 206828887 + }, + "evictions" : { + "MapsideReduce" : 12711135 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "REDUCE_INPUT_RECORDS" : 75745072, + "SPILLED_RECORDS" : 247366146, + "VIRTUAL_MEMORY_BYTES" : 2261545836544, + "LOCALIZED_NANOS" : 621652509094, + "SPLIT_RAW_BYTES" : 888563, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 410957748527, + "REDUCE_SHUFFLE_BYTES" : 119607357030, + "PHYSICAL_MEMORY_BYTES" : 514274934784, + "COMBINE_OUTPUT_RECORDS" : 0, + "LOCALIZED_FILES_CACHED" : 573353, + "CPU_MILLISECONDS" : 36940780, + "LOCALIZED_FILES_MISSED" : 59363, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 119607357030, + "MERGED_MAP_OUTPUTS" : 23560, + "MAP_INPUT_RECORDS" : 206828887, + "LOCALIZED_BYTES_MISSED" : 69180413795, + "GC_TIME_MILLIS" : 3829080, + "REDUCE_INPUT_GROUPS" : 57952423, + "SHUFFLED_MAPS" : 23560, + "REDUCE_OUTPUT_RECORDS" : 51711261, + "LOCALIZED_BYTES_CACHED" : 527031421131, + "MAP_OUTPUT_RECORDS" : 75745072, + "COMBINE_INPUT_RECORDS" : 0, + "COMMITTED_HEAP_BYTES" : 524035977216 + }, + "hits" : { + "MapsideReduce" : 1935 + }, + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 424018044192, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 2093, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 282024326201, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 303973748467, + "HDFS_WRITE_OPS" : 302, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 109915817955 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 127456333, + "Tuples_Read" : 282573959, + "Read_Duration" : 17267767, + "Write_Duration" : 6329319, + "Process_Begin_Time" : 829902072807036, + "Process_End_Time" : 829902097833076 + }, + "LzoBlocks of com.twitter.tweetypie.thriftscala.TweetEvent" : { + "Errors" : 0, + "Records Read" : 172118504, + "Records Skipped" : 0 + }, + "misses" : { + "MapsideReduce" : 17739761 + }, + "org.apache.hadoop.mapreduce.JobCounter" : { + "TOTAL_LAUNCHED_MAPS" : 432, + "VCORES_MILLIS_REDUCES" : 15591072, + "TOTAL_LAUNCHED_REDUCES" : 145, + "NUM_KILLED_MAPS" : 17, + "OTHER_LOCAL_MAPS" : 204, + "DATA_LOCAL_MAPS" : 15, + "NUM_KILLED_REDUCES" : 4, + "MB_MILLIS_MAPS" : 93561226752, + "SLOTS_MILLIS_REDUCES" : 17818367, + "VCORES_MILLIS_MAPS" : 26105253, + "MB_MILLIS_REDUCES" : 55878402048, + "SLOTS_MILLIS_MAPS" : 29834574, + "RACK_LOCAL_MAPS" : 213, + "MILLIS_REDUCES" : 15591072, + "MILLIS_MAPS" : 26105253 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + } + }, + "mapCounters" : { + "Scalding Custom" : { + "test_user" : 805 + }, + "hits" : { + "MapsideReduce" : 1935 + }, + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 239507162599, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 1660, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 282024326201, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 119652359797, + "HDFS_WRITE_OPS" : 0, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Read" : 206828887 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 75745072, + "Tuples_Read" : 206828887, + "Read_Duration" : 13387512, + "Write_Duration" : 2169585, + "Process_Begin_Time" : 619441287516435, + "Process_End_Time" : 619441303935634 + }, + "LzoBlocks of com.twitter.tweetypie.thriftscala.TweetEvent" : { + "Errors" : 0, + "Records Read" : 172118504, + "Records Skipped" : 0 + }, + "misses" : { + "MapsideReduce" : 17739761 + }, + "evictions" : { + "MapsideReduce" : 12711135 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 45265, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 119607357030, + "MERGED_MAP_OUTPUTS" : 0, + "SPILLED_RECORDS" : 128166590, + "VIRTUAL_MEMORY_BYTES" : 1678871752704, + "MAP_INPUT_RECORDS" : 206828887, + "LOCALIZED_NANOS" : 439998797901, + "SPLIT_RAW_BYTES" : 888563, + "LOCALIZED_BYTES_MISSED" : 52866562203, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 410957748527, + "PHYSICAL_MEMORY_BYTES" : 450766368768, + "GC_TIME_MILLIS" : 3740200, + "LOCALIZED_FILES_CACHED" : 424448, + "LOCALIZED_BYTES_CACHED" : 389758256396, + "MAP_OUTPUT_RECORDS" : 75745072, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 24525290, + "COMMITTED_HEAP_BYTES" : 452546949120 + } + }, + "reduceCounters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 184510881593, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 433, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 0, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 184321388670, + "HDFS_WRITE_OPS" : 302, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 109915817955 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 51711261 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 51711261, + "Tuples_Read" : 75745072, + "Read_Duration" : 3880255, + "Write_Duration" : 4159734, + "Process_Begin_Time" : 210460785290601, + "Process_End_Time" : 210460793897442 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 14087, + "MERGED_MAP_OUTPUTS" : 23560, + "REDUCE_INPUT_RECORDS" : 75745072, + "SPILLED_RECORDS" : 119199556, + "VIRTUAL_MEMORY_BYTES" : 582674083840, + "LOCALIZED_NANOS" : 175613465264, + "LOCALIZED_BYTES_MISSED" : 16309415297, + "FAILED_SHUFFLE" : 0, + "REDUCE_SHUFFLE_BYTES" : 119607357030, + "PHYSICAL_MEMORY_BYTES" : 63508566016, + "GC_TIME_MILLIS" : 88880, + "REDUCE_INPUT_GROUPS" : 57952423, + "COMBINE_OUTPUT_RECORDS" : 0, + "SHUFFLED_MAPS" : 23560, + "LOCALIZED_FILES_CACHED" : 145515, + "REDUCE_OUTPUT_RECORDS" : 51711261, + "LOCALIZED_BYTES_CACHED" : 134076999091, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 12415490, + "COMMITTED_HEAP_BYTES" : 71489028096 + } + }, + "jobs" : [ { + "jobKey" : { + "cluster" : "test@cluster", + "userName" : "testuser", + "appId" : "somegoodjob", + "runId" : 1492629120238, + "jobId" : { + "cluster" : "test@cluster", + "jobEpoch" : 1470171371859, + "jobSequence" : 6607397, + "jobIdString" : "job_1470171371859_6607397" + }, + "qualifiedJobId" : { + "cluster" : "test@cluster", + "jobEpoch" : 1470171371859, + "jobSequence" : 6607397, + "jobIdString" : "job_1470171371859_6607397" + }, + "encodedRunId" : 9223370544225655569 + }, + "jobId" : "job_1470171371859_6607397", + "jobName" : "[8BEFB46B10BB4992B069A0452C1A7209/6CEF6CE735EA4F22A1D10798AB978AC9] com.twitter.testuser.forward.testuserUnhydratedBatchJob/(2/3)", + "user" : "testuser", + "priority" : "", + "status" : "SUCCEEDED", + "version" : "02CFBD0A94AD5E297C2E4D6665B3B6F0", + "hadoopVersion" : "TWO", + "queue" : "testuser", + "submitTime" : 1492629217932, + "launchTime" : 1492629253021, + "finishTime" : 1492629545471, + "totalMaps" : 142, + "totalReduces" : 64, + "finishedMaps" : 142, + "finishedReduces" : 64, + "failedMaps" : 0, + "failedReduces" : 0, + "mapFileBytesRead" : 40340004073, + "mapFileBytesWritten" : 80595282050, + "reduceFileBytesRead" : 52002327599, + "hdfsBytesRead" : 101477835921, + "hdfsBytesWritten" : 40416854940, + "mapSlotMillis" : 13609547, + "reduceSlotMillis" : 6905340, + "reduceShuffleBytes" : 40209861419, + "megabyteMillis" : 65532563968, + "cost" : 0.07110738277777777, + "counters" : { + "Scalding Custom" : { + "test_user" : 805 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 17709505, + "Tuples_Read" : 86059252 + }, + "evictions" : { + "MapsideReduce" : 12711135 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "REDUCE_INPUT_RECORDS" : 17739761, + "SPILLED_RECORDS" : 58676037, + "VIRTUAL_MEMORY_BYTES" : 838904201216, + "LOCALIZED_NANOS" : 198603202103, + "SPLIT_RAW_BYTES" : 376726, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 138812514335, + "REDUCE_SHUFFLE_BYTES" : 40209861419, + "PHYSICAL_MEMORY_BYTES" : 268000686080, + "COMBINE_OUTPUT_RECORDS" : 0, + "LOCALIZED_FILES_CACHED" : 213660, + "CPU_MILLISECONDS" : 15391480, + "LOCALIZED_FILES_MISSED" : 20666, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 40209861419, + "MERGED_MAP_OUTPUTS" : 9088, + "MAP_INPUT_RECORDS" : 86059252, + "LOCALIZED_BYTES_MISSED" : 24468451612, + "GC_TIME_MILLIS" : 3670350, + "REDUCE_INPUT_GROUPS" : 17709505, + "SHUFFLED_MAPS" : 9088, + "REDUCE_OUTPUT_RECORDS" : 17709505, + "LOCALIZED_BYTES_CACHED" : 196312461278, + "MAP_OUTPUT_RECORDS" : 17739761, + "COMBINE_INPUT_RECORDS" : 0, + "COMMITTED_HEAP_BYTES" : 263016857600 + }, + "hits" : { + "MapsideReduce" : 1935 + }, + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 132683302440, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 760, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 101477835921, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 92342331672, + "HDFS_WRITE_OPS" : 128, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 40416854940 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 35449266, + "Tuples_Read" : 103799013, + "Read_Duration" : 8173153, + "Write_Duration" : 1968986, + "Process_Begin_Time" : 307481645061346, + "Process_End_Time" : 307481656087835 + }, + "LzoBlocks of com.twitter.tweetypie.thriftscala.TweetEvent" : { + "Errors" : 0, + "Records Read" : 86059252, + "Records Skipped" : 0 + }, + "misses" : { + "MapsideReduce" : 17739761 + }, + "org.apache.hadoop.mapreduce.JobCounter" : { + "TOTAL_LAUNCHED_MAPS" : 153, + "VCORES_MILLIS_REDUCES" : 6042173, + "TOTAL_LAUNCHED_REDUCES" : 64, + "NUM_KILLED_MAPS" : 11, + "OTHER_LOCAL_MAPS" : 79, + "DATA_LOCAL_MAPS" : 3, + "MB_MILLIS_MAPS" : 42679540736, + "SLOTS_MILLIS_REDUCES" : 6905340, + "VCORES_MILLIS_MAPS" : 11908354, + "MB_MILLIS_REDUCES" : 21655148032, + "SLOTS_MILLIS_MAPS" : 13609547, + "RACK_LOCAL_MAPS" : 71, + "MILLIS_REDUCES" : 6042173, + "MILLIS_MAPS" : 11908354 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + } + }, + "mapCounters" : { + "Scalding Custom" : { + "test_user" : 805 + }, + "hits" : { + "MapsideReduce" : 1935 + }, + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 80595282050, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 568, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 101477835921, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 40340004073, + "HDFS_WRITE_OPS" : 0, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Read" : 86059252 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 17739761, + "Tuples_Read" : 86059252, + "Read_Duration" : 6780865, + "Write_Duration" : 1351893, + "Process_Begin_Time" : 211953357561478, + "Process_End_Time" : 211953366347172 + }, + "LzoBlocks of com.twitter.tweetypie.thriftscala.TweetEvent" : { + "Errors" : 0, + "Records Read" : 86059252, + "Records Skipped" : 0 + }, + "misses" : { + "MapsideReduce" : 17739761 + }, + "evictions" : { + "MapsideReduce" : 12711135 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 14551, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 40209861419, + "MERGED_MAP_OUTPUTS" : 0, + "SPILLED_RECORDS" : 35450896, + "VIRTUAL_MEMORY_BYTES" : 574310744064, + "MAP_INPUT_RECORDS" : 86059252, + "LOCALIZED_NANOS" : 135576295314, + "SPLIT_RAW_BYTES" : 376726, + "LOCALIZED_BYTES_MISSED" : 17303870269, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 138812514335, + "PHYSICAL_MEMORY_BYTES" : 238844108800, + "GC_TIME_MILLIS" : 3634307, + "LOCALIZED_FILES_CACHED" : 146193, + "LOCALIZED_BYTES_CACHED" : 134149432257, + "MAP_OUTPUT_RECORDS" : 17739761, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 11684060, + "COMMITTED_HEAP_BYTES" : 230569721856 + } + }, + "reduceCounters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 52088020390, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 192, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 0, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 52002327599, + "HDFS_WRITE_OPS" : 128, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 40416854940 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 17709505 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 17709505, + "Tuples_Read" : 17739761, + "Read_Duration" : 1392288, + "Write_Duration" : 617093, + "Process_Begin_Time" : 95528287499868, + "Process_End_Time" : 95528289740663 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 6111, + "MERGED_MAP_OUTPUTS" : 9088, + "REDUCE_INPUT_RECORDS" : 17739761, + "SPILLED_RECORDS" : 23225141, + "VIRTUAL_MEMORY_BYTES" : 264593457152, + "LOCALIZED_NANOS" : 57979043535, + "LOCALIZED_BYTES_MISSED" : 7163002919, + "FAILED_SHUFFLE" : 0, + "REDUCE_SHUFFLE_BYTES" : 40209861419, + "PHYSICAL_MEMORY_BYTES" : 29156577280, + "GC_TIME_MILLIS" : 36043, + "REDUCE_INPUT_GROUPS" : 17709505, + "COMBINE_OUTPUT_RECORDS" : 0, + "SHUFFLED_MAPS" : 9088, + "LOCALIZED_FILES_CACHED" : 66337, + "REDUCE_OUTPUT_RECORDS" : 17709505, + "LOCALIZED_BYTES_CACHED" : 61097640473, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 3707420, + "COMMITTED_HEAP_BYTES" : 32447135744 + } + }, + "tasks" : [ ], + "configuration" : { + "cascading.flow.step.num" : "2" + }, + "submitDate" : 1492629217932, + "launchDate" : 1492629253021, + "finishDate" : 1492629545471, + "runTime" : 292450 + }, { + "jobKey" : { + "cluster" : "test@cluster", + "userName" : "testuser", + "appId" : "somegoodjob", + "runId" : 1492629120238, + "jobId" : { + "cluster" : "test@cluster", + "jobEpoch" : 1470171371859, + "jobSequence" : 6607542, + "jobIdString" : "job_1470171371859_6607542" + }, + "qualifiedJobId" : { + "cluster" : "test@cluster", + "jobEpoch" : 1470171371859, + "jobSequence" : 6607542, + "jobIdString" : "job_1470171371859_6607542" + }, + "encodedRunId" : 9223370544225655569 + }, + "jobId" : "job_1470171371859_6607542", + "jobName" : "[8BEFB46B10BB4992B069A0452C1A7209/6A94935E7180499FAC11473213CE76EF] com.twitter.testuser.forward.testuserUnhydratedBatchJob/(1/3)", + "user" : "testuser", + "priority" : "", + "status" : "SUCCEEDED", + "version" : "02CFBD0A94AD5E297C2E4D6665B3B6F0", + "hadoopVersion" : "TWO", + "queue" : "testuser", + "submitTime" : 1492629664906, + "launchTime" : 1492629678327, + "finishTime" : 1492629922728, + "totalMaps" : 206, + "totalReduces" : 67, + "finishedMaps" : 206, + "finishedReduces" : 67, + "failedMaps" : 0, + "failedReduces" : 0, + "mapFileBytesRead" : 40706072815, + "mapFileBytesWritten" : 81646497859, + "reduceFileBytesRead" : 67871106082, + "hdfsBytesRead" : 141894757029, + "hdfsBytesWritten" : 38651664308, + "mapSlotMillis" : 12455721, + "reduceSlotMillis" : 5467194, + "reduceShuffleBytes" : 40814059608, + "megabyteMillis" : 57207329280, + "cost" : 0.062073925, + "counters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 149607275269, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 1025, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 141894757029, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 108577178897, + "HDFS_WRITE_OPS" : 134, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 38651664308 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 17000878, + "Tuples_Read" : 103768757 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 58005311, + "Tuples_Read" : 144773190, + "Read_Duration" : 6765949, + "Write_Duration" : 1103794, + "Process_Begin_Time" : 407487916987017, + "Process_End_Time" : 407487925303041 + }, + "LzoBlocks of com.twitter.tweetypie.thriftscala.TweetEvent" : { + "Errors" : 0, + "Records Read" : 86059252, + "Records Skipped" : 0 + }, + "org.apache.hadoop.mapreduce.JobCounter" : { + "TOTAL_LAUNCHED_MAPS" : 210, + "VCORES_MILLIS_REDUCES" : 4783795, + "TOTAL_LAUNCHED_REDUCES" : 71, + "NUM_KILLED_MAPS" : 4, + "OTHER_LOCAL_MAPS" : 80, + "DATA_LOCAL_MAPS" : 12, + "NUM_KILLED_REDUCES" : 4, + "MB_MILLIS_MAPS" : 39061141504, + "SLOTS_MILLIS_REDUCES" : 5467194, + "VCORES_MILLIS_MAPS" : 10898756, + "MB_MILLIS_REDUCES" : 17145121280, + "SLOTS_MILLIS_MAPS" : 12455721, + "RACK_LOCAL_MAPS" : 118, + "MILLIS_REDUCES" : 4783795, + "MILLIS_MAPS" : 10898756 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "REDUCE_INPUT_RECORDS" : 41004433, + "SPILLED_RECORDS" : 126273593, + "VIRTUAL_MEMORY_BYTES" : 1110279864320, + "LOCALIZED_NANOS" : 345506465153, + "SPLIT_RAW_BYTES" : 442894, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 139514063958, + "REDUCE_SHUFFLE_BYTES" : 40814059608, + "PHYSICAL_MEMORY_BYTES" : 189141204992, + "COMBINE_OUTPUT_RECORDS" : 0, + "LOCALIZED_FILES_CACHED" : 280089, + "CPU_MILLISECONDS" : 13531660, + "LOCALIZED_FILES_MISSED" : 30081, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 40814059608, + "MERGED_MAP_OUTPUTS" : 13802, + "MAP_INPUT_RECORDS" : 103768757, + "LOCALIZED_BYTES_MISSED" : 34340180070, + "GC_TIME_MILLIS" : 115317, + "REDUCE_INPUT_GROUPS" : 40242908, + "SHUFFLED_MAPS" : 13802, + "REDUCE_OUTPUT_RECORDS" : 17000878, + "LOCALIZED_BYTES_CACHED" : 257901926946, + "MAP_OUTPUT_RECORDS" : 41004433, + "COMBINE_INPUT_RECORDS" : 0, + "COMMITTED_HEAP_BYTES" : 201552314368 + } + }, + "mapCounters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 81646497859, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 824, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 141894757029, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 40706072815, + "HDFS_WRITE_OPS" : 0, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Read" : 103768757 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 41004433, + "Tuples_Read" : 103768757, + "Read_Duration" : 5291658, + "Write_Duration" : 508458, + "Process_Begin_Time" : 307481720391658, + "Process_End_Time" : 307481726373180 + }, + "LzoBlocks of com.twitter.tweetypie.thriftscala.TweetEvent" : { + "Errors" : 0, + "Records Read" : 86059252, + "Records Skipped" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 22237, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 40814059608, + "MERGED_MAP_OUTPUTS" : 0, + "SPILLED_RECORDS" : 58713938, + "VIRTUAL_MEMORY_BYTES" : 833510277120, + "MAP_INPUT_RECORDS" : 103768757, + "LOCALIZED_NANOS" : 232395288843, + "SPLIT_RAW_BYTES" : 442894, + "LOCALIZED_BYTES_MISSED" : 25571400633, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 139514063958, + "PHYSICAL_MEMORY_BYTES" : 159147757568, + "GC_TIME_MILLIS" : 77909, + "LOCALIZED_FILES_CACHED" : 210955, + "LOCALIZED_BYTES_CACHED" : 194143118995, + "MAP_OUTPUT_RECORDS" : 41004433, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 9496830, + "COMMITTED_HEAP_BYTES" : 167580286976 + } + }, + "reduceCounters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 67960777410, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 201, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 0, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 67871106082, + "HDFS_WRITE_OPS" : 134, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 38651664308 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 17000878 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 17000878, + "Tuples_Read" : 41004433, + "Read_Duration" : 1474291, + "Write_Duration" : 595336, + "Process_Begin_Time" : 100006196595359, + "Process_End_Time" : 100006198929861 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 7840, + "MERGED_MAP_OUTPUTS" : 13802, + "REDUCE_INPUT_RECORDS" : 41004433, + "SPILLED_RECORDS" : 67559655, + "VIRTUAL_MEMORY_BYTES" : 276769587200, + "LOCALIZED_NANOS" : 112735693806, + "LOCALIZED_BYTES_MISSED" : 8767128243, + "FAILED_SHUFFLE" : 0, + "REDUCE_SHUFFLE_BYTES" : 40814059608, + "PHYSICAL_MEMORY_BYTES" : 29993447424, + "GC_TIME_MILLIS" : 37408, + "REDUCE_INPUT_GROUPS" : 40242908, + "COMBINE_OUTPUT_RECORDS" : 0, + "SHUFFLED_MAPS" : 13802, + "LOCALIZED_FILES_CACHED" : 68004, + "REDUCE_OUTPUT_RECORDS" : 17000878, + "LOCALIZED_BYTES_CACHED" : 62693419403, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 4034830, + "COMMITTED_HEAP_BYTES" : 33972027392 + } + }, + "tasks" : [ ], + "configuration" : { + "cascading.flow.step.num" : "1" + }, + "submitDate" : 1492629664906, + "launchDate" : 1492629678327, + "finishDate" : 1492629922728, + "runTime" : 244401 + }, { + "jobKey" : { + "cluster" : "test@cluster", + "userName" : "testuser", + "appId" : "somegoodjob", + "runId" : 1492629120238, + "jobId" : { + "cluster" : "test@cluster", + "jobEpoch" : 1470171371859, + "jobSequence" : 6607618, + "jobIdString" : "job_1470171371859_6607618" + }, + "qualifiedJobId" : { + "cluster" : "test@cluster", + "jobEpoch" : 1470171371859, + "jobSequence" : 6607618, + "jobIdString" : "job_1470171371859_6607618" + }, + "encodedRunId" : 9223370544225655569 + }, + "jobId" : "job_1470171371859_6607618", + "jobName" : "[8BEFB46B10BB4992B069A0452C1A7209/B2D7B4CDD6864E76BE5F19F61600A47B] com.twitter.testuser.forward.testuserUnhydratedBatchJob/(3/3) .../unhydrated/2017/04/19/17", + "user" : "testuser", + "priority" : "", + "status" : "SUCCEEDED", + "version" : "02CFBD0A94AD5E297C2E4D6665B3B6F0", + "hadoopVersion" : "TWO", + "queue" : "testuser", + "submitTime" : 1492629953696, + "launchTime" : 1492629971471, + "finishTime" : 1492630639932, + "totalMaps" : 67, + "totalReduces" : 10, + "finishedMaps" : 67, + "finishedReduces" : 10, + "failedMaps" : 0, + "failedReduces" : 0, + "mapFileBytesRead" : 38606282909, + "mapFileBytesWritten" : 77265382690, + "reduceFileBytesRead" : 64447954989, + "hdfsBytesRead" : 38651733251, + "hdfsBytesWritten" : 30847298707, + "mapSlotMillis" : 3769306, + "reduceSlotMillis" : 5445833, + "reduceShuffleBytes" : 38583436003, + "megabyteMillis" : 31636693504, + "cost" : 0.03432800944444444, + "counters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 141727466483, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 308, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 38651733251, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 103054237898, + "HDFS_WRITE_OPS" : 40, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 30847298707 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 17000878, + "Tuples_Read" : 17000878 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 34001756, + "Tuples_Read" : 34001756, + "Read_Duration" : 2328665, + "Write_Duration" : 3256539, + "Process_Begin_Time" : 114932510758673, + "Process_End_Time" : 114932516442200 + }, + "org.apache.hadoop.mapreduce.JobCounter" : { + "TOTAL_LAUNCHED_MAPS" : 69, + "VCORES_MILLIS_REDUCES" : 4765104, + "TOTAL_LAUNCHED_REDUCES" : 10, + "NUM_KILLED_MAPS" : 2, + "OTHER_LOCAL_MAPS" : 45, + "MB_MILLIS_MAPS" : 11820544512, + "SLOTS_MILLIS_REDUCES" : 5445833, + "VCORES_MILLIS_MAPS" : 3298143, + "MB_MILLIS_REDUCES" : 17078132736, + "SLOTS_MILLIS_MAPS" : 3769306, + "RACK_LOCAL_MAPS" : 24, + "MILLIS_REDUCES" : 4765104, + "MILLIS_MAPS" : 3298143 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "REDUCE_INPUT_RECORDS" : 17000878, + "SPILLED_RECORDS" : 62416516, + "VIRTUAL_MEMORY_BYTES" : 312361771008, + "LOCALIZED_NANOS" : 77542841838, + "SPLIT_RAW_BYTES" : 68943, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 132631170234, + "REDUCE_SHUFFLE_BYTES" : 38583436003, + "PHYSICAL_MEMORY_BYTES" : 57133043712, + "COMBINE_OUTPUT_RECORDS" : 0, + "LOCALIZED_FILES_CACHED" : 79604, + "CPU_MILLISECONDS" : 8017640, + "LOCALIZED_FILES_MISSED" : 8616, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 38583436003, + "MERGED_MAP_OUTPUTS" : 670, + "MAP_INPUT_RECORDS" : 17000878, + "LOCALIZED_BYTES_MISSED" : 10371782113, + "GC_TIME_MILLIS" : 43413, + "REDUCE_INPUT_GROUPS" : 10, + "SHUFFLED_MAPS" : 670, + "REDUCE_OUTPUT_RECORDS" : 17000878, + "LOCALIZED_BYTES_CACHED" : 72817032907, + "MAP_OUTPUT_RECORDS" : 17000878, + "COMBINE_INPUT_RECORDS" : 0, + "COMMITTED_HEAP_BYTES" : 59466805248 + } + }, + "mapCounters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 77265382690, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 268, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 38651733251, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 38606282909, + "HDFS_WRITE_OPS" : 0, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 0 + }, + "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter" : { + "BYTES_READ" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Read" : 17000878 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 17000878, + "Tuples_Read" : 17000878, + "Read_Duration" : 1314989, + "Write_Duration" : 309234, + "Process_Begin_Time" : 100006209563299, + "Process_End_Time" : 100006211215282 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 8477, + "MAP_OUTPUT_MATERIALIZED_BYTES" : 38583436003, + "MERGED_MAP_OUTPUTS" : 0, + "SPILLED_RECORDS" : 34001756, + "VIRTUAL_MEMORY_BYTES" : 271050731520, + "MAP_INPUT_RECORDS" : 17000878, + "LOCALIZED_NANOS" : 72027213744, + "SPLIT_RAW_BYTES" : 68943, + "LOCALIZED_BYTES_MISSED" : 9991291301, + "FAILED_SHUFFLE" : 0, + "MAP_OUTPUT_BYTES" : 132631170234, + "PHYSICAL_MEMORY_BYTES" : 52774502400, + "GC_TIME_MILLIS" : 27984, + "LOCALIZED_FILES_CACHED" : 67300, + "LOCALIZED_BYTES_CACHED" : 61465705144, + "MAP_OUTPUT_RECORDS" : 17000878, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 3344400, + "COMMITTED_HEAP_BYTES" : 54396940288 + } + }, + "reduceCounters" : { + "org.apache.hadoop.mapreduce.FileSystemCounter" : { + "VIEWFS_LARGE_READ_OPS" : 0, + "FILE_BYTES_WRITTEN" : 64462083793, + "FILE_LARGE_READ_OPS" : 0, + "FILE_WRITE_OPS" : 0, + "VIEWFS_BYTES_READ" : 0, + "VIEWFS_READ_OPS" : 0, + "HDFS_READ_OPS" : 40, + "VIEWFS_WRITE_OPS" : 0, + "HDFS_BYTES_READ" : 0, + "HDFS_LARGE_READ_OPS" : 0, + "FILE_READ_OPS" : 0, + "FILE_BYTES_READ" : 64447954989, + "HDFS_WRITE_OPS" : 40, + "VIEWFS_BYTES_WRITTEN" : 0, + "HDFS_BYTES_WRITTEN" : 30847298707 + }, + "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter" : { + "BYTES_WRITTEN" : 0 + }, + "cascading.flow.StepCounters" : { + "Tuples_Written" : 17000878 + }, + "cascading.flow.SliceCounters" : { + "Tuples_Written" : 17000878, + "Tuples_Read" : 17000878, + "Read_Duration" : 1013676, + "Write_Duration" : 2947305, + "Process_Begin_Time" : 14926301195374, + "Process_End_Time" : 14926305226918 + }, + "Shuffle Errors" : { + "CONNECTION" : 0, + "WRONG_LENGTH" : 0, + "BAD_ID" : 0, + "WRONG_MAP" : 0, + "WRONG_REDUCE" : 0, + "IO_ERROR" : 0 + }, + "org.apache.hadoop.mapreduce.TaskCounter" : { + "LOCALIZED_FILES_MISSED" : 136, + "MERGED_MAP_OUTPUTS" : 670, + "REDUCE_INPUT_RECORDS" : 17000878, + "SPILLED_RECORDS" : 28414760, + "VIRTUAL_MEMORY_BYTES" : 41311039488, + "LOCALIZED_NANOS" : 4898727923, + "LOCALIZED_BYTES_MISSED" : 379284135, + "FAILED_SHUFFLE" : 0, + "REDUCE_SHUFFLE_BYTES" : 38583436003, + "PHYSICAL_MEMORY_BYTES" : 4358541312, + "GC_TIME_MILLIS" : 15429, + "REDUCE_INPUT_GROUPS" : 10, + "COMBINE_OUTPUT_RECORDS" : 0, + "SHUFFLED_MAPS" : 670, + "LOCALIZED_FILES_CACHED" : 11174, + "REDUCE_OUTPUT_RECORDS" : 17000878, + "LOCALIZED_BYTES_CACHED" : 10285939215, + "COMBINE_INPUT_RECORDS" : 0, + "CPU_MILLISECONDS" : 4673240, + "COMMITTED_HEAP_BYTES" : 5069864960 + } + }, + "tasks" : [ ], + "configuration" : { + "cascading.flow.step.num" : "3" + }, + "submitDate" : 1492629953696, + "launchDate" : 1492629971471, + "finishDate" : 1492630639932, + "runTime" : 668461 + } ] +} ] \ No newline at end of file diff --git a/scalding-hraven/src/test/resources/jobResponse_job_1470171371859_6607542.json b/scalding-hraven/src/test/resources/jobResponse_job_1470171371859_6607542.json new file mode 100644 index 0000000000..43e7e28c3d --- /dev/null +++ b/scalding-hraven/src/test/resources/jobResponse_job_1470171371859_6607542.json @@ -0,0 +1,65 @@ +[ { + "taskType" : "", + "counters" : { + } +}, { + "taskType" : "MAP", + "counters" : { + "org.apache.hadoop.mapreduce.TaskCounter" : { + "PHYSICAL_MEMORY_BYTES" : 751378432, + "GC_TIME_MILLIS" : 310, + "CPU_MILLISECONDS" : 38570, + "COMMITTED_HEAP_BYTES" : 810524672 + } + } +}, { + "taskType" : "MAP", + "counters" : { + "org.apache.hadoop.mapreduce.TaskCounter" : { + "PHYSICAL_MEMORY_BYTES" : 751378432, + "GC_TIME_MILLIS" : 310, + "CPU_MILLISECONDS" : 38570, + "COMMITTED_HEAP_BYTES" : 810524672 + } + } +}, { + "taskType" : "MAP", + "counters" : { + "org.apache.hadoop.mapreduce.TaskCounter" : { + "PHYSICAL_MEMORY_BYTES" : 759648256, + "GC_TIME_MILLIS" : 313, + "CPU_MILLISECONDS" : 38620, + "COMMITTED_HEAP_BYTES" : 810520576 + } + } +}, { + "taskType" : "REDUCE", + "counters" : { + "org.apache.hadoop.mapreduce.TaskCounter" : { + "PHYSICAL_MEMORY_BYTES" : 449499136, + "GC_TIME_MILLIS" : 444, + "CPU_MILLISECONDS" : 53720, + "COMMITTED_HEAP_BYTES" : 506986496 + } + } +}, { + "taskType" : "REDUCE", + "counters" : { + "org.apache.hadoop.mapreduce.TaskCounter" : { + "PHYSICAL_MEMORY_BYTES" : 449499136, + "GC_TIME_MILLIS" : 444, + "CPU_MILLISECONDS" : 53720, + "COMMITTED_HEAP_BYTES" : 506986496 + } + } +}, { + "taskType" : "REDUCE", + "counters" : { + "org.apache.hadoop.mapreduce.TaskCounter" : { + "PHYSICAL_MEMORY_BYTES" : 465207296, + "GC_TIME_MILLIS" : 529, + "CPU_MILLISECONDS" : 57210, + "COMMITTED_HEAP_BYTES" : 506986496 + } + } +} ] diff --git a/scalding-hraven/src/test/resources/jobResponse_job_1470171371859_6608570.json b/scalding-hraven/src/test/resources/jobResponse_job_1470171371859_6608570.json new file mode 100644 index 0000000000..f320d64c08 --- /dev/null +++ b/scalding-hraven/src/test/resources/jobResponse_job_1470171371859_6608570.json @@ -0,0 +1,65 @@ +[ { + "taskType" : "", + "counters" : { + } +}, { + "taskType" : "MAP", + "counters" : { + "org.apache.hadoop.mapreduce.TaskCounter" : { + "PHYSICAL_MEMORY_BYTES" : 768618496, + "GC_TIME_MILLIS" : 371, + "CPU_MILLISECONDS" : 45260, + "COMMITTED_HEAP_BYTES" : 814776320 + } + } +}, { + "taskType" : "MAP", + "counters" : { + "org.apache.hadoop.mapreduce.TaskCounter" : { + "PHYSICAL_MEMORY_BYTES" : 768618496, + "GC_TIME_MILLIS" : 371, + "CPU_MILLISECONDS" : 45260, + "COMMITTED_HEAP_BYTES" : 814776320 + } + } +}, { + "taskType" : "MAP", + "counters" : { + "org.apache.hadoop.mapreduce.TaskCounter" : { + "PHYSICAL_MEMORY_BYTES" : 758517760, + "GC_TIME_MILLIS" : 355, + "CPU_MILLISECONDS" : 43950, + "COMMITTED_HEAP_BYTES" : 814280704 + } + } +}, { + "taskType" : "REDUCE", + "counters" : { + "org.apache.hadoop.mapreduce.TaskCounter" : { + "PHYSICAL_MEMORY_BYTES" : 433074176, + "GC_TIME_MILLIS" : 671, + "CPU_MILLISECONDS" : 74270, + "COMMITTED_HEAP_BYTES" : 506986496 + } + } +}, { + "taskType" : "REDUCE", + "counters" : { + "org.apache.hadoop.mapreduce.TaskCounter" : { + "PHYSICAL_MEMORY_BYTES" : 421924864, + "GC_TIME_MILLIS" : 596, + "CPU_MILLISECONDS" : 64390, + "COMMITTED_HEAP_BYTES" : 506986496 + } + } +}, { + "taskType" : "REDUCE", + "counters" : { + "org.apache.hadoop.mapreduce.TaskCounter" : { + "PHYSICAL_MEMORY_BYTES" : 421924864, + "GC_TIME_MILLIS" : 596, + "CPU_MILLISECONDS" : 64390, + "COMMITTED_HEAP_BYTES" : 506986496 + } + } +} ] diff --git a/scalding-hraven/src/test/resources/jobResponse_job_1470171371859_6609558.json b/scalding-hraven/src/test/resources/jobResponse_job_1470171371859_6609558.json new file mode 100644 index 0000000000..de75bf940b --- /dev/null +++ b/scalding-hraven/src/test/resources/jobResponse_job_1470171371859_6609558.json @@ -0,0 +1,75 @@ +[ { + "taskType" : "", + "counters" : { + } +}, { + "taskType" : "MAP", + "counters" : { + "org.apache.hadoop.mapreduce.TaskCounter" : { + "PHYSICAL_MEMORY_BYTES" : 759197696, + "GC_TIME_MILLIS" : 458, + "CPU_MILLISECONDS" : 74500, + "COMMITTED_HEAP_BYTES" : 812834816 + } + } +}, { + "taskType" : "MAP", + "counters" : { + "org.apache.hadoop.mapreduce.TaskCounter" : { + "PHYSICAL_MEMORY_BYTES" : 759197696, + "GC_TIME_MILLIS" : 458, + "CPU_MILLISECONDS" : 74500, + "COMMITTED_HEAP_BYTES" : 812834816 + } + } +}, { + "taskType" : "MAP", + "counters" : { + "org.apache.hadoop.mapreduce.TaskCounter" : { + "PHYSICAL_MEMORY_BYTES" : 760983552, + "GC_TIME_MILLIS" : 370, + "CPU_MILLISECONDS" : 51290, + "COMMITTED_HEAP_BYTES" : 811675648 + } + } +}, { + "taskType" : "REDUCE", + "counters" : { + "org.apache.hadoop.mapreduce.TaskCounter" : { + "PHYSICAL_MEMORY_BYTES" : 457273344, + "GC_TIME_MILLIS" : 457, + "CPU_MILLISECONDS" : 51310, + "COMMITTED_HEAP_BYTES" : 506986496 + } + } +}, { + "taskType" : "REDUCE", + "counters" : { + "org.apache.hadoop.mapreduce.TaskCounter" : { + "PHYSICAL_MEMORY_BYTES" : 444268544, + "GC_TIME_MILLIS" : 490, + "CPU_MILLISECONDS" : 54550, + "COMMITTED_HEAP_BYTES" : 506986496 + } + } +}, { + "taskType" : "REDUCE", + "counters" : { + "org.apache.hadoop.mapreduce.TaskCounter" : { + "PHYSICAL_MEMORY_BYTES" : 444268544, + "GC_TIME_MILLIS" : 490, + "CPU_MILLISECONDS" : 54550, + "COMMITTED_HEAP_BYTES" : 506986496 + } + } +}, { + "taskType" : "REDUCE", + "counters" : { + "org.apache.hadoop.mapreduce.TaskCounter" : { + "PHYSICAL_MEMORY_BYTES" : 442941440, + "GC_TIME_MILLIS" : 520, + "CPU_MILLISECONDS" : 57050, + "COMMITTED_HEAP_BYTES" : 506986496 + } + } +} ] diff --git a/scalding-hraven/src/test/scala/com/twitter/scalding/hraven/estimation/HRavenHistoryServiceTest.scala b/scalding-hraven/src/test/scala/com/twitter/scalding/hraven/estimation/HRavenHistoryServiceTest.scala new file mode 100644 index 0000000000..734aa755e7 --- /dev/null +++ b/scalding-hraven/src/test/scala/com/twitter/scalding/hraven/estimation/HRavenHistoryServiceTest.scala @@ -0,0 +1,153 @@ +package com.twitter.scalding.hraven.estimation + +import cascading.flow.FlowStep +import com.twitter.hraven.JobDescFactory.RESOURCE_MANAGER_KEY +import com.twitter.hraven.rest.client.HRavenRestClient +import com.twitter.hraven.util.JSONUtil +import com.twitter.hraven.{ Flow, TaskDetails } +import com.twitter.scalding.estimation.FlowStrategyInfo +import com.twitter.scalding.hraven.estimation.memory.HRavenMemoryHistoryService +import com.twitter.scalding.hraven.reducer_estimation.HRavenReducerHistoryService +import java.util +import org.apache.hadoop.mapred.JobConf +import org.codehaus.jackson.`type`.TypeReference +import org.mockito.Matchers._ +import org.mockito.Mockito._ +import org.scalatest.{ Matchers, WordSpec } +import scala.collection.JavaConverters._ +import scala.util.Try + +class HRavenHistoryServiceTest extends WordSpec with Matchers { + "A HRaven history service" should { + "work as HRaven memory history service" in { + val tasks = List(7, 6, 6) + + val historyService = new HRavenMemoryHistoryService { + override def hRavenClient(conf: JobConf): Try[HRavenRestClient] = + HRavenMockedClient(super.hRavenClient(conf), detailFields, counterFields) + } + + val history = historyService.fetchHistory( + TestFlowStrategyInfo.dummy(), + HRavenMockedClient.nFetch) + + if (history.isFailure) { + history.get + } else { + history.foreach(_.foreach { step => + tasks should contain(step.tasks.size) + + step.tasks.foreach { task => + assert(task.details.nonEmpty) + assert(task.counters.nonEmpty) + } + }) + } + } + + "work as HRaven reducer history service" in { + val tasks = List(7, 6, 6) + + val historyService = new HRavenReducerHistoryService { + override def hRavenClient(conf: JobConf): Try[HRavenRestClient] = + HRavenMockedClient(super.hRavenClient(conf), detailFields, counterFields) + } + + val history = historyService.fetchHistory( + TestFlowStrategyInfo.dummy(), + HRavenMockedClient.nFetch) + + if (history.isFailure) { + history.get + } else { + history.foreach(_.foreach { step => + tasks should contain(step.tasks.size) + + step.tasks.foreach { task => + assert(task.details.nonEmpty) + assert(task.counters.isEmpty) + } + }) + } + } + } +} + +object TestFlowStrategyInfo { + def dummy(stepNum: Int = 1): FlowStrategyInfo = { + val mockedConf = spy(new JobConf()) + + HRavenMockedClient.configure(mockedConf) + + val mockedStep = mock(classOf[FlowStep[JobConf]]) + val mockedInfo = mock(classOf[FlowStrategyInfo]) + + when(mockedStep.getConfig).thenReturn(mockedConf) + when(mockedStep.getStepNum).thenReturn(stepNum) + when(mockedInfo.step).thenReturn(mockedStep) + + mockedInfo + } +} + +object HRavenMockedClient { + val cluster = "test@cluster" + val user = "testuser" + val batch = "somegoodjob" + val signature = "02CFBD0A94AD5E297C2E4D6665B3B6F0" + val nFetch = 3 + + val jobs = List("job_1470171371859_6609558", "job_1470171371859_6608570", "job_1470171371859_6607542") + + val RequiredJobConfigs = Seq("cascading.flow.step.num") + + def apply( + hRaven: Try[HRavenRestClient], + detailFields: List[String], + counterFields: List[String]): Try[HRavenRestClient] = { + hRaven.map { hRaven => + val client = spy(hRaven) + + doReturn(HRavenMockedClient.cluster) + .when(client) + .getCluster(anyString()) + + doReturn(flowsResponse) + .when(client) + .fetchFlowsWithConfig(anyString(), anyString(), anyString(), anyString(), anyInt(), anyVararg()) + + for (jobId <- jobs) { + val response = jobResponse(jobId) + + doReturn(response) + .when(client) + .fetchTaskDetails(cluster, jobId, detailFields.asJava, counterFields.asJava) + + doReturn(response) + .when(client) + .fetchTaskDetails(cluster, jobId, detailFields.asJava) + } + + client + } + } + + def configure(conf: JobConf): Unit = { + conf.set(HRavenClient.apiHostnameKey, "test") + conf.set(RESOURCE_MANAGER_KEY, "test.com:5053") + conf.set("hraven.history.user.name", HRavenMockedClient.user) + conf.set("batch.desc", HRavenMockedClient.batch) + conf.set("scalding.flow.class.signature", HRavenMockedClient.signature) + conf.set("hraven.estimator.max.flow.histor", HRavenMockedClient.nFetch.toString) + } + + def flowsResponse: util.List[Flow] = + JSONUtil.readJson( + getClass.getResourceAsStream("../../../../../flowResponse.json"), + new TypeReference[util.List[Flow]] {}).asInstanceOf[util.List[Flow]] + + def jobResponse(jobId: String): util.List[TaskDetails] = + JSONUtil.readJson( + getClass.getResourceAsStream(s"../../../../../jobResponse_$jobId.json"), + new TypeReference[util.List[TaskDetails]] {}).asInstanceOf[util.List[TaskDetails]] +} From 65bd39167483553a25641a2d50b5015228ebff7e Mon Sep 17 00:00:00 2001 From: Piyush Narang Date: Fri, 23 Jun 2017 16:07:20 -0700 Subject: [PATCH 3/3] Break out estimator tests into separate module (#1701) --- .travis.yml | 8 ++++++++ build.sbt | 18 ++++++++++++++++++ .../src/test/resources/hipster.txt | 0 .../src/test/resources/scores.tsv | 0 .../memory/MemoryEstimatorTest.scala | 0 .../RatioBasedEstimatorTest.scala | 0 .../ReducerEstimatorTest.scala | 0 .../RuntimeReducerEstimatorTest.scala | 0 8 files changed, 26 insertions(+) rename {scalding-hadoop-test => scalding-estimators-test}/src/test/resources/hipster.txt (100%) rename {scalding-hadoop-test => scalding-estimators-test}/src/test/resources/scores.tsv (100%) rename {scalding-hadoop-test => scalding-estimators-test}/src/test/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorTest.scala (100%) rename {scalding-hadoop-test => scalding-estimators-test}/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala (100%) rename {scalding-hadoop-test => scalding-estimators-test}/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala (100%) rename {scalding-hadoop-test => scalding-estimators-test}/src/test/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala (100%) diff --git a/.travis.yml b/.travis.yml index 7342e19a7a..abbcf7ad4a 100644 --- a/.travis.yml +++ b/.travis.yml @@ -59,6 +59,14 @@ matrix: env: BUILD="base" TEST_TARGET="scalding-hadoop-test" script: "scripts/run_test.sh" + - scala: 2.11.8 + env: BUILD="base" TEST_TARGET="scalding-estimators-test" + script: "scripts/run_test.sh" + + - scala: 2.12.1 + env: BUILD="base" TEST_TARGET="scalding-estimators-test" + script: "scripts/run_test.sh" + - scala: 2.11.8 env: BUILD="base" TEST_TARGET="scalding-serialization" script: "scripts/run_test.sh" diff --git a/build.sbt b/build.sbt index 8c0272d059..cb0a4d875b 100644 --- a/build.sbt +++ b/build.sbt @@ -223,6 +223,7 @@ lazy val scalding = Project( scaldingJson, scaldingJdbc, scaldingHadoopTest, + scaldingEstimatorsTest, scaldingDb, maple, executionTutorial, @@ -526,6 +527,23 @@ lazy val scaldingHadoopTest = module("hadoop-test").settings( ) ).dependsOn(scaldingCore, scaldingSerialization) +lazy val scaldingEstimatorsTest = module("estimators-test").settings( + libraryDependencies ++= Seq( + "org.apache.hadoop" % "hadoop-client" % hadoopVersion, + "org.apache.hadoop" % "hadoop-minicluster" % hadoopVersion, + "org.apache.hadoop" % "hadoop-yarn-server-tests" % hadoopVersion classifier "tests", + "org.apache.hadoop" % "hadoop-yarn-server" % hadoopVersion, + "org.apache.hadoop" % "hadoop-hdfs" % hadoopVersion classifier "tests", + "org.apache.hadoop" % "hadoop-common" % hadoopVersion classifier "tests", + "org.apache.hadoop" % "hadoop-mapreduce-client-jobclient" % hadoopVersion classifier "tests", + "com.twitter" %% "chill-algebird" % chillVersion, + "org.slf4j" % "slf4j-api" % slf4jVersion, + "org.slf4j" % "slf4j-log4j12" % slf4jVersion, + "org.scalacheck" %% "scalacheck" % scalaCheckVersion, + "org.scalatest" %% "scalatest" % scalaTestVersion + ) +).dependsOn(scaldingHadoopTest % "test") + // This one uses a different naming convention lazy val maple = Project( id = "maple", diff --git a/scalding-hadoop-test/src/test/resources/hipster.txt b/scalding-estimators-test/src/test/resources/hipster.txt similarity index 100% rename from scalding-hadoop-test/src/test/resources/hipster.txt rename to scalding-estimators-test/src/test/resources/hipster.txt diff --git a/scalding-hadoop-test/src/test/resources/scores.tsv b/scalding-estimators-test/src/test/resources/scores.tsv similarity index 100% rename from scalding-hadoop-test/src/test/resources/scores.tsv rename to scalding-estimators-test/src/test/resources/scores.tsv diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorTest.scala b/scalding-estimators-test/src/test/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorTest.scala similarity index 100% rename from scalding-hadoop-test/src/test/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorTest.scala rename to scalding-estimators-test/src/test/scala/com/twitter/scalding/estimation/memory/MemoryEstimatorTest.scala diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala b/scalding-estimators-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala similarity index 100% rename from scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala rename to scalding-estimators-test/src/test/scala/com/twitter/scalding/reducer_estimation/RatioBasedEstimatorTest.scala diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala b/scalding-estimators-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala similarity index 100% rename from scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala rename to scalding-estimators-test/src/test/scala/com/twitter/scalding/reducer_estimation/ReducerEstimatorTest.scala diff --git a/scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala b/scalding-estimators-test/src/test/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala similarity index 100% rename from scalding-hadoop-test/src/test/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala rename to scalding-estimators-test/src/test/scala/com/twitter/scalding/reducer_estimation/RuntimeReducerEstimatorTest.scala