-
Couldn't load subscription status.
- Fork 706
Beam backend: use TypedPipe descriptions as names for PTransforms #1983
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
base: develop
Are you sure you want to change the base?
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||
|---|---|---|---|---|
| @@ -1,20 +1,33 @@ | ||||
| package com.twitter.scalding.beam_backend | ||||
|
|
||||
| import com.twitter.scalding.dagon.{FunctionK, Memoize, Rule} | ||||
| import com.twitter.chill.KryoInstantiator | ||||
| import com.twitter.chill.config.ScalaMapConfig | ||||
| import com.twitter.scalding.Config | ||||
| import com.twitter.scalding.beam_backend.BeamOp.{CoGroupedOp, MergedBeamOp} | ||||
| import com.twitter.scalding.beam_backend.BeamOp.CoGroupedOp | ||||
| import com.twitter.scalding.beam_backend.BeamOp.MergedBeamOp | ||||
| import com.twitter.scalding.dagon.FunctionK | ||||
| import com.twitter.scalding.dagon.Memoize | ||||
| import com.twitter.scalding.dagon.Rule | ||||
| import com.twitter.scalding.serialization.KryoHadoop | ||||
| import com.twitter.scalding.typed.CoGrouped.WithDescription | ||||
| import com.twitter.scalding.typed.OptimizationRules.AddExplicitForks | ||||
| import com.twitter.scalding.typed.OptimizationRules.ComposeDescriptions | ||||
| import com.twitter.scalding.typed.OptimizationRules.DeferMerge | ||||
| import com.twitter.scalding.typed.OptimizationRules.DiamondToFlatMap | ||||
| import com.twitter.scalding.typed.OptimizationRules.FilterKeysEarly | ||||
| import com.twitter.scalding.typed.OptimizationRules.IgnoreNoOpGroup | ||||
| import com.twitter.scalding.typed.OptimizationRules.MapValuesInReducers | ||||
| import com.twitter.scalding.typed.OptimizationRules.RemoveDuplicateForceFork | ||||
| import com.twitter.scalding.typed.OptimizationRules.RemoveUselessFork | ||||
| import com.twitter.scalding.typed.OptimizationRules.composeIntoFlatMap | ||||
| import com.twitter.scalding.typed.OptimizationRules.composeSame | ||||
| import com.twitter.scalding.typed.OptimizationRules.simplifyEmpty | ||||
| import com.twitter.scalding.typed._ | ||||
| import com.twitter.scalding.typed.functions.{ | ||||
| FilterKeysToFilter, | ||||
| FlatMapValuesToFlatMap, | ||||
| MapValuesToMap, | ||||
| ScaldingPriorityQueueMonoid | ||||
| } | ||||
|
|
||||
| import com.twitter.scalding.typed.cascading_backend.CascadingExtensions.ConfigCascadingExtensions | ||||
navinvishy marked this conversation as resolved.
Show resolved
Hide resolved
|
||||
| import com.twitter.scalding.typed.functions.FilterKeysToFilter | ||||
| import com.twitter.scalding.typed.functions.FlatMapValuesToFlatMap | ||||
| import com.twitter.scalding.typed.functions.MapValuesToMap | ||||
| import com.twitter.scalding.typed.functions.ScaldingPriorityQueueMonoid | ||||
|
|
||||
| object BeamPlanner { | ||||
| def plan( | ||||
|
|
@@ -61,8 +74,10 @@ object BeamPlanner { | |||
| BeamOp.Source(config, src, srcs(src)) | ||||
| case (IterablePipe(iterable), _) => | ||||
| BeamOp.FromIterable(iterable, kryoCoder) | ||||
| case (wd: WithDescriptionTypedPipe[a], rec) => | ||||
| rec[a](wd.input) | ||||
| case (wd: WithDescriptionTypedPipe[_], rec) => { | ||||
| val op = rec(wd.input) | ||||
| op.withName(wd.descriptions.map(_._1).head) | ||||
|
||||
| } | ||||
| case (SumByLocalKeys(pipe, sg), rec) => | ||||
| val op = rec(pipe) | ||||
| config.getMapSideAggregationThreshold match { | ||||
|
|
@@ -97,7 +112,10 @@ object BeamPlanner { | |||
| uir.evidence.subst[BeamOpT](sortedOp) | ||||
| } | ||||
| go(ivsr) | ||||
| case (ReduceStepPipe(ValueSortedReduce(keyOrdering, pipe, valueSort, reduceFn, _, _)), rec) => | ||||
| case ( | ||||
| ReduceStepPipe(ValueSortedReduce(keyOrdering, pipe, valueSort, reduceFn, _, _)), | ||||
| rec | ||||
| ) => | ||||
| val op = rec(pipe) | ||||
| op.sortedMapGroup(reduceFn)(keyOrdering, valueSort, kryoCoder) | ||||
| case (ReduceStepPipe(IteratorMappedReduce(keyOrdering, pipe, reduceFn, _, _)), rec) => | ||||
|
|
@@ -116,7 +134,7 @@ object BeamPlanner { | |||
| val ops: Seq[BeamOp[(K, Any)]] = cg.inputs.map(tp => rec(tp)) | ||||
| CoGroupedOp(cg, ops) | ||||
| } | ||||
| go(cg) | ||||
| if (cg.descriptions.isEmpty) go(cg) else go(cg).withName(cg.descriptions.last) | ||||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why using There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I meant to ask you about this myself. If you look at the test case in this PR, The cogrouped expression has two descriptions "Count words" and "Join with t1", both of which appear in the descriptions of There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why not combine all of them? Why not just There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, let me try this out on a job first. Thanks! |
||||
| case (Fork(input), rec) => | ||||
| rec(input) | ||||
| case (m @ MergedTypedPipe(_, _), rec) => | ||||
|
|
@@ -137,7 +155,21 @@ object BeamPlanner { | |||
|
|
||||
| def defaultOptimizationRules(config: Config): Seq[Rule[TypedPipe]] = { | ||||
| def std(forceHash: Rule[TypedPipe]) = | ||||
| OptimizationRules.standardMapReduceRules ::: | ||||
| List( | ||||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. why the change here? Is this copying the same cascading optimizations? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It really is the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. sorry I missed that. Note: the line numbers are already captured by the time that rule runs. The line numbers are collected on the If I were you, I would look at all the descriptions and add the full list. The problem with removing that rule is that it will block merging nodes together. It may be fine, maybe Beam will follow up with optimizations, but I would be careful: scalding may do some optimizations that beam doesn't. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The line numbers are in the descriptions though, if users don't explicitly add them with a call to There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't really know the answer. So, line numbers are added to descriptions when the user constructs the TypedPipe, see: scalding/scalding-base/src/main/scala/com/twitter/scalding/typed/TypedPipe.scala Line 533 in 6434348
The idea of As to what to do, I don't know. If beam's optimizer is very good, it maybe doesn't matter. Maybe you should try to separate runs with a somewhat complex job and compare? Also, I can imagine a I would probably bias to just combining all the descriptions into a single beam description unless you actually see problems. You can always come back and add that setting. I would personally bias to maximizing the utility of the optimizer. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks. Let me try out some jobs with the rule enabled and showing all the descriptions. |
||||
| // phase 0, add explicit forks to not duplicate pipes on fanout below | ||||
| AddExplicitForks, | ||||
| RemoveUselessFork, | ||||
| // phase 1, compose flatMap/map, move descriptions down, defer merge, filter pushup etc... | ||||
| IgnoreNoOpGroup.orElse(composeSame).orElse(FilterKeysEarly).orElse(DeferMerge), | ||||
| // phase 2, combine different kinds of mapping operations into flatMaps, including redundant merges | ||||
| composeIntoFlatMap | ||||
| .orElse(simplifyEmpty) | ||||
| .orElse(DiamondToFlatMap) | ||||
| .orElse(ComposeDescriptions) | ||||
| .orElse(MapValuesInReducers), | ||||
| // phase 3, remove duplicates forces/forks (e.g. .fork.fork or .forceToDisk.fork, ....) | ||||
| RemoveDuplicateForceFork | ||||
| ) ::: | ||||
| List( | ||||
| OptimizationRules.FilterLocally, // after filtering, we may have filtered to nothing, lets see | ||||
| OptimizationRules.simplifyEmpty, | ||||
|
|
||||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -6,9 +6,10 @@ import com.twitter.scalding.Config | |
| import com.twitter.scalding.beam_backend.BeamFunctions._ | ||
| import com.twitter.scalding.beam_backend.BeamJoiner.MultiJoinFunction | ||
| import com.twitter.scalding.serialization.Externalizer | ||
| import com.twitter.scalding.typed.Input | ||
| import com.twitter.scalding.typed.functions.ComposedFunctions.ComposedMapGroup | ||
| import com.twitter.scalding.typed.functions.{EmptyGuard, MapValueStream, ScaldingPriorityQueueMonoid, SumAll} | ||
| import com.twitter.scalding.typed.{CoGrouped, Input} | ||
| import com.twitter.scalding.typed.{CoGrouped, TypedSource} | ||
|
||
| import java.util.{Comparator, PriorityQueue} | ||
| import org.apache.beam.sdk.Pipeline | ||
| import org.apache.beam.sdk.coders.{Coder, IterableCoder, KvCoder} | ||
|
|
@@ -55,6 +56,8 @@ sealed abstract class BeamOp[+A] { | |
|
|
||
| def flatMap[B](f: A => TraversableOnce[B])(implicit kryoCoder: KryoCoder): BeamOp[B] = | ||
| parDo(FlatMapFn(f), "flatMap") | ||
|
|
||
| def withName(name: String): BeamOp[A] | ||
| } | ||
|
|
||
| private final case class SerializableComparator[T](comp: Comparator[T]) extends Comparator[T] { | ||
|
|
@@ -136,11 +139,16 @@ object BeamOp extends Serializable { | |
| ) | ||
| case Some(src) => src.read(pipeline, conf) | ||
| } | ||
|
|
||
| override def withName(name: String): BeamOp[A] = this | ||
| } | ||
|
|
||
| final case class FromIterable[A](iterable: Iterable[A], kryoCoder: KryoCoder) extends BeamOp[A] { | ||
| final case class FromIterable[A](iterable: Iterable[A], kryoCoder: KryoCoder, name: Option[String] = None) | ||
| extends BeamOp[A] { | ||
| override def runNoCache(pipeline: Pipeline): PCollection[_ <: A] = | ||
| pipeline.apply(Create.of(iterable.asJava).withCoder(kryoCoder)) | ||
| pipeline.apply(name.getOrElse("Iterable source"), Create.of(iterable.asJava).withCoder(kryoCoder)) | ||
|
|
||
| override def withName(name: String): BeamOp[A] = FromIterable(iterable, kryoCoder, Some(name)) | ||
| } | ||
|
|
||
| final case class TransformBeamOp[A, B]( | ||
|
|
@@ -153,6 +161,8 @@ object BeamOp extends Serializable { | |
| val pCollection: PCollection[A] = widenPCollection(source.run(pipeline)) | ||
| pCollection.apply(name, f).setCoder(kryoCoder) | ||
| } | ||
|
|
||
| override def withName(desc: String): BeamOp[B] = TransformBeamOp(source, f, kryoCoder, desc) | ||
| } | ||
|
|
||
| final case class HashJoinTransform[K, V, U, W]( | ||
|
|
@@ -184,7 +194,8 @@ object BeamOp extends Serializable { | |
| final case class HashJoinOp[K, V, U, W]( | ||
| left: BeamOp[(K, V)], | ||
| right: BeamOp[(K, U)], | ||
| joiner: (K, V, Iterable[U]) => Iterator[W] | ||
| joiner: (K, V, Iterable[U]) => Iterator[W], | ||
| name: Option[String] = None | ||
| )(implicit kryoCoder: KryoCoder, ordK: Ordering[K]) | ||
| extends BeamOp[(K, W)] { | ||
| override def runNoCache(pipeline: Pipeline): PCollection[_ <: (K, W)] = { | ||
|
|
@@ -199,20 +210,28 @@ object BeamOp extends Serializable { | |
| widenPCollection(rightPCollection): PCollection[(K, _)] | ||
| ) | ||
|
|
||
| tuple.apply(HashJoinTransform(keyCoder, joiner)) | ||
| tuple.apply(name.getOrElse("HashJoin"), HashJoinTransform(keyCoder, joiner)) | ||
| } | ||
|
|
||
| override def withName(name: String): BeamOp[(K, W)] = HashJoinOp(left, right, joiner, Some(name)) | ||
| } | ||
|
|
||
| final case class MergedBeamOp[A](first: BeamOp[A], second: BeamOp[A], tail: Seq[BeamOp[A]]) | ||
| extends BeamOp[A] { | ||
| final case class MergedBeamOp[A]( | ||
| first: BeamOp[A], | ||
| second: BeamOp[A], | ||
| tail: Seq[BeamOp[A]], | ||
| name: Option[String] = None | ||
| ) extends BeamOp[A] { | ||
| override def runNoCache(pipeline: Pipeline): PCollection[_ <: A] = { | ||
| val collections = PCollectionList | ||
| .of(widenPCollection(first.run(pipeline)): PCollection[A]) | ||
| .and(widenPCollection(second.run(pipeline)): PCollection[A]) | ||
| .and(tail.map(op => widenPCollection(op.run(pipeline)): PCollection[A]).asJava) | ||
|
|
||
| collections.apply(Flatten.pCollections[A]()) | ||
| collections.apply(name.getOrElse("Merge"), Flatten.pCollections[A]()) | ||
| } | ||
|
|
||
| override def withName(name: String): BeamOp[A] = MergedBeamOp(first, second, tail, Some(name)) | ||
| } | ||
|
|
||
| final case class CoGroupedTransform[K, V]( | ||
|
|
@@ -241,7 +260,8 @@ object BeamOp extends Serializable { | |
|
|
||
| final case class CoGroupedOp[K, V]( | ||
| cg: CoGrouped[K, V], | ||
| inputOps: Seq[BeamOp[(K, Any)]] | ||
| inputOps: Seq[BeamOp[(K, Any)]], | ||
| name: Option[String] = None | ||
| )(implicit kryoCoder: KryoCoder) | ||
| extends BeamOp[(K, V)] { | ||
| override def runNoCache(pipeline: Pipeline): PCollection[_ <: (K, V)] = { | ||
|
|
@@ -256,8 +276,10 @@ object BeamOp extends Serializable { | |
|
|
||
| PCollectionList | ||
| .of(pcols.asJava) | ||
| .apply(CoGroupedTransform(joinFunction, tupleTags, keyCoder)) | ||
| .apply(name.getOrElse("CoGrouped"), CoGroupedTransform(joinFunction, tupleTags, keyCoder)) | ||
| } | ||
|
|
||
| override def withName(name: String): BeamOp[(K, V)] = CoGroupedOp(cg, inputOps, Some(name)) | ||
| } | ||
|
|
||
| final case class CoGroupDoFn[K, V]( | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.