Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

Use JobConf input files list for input size computation used by ReducersEstimators #1644

Merged
merged 1 commit into from
Feb 23, 2017
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -4,8 +4,8 @@ import cascading.flow.{ FlowStep, Flow, FlowStepStrategy }
import com.twitter.algebird.Monoid
import com.twitter.scalding.{ StringUtility, Config }
import cascading.tap.{ Tap, CompositeTap }
import cascading.tap.hadoop.Hfs
import org.apache.hadoop.mapred.JobConf
import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapred.{FileInputFormat, JobConf}
import org.slf4j.LoggerFactory
import java.util.{ List => JList }

Expand Down Expand Up @@ -54,23 +54,12 @@ object Common {
def unrollTaps(step: FlowStep[JobConf]): Seq[Tap[_, _, _]] =
unrollTaps(step.getSources.asScala.toSeq)

/**
* Get the total size of the file(s) specified by the Hfs, which may contain a glob
* pattern in its path, so we must be ready to handle that case.
*/
def size(f: Hfs, conf: JobConf): Long = {
val fs = f.getPath.getFileSystem(conf)
fs.globStatus(f.getPath)
.map{ s => fs.getContentSummary(s.getPath).getLength }
.sum
}

def inputSizes(step: FlowStep[JobConf]): Seq[(String, Long)] = {
def inputSizes(step: FlowStep[JobConf]): Seq[(Path, Long)] = {
val conf = step.getConfig
unrollTaps(step).flatMap {
case tap: Hfs => Some(tap.toString -> size(tap, conf))
case _ => None
}

FileInputFormat
.getInputPaths(conf)
.map { path => path -> path.getFileSystem(conf).getContentSummary(path).getLength }
}

def totalInputSize(step: FlowStep[JobConf]): Long = inputSizes(step).map(_._2).sum
Expand Down