From de8c99adf99103bfd4f58b978ee65ec5a75fbc89 Mon Sep 17 00:00:00 2001 From: Andrew Martin Date: Tue, 13 Dec 2016 14:31:43 -0500 Subject: [PATCH 01/11] Add ability to name transforms in all SCollection variants. Don't leak variable transform name to subclasses. Also fix whitespace and make test names consistent. Add TransformNameProvider so CallSites.getCurrent can be deferred to the proper call site. Fix test. --- .../scala/com/spotify/scio/ScioContext.scala | 5 +- .../scio/values/PCollectionWrapper.scala | 4 +- .../com/spotify/scio/values/SCollection.scala | 4 +- .../values/SCollectionWithAccumulator.scala | 2 +- .../scio/values/SCollectionWithFanout.scala | 2 +- .../values/SCollectionWithHotKeyFanout.scala | 8 +- .../values/SCollectionWithSideInput.scala | 2 +- .../values/SCollectionWithSideOutput.scala | 2 +- .../scio/values/TransformNameable.scala | 42 +++++++ .../scio/values/WindowedSCollection.scala | 2 +- .../scio/values/NamedTransformTest.scala | 119 ++++++++++++++++++ 11 files changed, 180 insertions(+), 12 deletions(-) create mode 100644 scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala create mode 100644 scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala diff --git a/scio-core/src/main/scala/com/spotify/scio/ScioContext.scala b/scio-core/src/main/scala/com/spotify/scio/ScioContext.scala index 73334ca8c4..a5aa85eed2 100644 --- a/scio-core/src/main/scala/com/spotify/scio/ScioContext.scala +++ b/scio-core/src/main/scala/com/spotify/scio/ScioContext.scala @@ -138,7 +138,8 @@ object ScioContext { */ // scalastyle:off number.of.methods class ScioContext private[scio] (val options: PipelineOptions, - private var artifacts: List[String]) { + private var artifacts: List[String]) + extends TransformNameable[ScioContext] { private implicit val context: ScioContext = this @@ -398,7 +399,7 @@ class ScioContext private[scio] (val options: PipelineOptions, private[scio] def applyInternal[Output <: POutput](root: PTransform[_ >: PBegin, Output]) : Output = - pipeline.apply(CallSites.getCurrent, root) + pipeline.apply(this.tfName, root) /** * Get an SCollection for an object file. diff --git a/scio-core/src/main/scala/com/spotify/scio/values/PCollectionWrapper.scala b/scio-core/src/main/scala/com/spotify/scio/values/PCollectionWrapper.scala index ad1fc59066..d6bcdd1c75 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/PCollectionWrapper.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/PCollectionWrapper.scala @@ -20,12 +20,12 @@ package com.spotify.scio.values import com.google.cloud.dataflow.sdk.coders.Coder import com.google.cloud.dataflow.sdk.transforms.{Combine, DoFn, PTransform, ParDo} import com.google.cloud.dataflow.sdk.values.{KV, PCollection, POutput} -import com.spotify.scio.util.CallSites import com.spotify.scio.{Implicits, ScioContext} import scala.reflect.ClassTag private[values] trait PCollectionWrapper[T] { + this: TransformNameable[_ <: PCollectionWrapper[T]] => import Implicits._ @@ -39,7 +39,7 @@ private[values] trait PCollectionWrapper[T] { private[scio] def applyInternal[Output <: POutput] (transform: PTransform[_ >: PCollection[T], Output]): Output = - internal.apply(CallSites.getCurrent, transform) + internal.apply(this.tfName, transform) protected def pApply[U: ClassTag] (transform: PTransform[_ >: PCollection[T], PCollection[U]]): SCollection[U] = { diff --git a/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala b/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala index 1fa25ffa97..9ba821d08a 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala @@ -61,7 +61,7 @@ object SCollection { val o = PCollectionList .of(scs.map(_.internal).asJava) .apply(CallSites.getCurrent, Flatten.pCollections()) - new SCollectionImpl(o, scs.head.context) + scs.head.context.wrap(o) } import scala.language.implicitConversions @@ -98,7 +98,7 @@ object SCollection { * @groupname transform Transformations * @groupname window Windowing Operations */ -sealed trait SCollection[T] extends PCollectionWrapper[T] { +sealed trait SCollection[T] extends PCollectionWrapper[T] with TransformNameable[SCollection[T]] { import TupleFunctions._ diff --git a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithAccumulator.scala b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithAccumulator.scala index c3ba299fba..85c382e82b 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithAccumulator.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithAccumulator.scala @@ -29,7 +29,7 @@ import scala.reflect.ClassTag */ class SCollectionWithAccumulator[T: ClassTag] private[values] (val internal: PCollection[T], val context: ScioContext, acc: Seq[Accumulator[_]]) - extends PCollectionWrapper[T] { + extends PCollectionWrapper[T] with TransformNameable[SCollectionWithAccumulator[T]] { protected val ct: ClassTag[T] = implicitly[ClassTag[T]] diff --git a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithFanout.scala b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithFanout.scala index b0a21c0c7d..8927c6f751 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithFanout.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithFanout.scala @@ -32,7 +32,7 @@ import scala.reflect.ClassTag class SCollectionWithFanout[T: ClassTag] private[values] (val internal: PCollection[T], val context: ScioContext, private val fanout: Int) - extends PCollectionWrapper[T] { + extends PCollectionWrapper[T] with TransformNameable[SCollectionWithFanout[T]] { protected val ct: ClassTag[T] = implicitly[ClassTag[T]] diff --git a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithHotKeyFanout.scala b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithHotKeyFanout.scala index feaebde478..0f7db78781 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithHotKeyFanout.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithHotKeyFanout.scala @@ -31,7 +31,8 @@ import scala.reflect.ClassTag */ class SCollectionWithHotKeyFanout[K: ClassTag, V: ClassTag] (private val self: PairSCollectionFunctions[K, V], - private val hotKeyFanout: Either[K => Int, Int]) { + private val hotKeyFanout: Either[K => Int, Int]) + extends TransformNameable[SCollectionWithHotKeyFanout[K, V]] { private def withFanout[K, I, O](combine: Combine.PerKey[K, I, O]) : PerKeyWithHotKeyFanout[K, I, O] = this.hotKeyFanout match { @@ -42,6 +43,11 @@ class SCollectionWithHotKeyFanout[K: ClassTag, V: ClassTag] combine.withHotKeyFanout(f) } + override def withName(name: String): SCollectionWithHotKeyFanout[K, V] = { + self.self.withName(name) + this + } + /** * Aggregate the values of each key, using given combine functions and a neutral "zero value". * This function can return a different result type, U, than the type of the values in this diff --git a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideInput.scala b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideInput.scala index 1df6507464..bce434f35a 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideInput.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideInput.scala @@ -34,7 +34,7 @@ import scala.util.Try class SCollectionWithSideInput[T: ClassTag] private[values] (val internal: PCollection[T], val context: ScioContext, sides: Iterable[SideInput[_]]) - extends PCollectionWrapper[T] { + extends PCollectionWrapper[T] with TransformNameable[SCollectionWithSideInput[T]] { protected val ct: ClassTag[T] = implicitly[ClassTag[T]] diff --git a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideOutput.scala b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideOutput.scala index aa29e348cc..8b228e2109 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideOutput.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideOutput.scala @@ -35,7 +35,7 @@ class SCollectionWithSideOutput[T: ClassTag] private[values] (val internal: PCollection[T], val context: ScioContext, sides: Iterable[SideOutput[_]]) - extends PCollectionWrapper[T] { + extends PCollectionWrapper[T] with TransformNameable[SCollectionWithSideOutput[T]] { protected val ct: ClassTag[T] = implicitly[ClassTag[T]] diff --git a/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala b/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala new file mode 100644 index 0000000000..e2d515e07d --- /dev/null +++ b/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala @@ -0,0 +1,42 @@ +/* + * Copyright 2016 Spotify AB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.spotify.scio.values + +import com.spotify.scio.util.CallSites + +trait TransformNameable[T <: TransformNameable[T]] { + this: T => + private var nameProvider: TransformNameProvider = CallSiteNameProvider + + def tfName: String = nameProvider.name + + def withName(name: String): T = { + nameProvider = new ConstNameProvider(name) + this + } +} + +private trait TransformNameProvider { + def name: String +} + +private object CallSiteNameProvider extends TransformNameProvider { + def name: String = CallSites.getCurrent +} + +private class ConstNameProvider(val name: String) extends TransformNameProvider diff --git a/scio-core/src/main/scala/com/spotify/scio/values/WindowedSCollection.scala b/scio-core/src/main/scala/com/spotify/scio/values/WindowedSCollection.scala index f4311efbdd..46f5f2fe1e 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/WindowedSCollection.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/WindowedSCollection.scala @@ -40,7 +40,7 @@ case class WindowedValue[T](value: T, timestamp: Instant, window: BoundedWindow, class WindowedSCollection[T: ClassTag] private[values] (val internal: PCollection[T], val context: ScioContext) - extends PCollectionWrapper[T] { + extends PCollectionWrapper[T] with TransformNameable[WindowedSCollection[T]] { protected val ct: ClassTag[T] = implicitly[ClassTag[T]] diff --git a/scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala b/scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala new file mode 100644 index 0000000000..e0bd2dd2ca --- /dev/null +++ b/scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala @@ -0,0 +1,119 @@ +/* + * Copyright 2016 Spotify AB. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package com.spotify.scio.values + +import com.spotify.scio.testing.PipelineSpec + +class NamedTransformTest extends PipelineSpec { + + "ScioContext" should "support custom transform name" in { + runWithContext { sc => + val p = sc.withName("ReadInput").parallelize(Seq("a", "b", "c")) + assertTransformName(p, "ReadInput/Read(InMemorySource)") + } + } + + "SCollection" should "support custom transform name" in { + runWithContext { sc => + val p = sc.parallelize(Seq(1, 2, 3, 4, 5)) + .map(_ * 3) + .withName("OnlyEven").filter(_ % 2 == 0) + assertTransformName(p, "OnlyEven/Filter") + } + } + + "DoubleSCollectionFunctions" should "support custom transform name" in { + runWithContext { sc => + val p = sc.parallelize(Seq(1.0, 2.0, 3.0, 4.0, 5.0)) + .withName("CalcVariance").variance + // TODO: give explicit names to internal transforms + //assertTransformName(p, "CalcVariance") + } + } + + "PairSCollectionFunctions" should "support custom transform name" in { + runWithContext { sc => + val p = sc.parallelize(Seq(("a", 1), ("b", 2), ("c", 3))) + .withName("SumPerKey").sumByKey + assertTransformName(p, "SumPerKey/KvToTuple") + } + } + + "SCollectionWithAccumulator" should "support custom transform name" in { + runWithContext { sc => + val intSum = sc.sumAccumulator[Int]("IntSum") + val p = sc.parallelize(Seq(1, 2, 3, 4, 5)) + .withAccumulator(intSum) + .withName("TripleSum").map { (i, c) => + val n = i * 3 + c.addValue(intSum, n) + n + } + assertTransformName(p, "TripleSum") + } + } + + "SCollectionWithFanout" should "support custom transform name" in { + runWithContext { sc => + val p = sc.parallelize(Seq(1, 2, 3)).withFanout(10) + .withName("Sum").sum + assertTransformName(p, "Sum/Values/Values") + } + } + + "SCollectionWithHotKeyFanout" should "support custom transform name" in { + runWithContext { sc => + val p = sc.parallelize(Seq(("a", 1), ("b", 2), ("c", 3))).withHotKeyFanout(10) + .withName("Sum").sumByKey + assertTransformName(p, "Sum/KvToTuple") + } + } + + "SCollectionWithSideInput" should "support custom transform name" in { + runWithContext { sc => + val p1 = sc.parallelize(Seq("a", "b", "c")) + val p2 = sc.parallelize(Seq(1, 2, 3)).asListSideInput + val s = p1.withSideInputs(p2) + .withName("GetX").filter((x, s) => x == "a") + assertTransformName(s, "GetX") + } + } + + "SCollectionWithSideOutput" should "support custom transform name" in { + runWithContext { sc => + val p1 = sc.parallelize(Seq("a", "b", "c")) + val p2 = SideOutput[String]() + val (main, side) = p1.withSideOutputs(p2) + .withName("MakeSideOutput").map { (x, s) => s.output(p2, x + "2"); x + "1" } + assertTransformName(main, "MakeSideOutput") + assertTransformName(side(p2), "MakeSideOutput") + } + } + + "WindowedSCollection" should "support custom transform name" in { + runWithContext { sc => + val p = sc.parallelize(Seq(1, 2, 3, 4, 5)) + .toWindowed + .withName("Triple").map(x => x.withValue(x.value * 3)) + assertTransformName(p, "Triple") + } + } + + private def assertTransformName(p: PCollectionWrapper[_], tfName: String) = + p.internal.getProducingTransformInternal.getFullName shouldBe tfName +} From 6289fd8d4200ae5d9b4b7dfce193caf7e9a2aec6 Mon Sep 17 00:00:00 2001 From: Andrew Martin Date: Tue, 13 Dec 2016 20:11:58 -0500 Subject: [PATCH 02/11] Remove all occurrences of CallSites.getCurrent Also give explicit names to internal transforms in DoubleSCollectionFunctions --- .../spotify/scio/accumulators/package.scala | 6 +- .../com/spotify/scio/util/MultiJoin.scala | 342 +++++++++--------- .../values/DoubleSCollectionFunctions.scala | 12 +- .../com/spotify/scio/values/SCollection.scala | 6 +- .../values/SCollectionWithSideInput.scala | 2 +- .../scio/values/TransformNameable.scala | 6 +- .../scio/values/NamedTransformTest.scala | 27 +- scripts/multijoin.py | 22 +- 8 files changed, 229 insertions(+), 194 deletions(-) diff --git a/scio-core/src/main/scala/com/spotify/scio/accumulators/package.scala b/scio-core/src/main/scala/com/spotify/scio/accumulators/package.scala index b52ae8fecd..f9b7053c41 100644 --- a/scio-core/src/main/scala/com/spotify/scio/accumulators/package.scala +++ b/scio-core/src/main/scala/com/spotify/scio/accumulators/package.scala @@ -60,7 +60,7 @@ package object accumulators { /** Count elements with an automatically created accumulator. */ def accumulateCount: SCollection[T] = - self.accumulateCount(self.context.sumAccumulator[Long](CallSites.getCurrentName)) + self.accumulateCount(self.context.sumAccumulator[Long](self.tfName)) /** Count positive and negative results in a filter with the given accumulators. */ def accumulateCountFilter(accPos: Accumulator[Long], @@ -79,8 +79,8 @@ package object accumulators { /** Count positive and negative results in a filter with automatically created accumulators. */ def accumulateCountFilter(f: T => Boolean): SCollection[T] = { - val accPos = self.context.sumAccumulator[Long]("Positive#" + CallSites.getCurrentName) - val accNeg = self.context.sumAccumulator[Long]("Negative#" + CallSites.getCurrentName) + val accPos = self.context.sumAccumulator[Long]("Positive#" + self.tfName) + val accNeg = self.context.sumAccumulator[Long]("Negative#" + self.tfName) self.accumulateCountFilter(accPos, accNeg)(f) } diff --git a/scio-core/src/main/scala/com/spotify/scio/util/MultiJoin.scala b/scio-core/src/main/scala/com/spotify/scio/util/MultiJoin.scala index 807ac2f15d..f42fe9c24d 100644 --- a/scio-core/src/main/scala/com/spotify/scio/util/MultiJoin.scala +++ b/scio-core/src/main/scala/com/spotify/scio/util/MultiJoin.scala @@ -29,13 +29,17 @@ package com.spotify.scio.util import com.google.cloud.dataflow.sdk.transforms.join.{CoGroupByKey, KeyedPCollectionTuple} import com.google.cloud.dataflow.sdk.values.TupleTag import com.google.common.collect.Lists -import com.spotify.scio.values.SCollection +import com.spotify.scio.values.{CallSiteNameProvider, ConstNameProvider, SCollection, TransformNameProvider} import scala.collection.JavaConverters._ import scala.reflect.ClassTag object MultiJoin { + private var tfNameProvider: TransformNameProvider = CallSiteNameProvider + + def withName(name: String): MultiJoin.type = {tfNameProvider = new ConstNameProvider(name); this} + def toOptions[T](xs: Iterator[T]): Iterator[Option[T]] = if (xs.isEmpty) Iterator(None) else xs.map(Option(_)) def cogroup[KEY: ClassTag, A: ClassTag, B: ClassTag](a: SCollection[(KEY, A)], b: SCollection[(KEY, B)]): SCollection[(KEY, (Iterable[A], Iterable[B]))] = { @@ -43,8 +47,8 @@ object MultiJoin { val keyed = KeyedPCollectionTuple .of(tagA, a.toKV.internal) .and(tagB, b.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala)) } @@ -56,8 +60,8 @@ object MultiJoin { .of(tagA, a.toKV.internal) .and(tagB, b.toKV.internal) .and(tagC, c.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala)) } @@ -70,8 +74,8 @@ object MultiJoin { .and(tagB, b.toKV.internal) .and(tagC, c.toKV.internal) .and(tagD, d.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala)) } @@ -85,8 +89,8 @@ object MultiJoin { .and(tagC, c.toKV.internal) .and(tagD, d.toKV.internal) .and(tagE, e.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala)) } @@ -101,8 +105,8 @@ object MultiJoin { .and(tagD, d.toKV.internal) .and(tagE, e.toKV.internal) .and(tagF, f.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala)) } @@ -118,8 +122,8 @@ object MultiJoin { .and(tagE, e.toKV.internal) .and(tagF, f.toKV.internal) .and(tagG, g.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala)) } @@ -136,8 +140,8 @@ object MultiJoin { .and(tagF, f.toKV.internal) .and(tagG, g.toKV.internal) .and(tagH, h.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala)) } @@ -155,8 +159,8 @@ object MultiJoin { .and(tagG, g.toKV.internal) .and(tagH, h.toKV.internal) .and(tagI, i.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala)) } @@ -175,8 +179,8 @@ object MultiJoin { .and(tagH, h.toKV.internal) .and(tagI, i.toKV.internal) .and(tagJ, j.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala)) } @@ -196,8 +200,8 @@ object MultiJoin { .and(tagI, i.toKV.internal) .and(tagJ, j.toKV.internal) .and(tagK, k.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala)) } @@ -218,8 +222,8 @@ object MultiJoin { .and(tagJ, j.toKV.internal) .and(tagK, k.toKV.internal) .and(tagL, l.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala)) } @@ -241,8 +245,8 @@ object MultiJoin { .and(tagK, k.toKV.internal) .and(tagL, l.toKV.internal) .and(tagM, m.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala)) } @@ -265,8 +269,8 @@ object MultiJoin { .and(tagL, l.toKV.internal) .and(tagM, m.toKV.internal) .and(tagN, n.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala, result.getAll(tagN).asScala)) } @@ -290,8 +294,8 @@ object MultiJoin { .and(tagM, m.toKV.internal) .and(tagN, n.toKV.internal) .and(tagO, o.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala, result.getAll(tagN).asScala, result.getAll(tagO).asScala)) } @@ -316,8 +320,8 @@ object MultiJoin { .and(tagN, n.toKV.internal) .and(tagO, o.toKV.internal) .and(tagP, p.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala, result.getAll(tagN).asScala, result.getAll(tagO).asScala, result.getAll(tagP).asScala)) } @@ -343,8 +347,8 @@ object MultiJoin { .and(tagO, o.toKV.internal) .and(tagP, p.toKV.internal) .and(tagQ, q.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala, result.getAll(tagN).asScala, result.getAll(tagO).asScala, result.getAll(tagP).asScala, result.getAll(tagQ).asScala)) } @@ -371,8 +375,8 @@ object MultiJoin { .and(tagP, p.toKV.internal) .and(tagQ, q.toKV.internal) .and(tagR, r.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala, result.getAll(tagN).asScala, result.getAll(tagO).asScala, result.getAll(tagP).asScala, result.getAll(tagQ).asScala, result.getAll(tagR).asScala)) } @@ -400,8 +404,8 @@ object MultiJoin { .and(tagQ, q.toKV.internal) .and(tagR, r.toKV.internal) .and(tagS, s.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala, result.getAll(tagN).asScala, result.getAll(tagO).asScala, result.getAll(tagP).asScala, result.getAll(tagQ).asScala, result.getAll(tagR).asScala, result.getAll(tagS).asScala)) } @@ -430,8 +434,8 @@ object MultiJoin { .and(tagR, r.toKV.internal) .and(tagS, s.toKV.internal) .and(tagT, t.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala, result.getAll(tagN).asScala, result.getAll(tagO).asScala, result.getAll(tagP).asScala, result.getAll(tagQ).asScala, result.getAll(tagR).asScala, result.getAll(tagS).asScala, result.getAll(tagT).asScala)) } @@ -461,8 +465,8 @@ object MultiJoin { .and(tagS, s.toKV.internal) .and(tagT, t.toKV.internal) .and(tagU, u.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala, result.getAll(tagN).asScala, result.getAll(tagO).asScala, result.getAll(tagP).asScala, result.getAll(tagQ).asScala, result.getAll(tagR).asScala, result.getAll(tagS).asScala, result.getAll(tagT).asScala, result.getAll(tagU).asScala)) } @@ -493,8 +497,8 @@ object MultiJoin { .and(tagT, t.toKV.internal) .and(tagU, u.toKV.internal) .and(tagV, v.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).map { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala, result.getAll(tagN).asScala, result.getAll(tagO).asScala, result.getAll(tagP).asScala, result.getAll(tagQ).asScala, result.getAll(tagR).asScala, result.getAll(tagS).asScala, result.getAll(tagT).asScala, result.getAll(tagU).asScala, result.getAll(tagV).asScala)) } @@ -505,8 +509,8 @@ object MultiJoin { val keyed = KeyedPCollectionTuple .of(tagA, a.toKV.internal) .and(tagB, b.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { b <- result.getAll(tagB).asScala.iterator @@ -521,8 +525,8 @@ object MultiJoin { .of(tagA, a.toKV.internal) .and(tagB, b.toKV.internal) .and(tagC, c.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { c <- result.getAll(tagC).asScala.iterator @@ -539,8 +543,8 @@ object MultiJoin { .and(tagB, b.toKV.internal) .and(tagC, c.toKV.internal) .and(tagD, d.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { d <- result.getAll(tagD).asScala.iterator @@ -559,8 +563,8 @@ object MultiJoin { .and(tagC, c.toKV.internal) .and(tagD, d.toKV.internal) .and(tagE, e.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { e <- result.getAll(tagE).asScala.iterator @@ -581,8 +585,8 @@ object MultiJoin { .and(tagD, d.toKV.internal) .and(tagE, e.toKV.internal) .and(tagF, f.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { f <- result.getAll(tagF).asScala.iterator @@ -605,8 +609,8 @@ object MultiJoin { .and(tagE, e.toKV.internal) .and(tagF, f.toKV.internal) .and(tagG, g.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { g <- result.getAll(tagG).asScala.iterator @@ -631,8 +635,8 @@ object MultiJoin { .and(tagF, f.toKV.internal) .and(tagG, g.toKV.internal) .and(tagH, h.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { h <- result.getAll(tagH).asScala.iterator @@ -659,8 +663,8 @@ object MultiJoin { .and(tagG, g.toKV.internal) .and(tagH, h.toKV.internal) .and(tagI, i.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { i <- result.getAll(tagI).asScala.iterator @@ -689,8 +693,8 @@ object MultiJoin { .and(tagH, h.toKV.internal) .and(tagI, i.toKV.internal) .and(tagJ, j.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { j <- result.getAll(tagJ).asScala.iterator @@ -721,8 +725,8 @@ object MultiJoin { .and(tagI, i.toKV.internal) .and(tagJ, j.toKV.internal) .and(tagK, k.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { k <- result.getAll(tagK).asScala.iterator @@ -755,8 +759,8 @@ object MultiJoin { .and(tagJ, j.toKV.internal) .and(tagK, k.toKV.internal) .and(tagL, l.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { l <- result.getAll(tagL).asScala.iterator @@ -791,8 +795,8 @@ object MultiJoin { .and(tagK, k.toKV.internal) .and(tagL, l.toKV.internal) .and(tagM, m.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { m <- result.getAll(tagM).asScala.iterator @@ -829,8 +833,8 @@ object MultiJoin { .and(tagL, l.toKV.internal) .and(tagM, m.toKV.internal) .and(tagN, n.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { n <- result.getAll(tagN).asScala.iterator @@ -869,8 +873,8 @@ object MultiJoin { .and(tagM, m.toKV.internal) .and(tagN, n.toKV.internal) .and(tagO, o.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { o <- result.getAll(tagO).asScala.iterator @@ -911,8 +915,8 @@ object MultiJoin { .and(tagN, n.toKV.internal) .and(tagO, o.toKV.internal) .and(tagP, p.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { p <- result.getAll(tagP).asScala.iterator @@ -955,8 +959,8 @@ object MultiJoin { .and(tagO, o.toKV.internal) .and(tagP, p.toKV.internal) .and(tagQ, q.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { q <- result.getAll(tagQ).asScala.iterator @@ -1001,8 +1005,8 @@ object MultiJoin { .and(tagP, p.toKV.internal) .and(tagQ, q.toKV.internal) .and(tagR, r.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { r <- result.getAll(tagR).asScala.iterator @@ -1049,8 +1053,8 @@ object MultiJoin { .and(tagQ, q.toKV.internal) .and(tagR, r.toKV.internal) .and(tagS, s.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { s <- result.getAll(tagS).asScala.iterator @@ -1099,8 +1103,8 @@ object MultiJoin { .and(tagR, r.toKV.internal) .and(tagS, s.toKV.internal) .and(tagT, t.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { t <- result.getAll(tagT).asScala.iterator @@ -1151,8 +1155,8 @@ object MultiJoin { .and(tagS, s.toKV.internal) .and(tagT, t.toKV.internal) .and(tagU, u.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { u <- result.getAll(tagU).asScala.iterator @@ -1205,8 +1209,8 @@ object MultiJoin { .and(tagT, t.toKV.internal) .and(tagU, u.toKV.internal) .and(tagV, v.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { v <- result.getAll(tagV).asScala.iterator @@ -1240,8 +1244,8 @@ object MultiJoin { val keyed = KeyedPCollectionTuple .of(tagA, a.toKV.internal) .and(tagB, b.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { b <- toOptions(result.getAll(tagB).asScala.iterator) @@ -1256,8 +1260,8 @@ object MultiJoin { .of(tagA, a.toKV.internal) .and(tagB, b.toKV.internal) .and(tagC, c.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { c <- toOptions(result.getAll(tagC).asScala.iterator) @@ -1274,8 +1278,8 @@ object MultiJoin { .and(tagB, b.toKV.internal) .and(tagC, c.toKV.internal) .and(tagD, d.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { d <- toOptions(result.getAll(tagD).asScala.iterator) @@ -1294,8 +1298,8 @@ object MultiJoin { .and(tagC, c.toKV.internal) .and(tagD, d.toKV.internal) .and(tagE, e.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { e <- toOptions(result.getAll(tagE).asScala.iterator) @@ -1316,8 +1320,8 @@ object MultiJoin { .and(tagD, d.toKV.internal) .and(tagE, e.toKV.internal) .and(tagF, f.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { f <- toOptions(result.getAll(tagF).asScala.iterator) @@ -1340,8 +1344,8 @@ object MultiJoin { .and(tagE, e.toKV.internal) .and(tagF, f.toKV.internal) .and(tagG, g.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { g <- toOptions(result.getAll(tagG).asScala.iterator) @@ -1366,8 +1370,8 @@ object MultiJoin { .and(tagF, f.toKV.internal) .and(tagG, g.toKV.internal) .and(tagH, h.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { h <- toOptions(result.getAll(tagH).asScala.iterator) @@ -1394,8 +1398,8 @@ object MultiJoin { .and(tagG, g.toKV.internal) .and(tagH, h.toKV.internal) .and(tagI, i.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { i <- toOptions(result.getAll(tagI).asScala.iterator) @@ -1424,8 +1428,8 @@ object MultiJoin { .and(tagH, h.toKV.internal) .and(tagI, i.toKV.internal) .and(tagJ, j.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { j <- toOptions(result.getAll(tagJ).asScala.iterator) @@ -1456,8 +1460,8 @@ object MultiJoin { .and(tagI, i.toKV.internal) .and(tagJ, j.toKV.internal) .and(tagK, k.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { k <- toOptions(result.getAll(tagK).asScala.iterator) @@ -1490,8 +1494,8 @@ object MultiJoin { .and(tagJ, j.toKV.internal) .and(tagK, k.toKV.internal) .and(tagL, l.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { l <- toOptions(result.getAll(tagL).asScala.iterator) @@ -1526,8 +1530,8 @@ object MultiJoin { .and(tagK, k.toKV.internal) .and(tagL, l.toKV.internal) .and(tagM, m.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { m <- toOptions(result.getAll(tagM).asScala.iterator) @@ -1564,8 +1568,8 @@ object MultiJoin { .and(tagL, l.toKV.internal) .and(tagM, m.toKV.internal) .and(tagN, n.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { n <- toOptions(result.getAll(tagN).asScala.iterator) @@ -1604,8 +1608,8 @@ object MultiJoin { .and(tagM, m.toKV.internal) .and(tagN, n.toKV.internal) .and(tagO, o.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { o <- toOptions(result.getAll(tagO).asScala.iterator) @@ -1646,8 +1650,8 @@ object MultiJoin { .and(tagN, n.toKV.internal) .and(tagO, o.toKV.internal) .and(tagP, p.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { p <- toOptions(result.getAll(tagP).asScala.iterator) @@ -1690,8 +1694,8 @@ object MultiJoin { .and(tagO, o.toKV.internal) .and(tagP, p.toKV.internal) .and(tagQ, q.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { q <- toOptions(result.getAll(tagQ).asScala.iterator) @@ -1736,8 +1740,8 @@ object MultiJoin { .and(tagP, p.toKV.internal) .and(tagQ, q.toKV.internal) .and(tagR, r.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { r <- toOptions(result.getAll(tagR).asScala.iterator) @@ -1784,8 +1788,8 @@ object MultiJoin { .and(tagQ, q.toKV.internal) .and(tagR, r.toKV.internal) .and(tagS, s.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { s <- toOptions(result.getAll(tagS).asScala.iterator) @@ -1834,8 +1838,8 @@ object MultiJoin { .and(tagR, r.toKV.internal) .and(tagS, s.toKV.internal) .and(tagT, t.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { t <- toOptions(result.getAll(tagT).asScala.iterator) @@ -1886,8 +1890,8 @@ object MultiJoin { .and(tagS, s.toKV.internal) .and(tagT, t.toKV.internal) .and(tagU, u.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { u <- toOptions(result.getAll(tagU).asScala.iterator) @@ -1940,8 +1944,8 @@ object MultiJoin { .and(tagT, t.toKV.internal) .and(tagU, u.toKV.internal) .and(tagV, v.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { v <- toOptions(result.getAll(tagV).asScala.iterator) @@ -1975,8 +1979,8 @@ object MultiJoin { val keyed = KeyedPCollectionTuple .of(tagA, a.toKV.internal) .and(tagB, b.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { b <- toOptions(result.getAll(tagB).asScala.iterator) @@ -1991,8 +1995,8 @@ object MultiJoin { .of(tagA, a.toKV.internal) .and(tagB, b.toKV.internal) .and(tagC, c.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { c <- toOptions(result.getAll(tagC).asScala.iterator) @@ -2009,8 +2013,8 @@ object MultiJoin { .and(tagB, b.toKV.internal) .and(tagC, c.toKV.internal) .and(tagD, d.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { d <- toOptions(result.getAll(tagD).asScala.iterator) @@ -2029,8 +2033,8 @@ object MultiJoin { .and(tagC, c.toKV.internal) .and(tagD, d.toKV.internal) .and(tagE, e.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { e <- toOptions(result.getAll(tagE).asScala.iterator) @@ -2051,8 +2055,8 @@ object MultiJoin { .and(tagD, d.toKV.internal) .and(tagE, e.toKV.internal) .and(tagF, f.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { f <- toOptions(result.getAll(tagF).asScala.iterator) @@ -2075,8 +2079,8 @@ object MultiJoin { .and(tagE, e.toKV.internal) .and(tagF, f.toKV.internal) .and(tagG, g.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { g <- toOptions(result.getAll(tagG).asScala.iterator) @@ -2101,8 +2105,8 @@ object MultiJoin { .and(tagF, f.toKV.internal) .and(tagG, g.toKV.internal) .and(tagH, h.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { h <- toOptions(result.getAll(tagH).asScala.iterator) @@ -2129,8 +2133,8 @@ object MultiJoin { .and(tagG, g.toKV.internal) .and(tagH, h.toKV.internal) .and(tagI, i.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { i <- toOptions(result.getAll(tagI).asScala.iterator) @@ -2159,8 +2163,8 @@ object MultiJoin { .and(tagH, h.toKV.internal) .and(tagI, i.toKV.internal) .and(tagJ, j.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { j <- toOptions(result.getAll(tagJ).asScala.iterator) @@ -2191,8 +2195,8 @@ object MultiJoin { .and(tagI, i.toKV.internal) .and(tagJ, j.toKV.internal) .and(tagK, k.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { k <- toOptions(result.getAll(tagK).asScala.iterator) @@ -2225,8 +2229,8 @@ object MultiJoin { .and(tagJ, j.toKV.internal) .and(tagK, k.toKV.internal) .and(tagL, l.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { l <- toOptions(result.getAll(tagL).asScala.iterator) @@ -2261,8 +2265,8 @@ object MultiJoin { .and(tagK, k.toKV.internal) .and(tagL, l.toKV.internal) .and(tagM, m.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { m <- toOptions(result.getAll(tagM).asScala.iterator) @@ -2299,8 +2303,8 @@ object MultiJoin { .and(tagL, l.toKV.internal) .and(tagM, m.toKV.internal) .and(tagN, n.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { n <- toOptions(result.getAll(tagN).asScala.iterator) @@ -2339,8 +2343,8 @@ object MultiJoin { .and(tagM, m.toKV.internal) .and(tagN, n.toKV.internal) .and(tagO, o.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { o <- toOptions(result.getAll(tagO).asScala.iterator) @@ -2381,8 +2385,8 @@ object MultiJoin { .and(tagN, n.toKV.internal) .and(tagO, o.toKV.internal) .and(tagP, p.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { p <- toOptions(result.getAll(tagP).asScala.iterator) @@ -2425,8 +2429,8 @@ object MultiJoin { .and(tagO, o.toKV.internal) .and(tagP, p.toKV.internal) .and(tagQ, q.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { q <- toOptions(result.getAll(tagQ).asScala.iterator) @@ -2471,8 +2475,8 @@ object MultiJoin { .and(tagP, p.toKV.internal) .and(tagQ, q.toKV.internal) .and(tagR, r.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { r <- toOptions(result.getAll(tagR).asScala.iterator) @@ -2519,8 +2523,8 @@ object MultiJoin { .and(tagQ, q.toKV.internal) .and(tagR, r.toKV.internal) .and(tagS, s.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { s <- toOptions(result.getAll(tagS).asScala.iterator) @@ -2569,8 +2573,8 @@ object MultiJoin { .and(tagR, r.toKV.internal) .and(tagS, s.toKV.internal) .and(tagT, t.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { t <- toOptions(result.getAll(tagT).asScala.iterator) @@ -2621,8 +2625,8 @@ object MultiJoin { .and(tagS, s.toKV.internal) .and(tagT, t.toKV.internal) .and(tagU, u.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { u <- toOptions(result.getAll(tagU).asScala.iterator) @@ -2675,8 +2679,8 @@ object MultiJoin { .and(tagT, t.toKV.internal) .and(tagU, u.toKV.internal) .and(tagV, v.toKV.internal) - .apply(CallSites.getCurrent, CoGroupByKey.create()) - a.context.wrap(keyed).flatMap { kv => + .apply("CoGroupByKey", CoGroupByKey.create()) + a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { v <- toOptions(result.getAll(tagV).asScala.iterator) diff --git a/scio-core/src/main/scala/com/spotify/scio/values/DoubleSCollectionFunctions.scala b/scio-core/src/main/scala/com/spotify/scio/values/DoubleSCollectionFunctions.scala index f34dea0644..a6d8e911df 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/DoubleSCollectionFunctions.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/DoubleSCollectionFunctions.scala @@ -39,22 +39,26 @@ class DoubleSCollectionFunctions(self: SCollection[Double]) { // def sum: SCollection[Double] = this.stats().map(_.sum) /** Compute the standard deviation of this SCollection's elements. */ - def stdev: SCollection[Double] = self.transform(_.stats.map(_.stdev)) + def stdev: SCollection[Double] = self.transform(_.stats.withName("StdDev").map(_.stdev)) /** Compute the variance of this SCollection's elements. */ - def variance: SCollection[Double] = self.transform(_.stats.map(_.variance)) + def variance: SCollection[Double] = self.transform(_.stats.withName("Variance").map(_.variance)) /** * Compute the sample standard deviation of this SCollection's elements (which corrects for bias * in estimating the standard deviation by dividing by N-1 instead of N). */ - def sampleStdev: SCollection[Double] = self.transform(_.stats.map(_.sampleStdev)) + def sampleStdev: SCollection[Double] = self.transform { + _.stats.withName("SampleStdDev").map(_.sampleStdev) + } /** * Compute the sample variance of this SCollection's elements (which corrects for bias in * estimating the variance by dividing by N-1 instead of N). */ - def sampleVariance: SCollection[Double] = self.transform(_.stats.map(_.sampleVariance)) + def sampleVariance: SCollection[Double] = self.transform { + _.stats.withName("SampleVariance").map(_.sampleVariance) + } // Ported from org.apache.spark.rdd.DoubleRDDFunctions diff --git a/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala b/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala index 9ba821d08a..42be114440 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala @@ -60,7 +60,7 @@ object SCollection { def unionAll[T: ClassTag](scs: Iterable[SCollection[T]]): SCollection[T] = { val o = PCollectionList .of(scs.map(_.internal).asJava) - .apply(CallSites.getCurrent, Flatten.pCollections()) + .apply("FlattenList", Flatten.pCollections()) scs.head.context.wrap(o) } @@ -126,7 +126,7 @@ sealed trait SCollection[T] extends PCollectionWrapper[T] with TransformNameable /** Apply a transform. */ private[values] def transform[U: ClassTag](f: SCollection[T] => SCollection[U]) : SCollection[U] = { - val o = internal.apply(CallSites.getCurrent, new PTransform[PCollection[T], PCollection[U]]() { + val o = internal.apply(this.tfName, new PTransform[PCollection[T], PCollection[U]]() { override def apply(input: PCollection[T]): PCollection[U] = { f(context.wrap(input)).internal } @@ -163,7 +163,7 @@ sealed trait SCollection[T] extends PCollectionWrapper[T] with TransformNameable def union(that: SCollection[T]): SCollection[T] = { val o = PCollectionList .of(internal).and(that.internal) - .apply(CallSites.getCurrent, Flatten.pCollections()) + .apply("FlattenList", Flatten.pCollections()) context.wrap(o) } diff --git a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideInput.scala b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideInput.scala index bce434f35a..92ea2c3ce7 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideInput.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideInput.scala @@ -106,7 +106,7 @@ class SCollectionWithSideInput[T: ClassTag] private[values] (val internal: PColl .withOutputTags(_mainTag.tupleTag, sideTags) .of(transformWithSideOutputsFn(sideOutputs, f)) - val pCollectionWrapper = this.internal.apply(CallSites.getCurrent, transform) + val pCollectionWrapper = this.internal.apply("TransformWithSideOutputs", transform) pCollectionWrapper.getAll.asScala .mapValues(context.wrap(_).asInstanceOf[SCollection[T]].setCoder(internal.getCoder)) .flatMap{ case(tt, col) => Try{tagToSide(tt.getId) -> col}.toOption } diff --git a/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala b/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala index e2d515e07d..bd4d04943e 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala @@ -31,12 +31,12 @@ trait TransformNameable[T <: TransformNameable[T]] { } } -private trait TransformNameProvider { +private[scio] trait TransformNameProvider { def name: String } -private object CallSiteNameProvider extends TransformNameProvider { +private[scio] object CallSiteNameProvider extends TransformNameProvider { def name: String = CallSites.getCurrent } -private class ConstNameProvider(val name: String) extends TransformNameProvider +private[scio] class ConstNameProvider(val name: String) extends TransformNameProvider diff --git a/scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala b/scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala index e0bd2dd2ca..7f4ec72b1a 100644 --- a/scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala @@ -18,6 +18,7 @@ package com.spotify.scio.values import com.spotify.scio.testing.PipelineSpec +import com.spotify.scio.util.MultiJoin class NamedTransformTest extends PipelineSpec { @@ -41,8 +42,7 @@ class NamedTransformTest extends PipelineSpec { runWithContext { sc => val p = sc.parallelize(Seq(1.0, 2.0, 3.0, 4.0, 5.0)) .withName("CalcVariance").variance - // TODO: give explicit names to internal transforms - //assertTransformName(p, "CalcVariance") + assertTransformName(p, "CalcVariance/Variance") } } @@ -114,6 +114,29 @@ class NamedTransformTest extends PipelineSpec { } } + "MultiJoin" should "support custom transform name" in { + runWithContext { sc => + val p1 = sc.parallelize(Seq(("a", 1), ("a", 2), ("b", 3), ("c", 4))) + val p2 = sc.parallelize(Seq(("a", 11), ("b", 12), ("b", 13), ("d", 14))) + val p = MultiJoin.withName("JoinEm").left(p1, p2) + assertTransformName(p, "JoinEm") + } + } + + "Duplicate transform name" should "have number to make unique" in { + runWithContext { sc => + val p1 = sc.parallelize(1 to 5) + .withName("MyTransform").map(_ * 2) + val p2 = p1 + .withName("MyTransform").map(_ * 3) + val p3 = p1 + .withName("MyTransform").map(_ * 4) + assertTransformName(p1, "MyTransform") + assertTransformName(p2, "MyTransform2") + assertTransformName(p3, "MyTransform3") + } + } + private def assertTransformName(p: PCollectionWrapper[_], tfName: String) = p.internal.getProducingTransformInternal.getFullName shouldBe tfName } diff --git a/scripts/multijoin.py b/scripts/multijoin.py index d4306e4e46..79883a8c90 100755 --- a/scripts/multijoin.py +++ b/scripts/multijoin.py @@ -61,9 +61,9 @@ def cogroup(out, n): print >> out, ' .of(tagA, a.toKV.internal)' for x in vals[1:]: print >> out, ' .and(tag%s, %s.toKV.internal)' % (x, x.lower()) - print >> out, ' .apply(CallSites.getCurrent, CoGroupByKey.create())' + print >> out, ' .apply("CoGroupByKey", CoGroupByKey.create())' - print >> out, ' a.context.wrap(keyed).map { kv =>' + print >> out, ' a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv =>' print >> out, ' val (key, result) = (kv.getKey, kv.getValue)' print >> out, ' (key, (%s))' % ', '.join('result.getAll(tag%s).asScala' % x for x in vals) # NOQA print >> out, ' }' @@ -85,9 +85,9 @@ def join(out, n): print >> out, ' .of(tagA, a.toKV.internal)' for x in vals[1:]: print >> out, ' .and(tag%s, %s.toKV.internal)' % (x, x.lower()) - print >> out, ' .apply(CallSites.getCurrent, CoGroupByKey.create())' + print >> out, ' .apply("CoGroupByKey", CoGroupByKey.create())' - print >> out, ' a.context.wrap(keyed).flatMap { kv =>' + print >> out, ' a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv =>' print >> out, ' val (key, result) = (kv.getKey, kv.getValue)' print >> out, ' for {' for x in reversed(vals): @@ -111,9 +111,9 @@ def left(out, n): print >> out, ' .of(tagA, a.toKV.internal)' for x in vals[1:]: print >> out, ' .and(tag%s, %s.toKV.internal)' % (x, x.lower()) - print >> out, ' .apply(CallSites.getCurrent, CoGroupByKey.create())' + print >> out, ' .apply("CoGroupByKey", CoGroupByKey.create())' - print >> out, ' a.context.wrap(keyed).flatMap { kv =>' + print >> out, ' a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv =>' print >> out, ' val (key, result) = (kv.getKey, kv.getValue)' print >> out, ' for {' for (i, x) in enumerate(reversed(vals)): @@ -140,9 +140,9 @@ def outer(out, n): print >> out, ' .of(tagA, a.toKV.internal)' for x in vals[1:]: print >> out, ' .and(tag%s, %s.toKV.internal)' % (x, x.lower()) - print >> out, ' .apply(CallSites.getCurrent, CoGroupByKey.create())' + print >> out, ' .apply("CoGroupByKey", CoGroupByKey.create())' - print >> out, ' a.context.wrap(keyed).flatMap { kv =>' + print >> out, ' a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv =>' print >> out, ' val (key, result) = (kv.getKey, kv.getValue)' print >> out, ' for {' for (i, x) in enumerate(reversed(vals)): @@ -186,13 +186,17 @@ def main(out): import com.google.cloud.dataflow.sdk.transforms.join.{CoGroupByKey, KeyedPCollectionTuple} # NOQA import com.google.cloud.dataflow.sdk.values.TupleTag import com.google.common.collect.Lists - import com.spotify.scio.values.SCollection + import com.spotify.scio.values.{CallSiteNameProvider, ConstNameProvider, SCollection, TransformNameProvider} import scala.collection.JavaConverters._ import scala.reflect.ClassTag object MultiJoin { + private var tfNameProvider: TransformNameProvider = CallSiteNameProvider + + def withName(name: String): MultiJoin.type = {tfNameProvider = new ConstNameProvider(name); this} + def toOptions[T](xs: Iterator[T]): Iterator[Option[T]] = if (xs.isEmpty) Iterator(None) else xs.map(Option(_)) ''').replace(' # NOQA', '').lstrip('\n') From 940b48c8ddc17fcdf8d14be07e37808c96323f15 Mon Sep 17 00:00:00 2001 From: Andrew Martin Date: Wed, 14 Dec 2016 13:57:52 -0500 Subject: [PATCH 03/11] Prevent successive calls to withName --- .../scala/com/spotify/scio/values/TransformNameable.scala | 2 ++ .../com/spotify/scio/values/NamedTransformTest.scala | 8 ++++++++ 2 files changed, 10 insertions(+) diff --git a/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala b/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala index bd4d04943e..b5067cdec5 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala @@ -26,6 +26,8 @@ trait TransformNameable[T <: TransformNameable[T]] { def tfName: String = nameProvider.name def withName(name: String): T = { + require(nameProvider.getClass != classOf[ConstNameProvider], + s"withName() has already been used to set '${tfName}' as the name for the next transform.") nameProvider = new ConstNameProvider(name) this } diff --git a/scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala b/scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala index 7f4ec72b1a..8102d265f7 100644 --- a/scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala @@ -137,6 +137,14 @@ class NamedTransformTest extends PipelineSpec { } } + "TransformNameable" should "prevent repeated calls to .withName" in { + intercept[IllegalArgumentException](runWithContext { sc => + val p1 = sc.parallelize(1 to 5) + .withName("Double").withName("DoubleMap").map(_ * 2) + }).getMessage shouldBe "requirement failed: withName() has already been used to set 'Double'" + + " as the name for the next transform." + } + private def assertTransformName(p: PCollectionWrapper[_], tfName: String) = p.internal.getProducingTransformInternal.getFullName shouldBe tfName } From 64a72470a42a8b02458be222ed38d39d293a50fb Mon Sep 17 00:00:00 2001 From: Andrew Martin Date: Wed, 14 Dec 2016 14:13:47 -0500 Subject: [PATCH 04/11] Always reset NameProvider after tfName requested. --- .../com/spotify/scio/values/TransformNameable.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala b/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala index b5067cdec5..2758608d55 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala @@ -19,11 +19,14 @@ package com.spotify.scio.values import com.spotify.scio.util.CallSites -trait TransformNameable[T <: TransformNameable[T]] { - this: T => +trait TransformNameable[T <: TransformNameable[T]] { this: T => private var nameProvider: TransformNameProvider = CallSiteNameProvider - def tfName: String = nameProvider.name + def tfName: String = { + val n = nameProvider.name + nameProvider = CallSiteNameProvider + n + } def withName(name: String): T = { require(nameProvider.getClass != classOf[ConstNameProvider], From 98c25397e911b7dce61c4fcea2d689d42c3da84b Mon Sep 17 00:00:00 2001 From: Andrew Martin Date: Wed, 14 Dec 2016 15:22:57 -0500 Subject: [PATCH 05/11] No need to use F-bounded type for TransformNameable. --- .../scala/com/spotify/scio/ScioContext.scala | 2 +- .../com/spotify/scio/util/MultiJoin.scala | 176 +++++++++--------- .../scio/values/PCollectionWrapper.scala | 3 +- .../com/spotify/scio/values/SCollection.scala | 2 +- .../values/SCollectionWithAccumulator.scala | 2 +- .../scio/values/SCollectionWithFanout.scala | 2 +- .../values/SCollectionWithHotKeyFanout.scala | 4 +- .../values/SCollectionWithSideInput.scala | 2 +- .../values/SCollectionWithSideOutput.scala | 2 +- .../scio/values/TransformNameable.scala | 10 +- .../scio/values/WindowedSCollection.scala | 2 +- scripts/multijoin.py | 16 +- 12 files changed, 107 insertions(+), 116 deletions(-) diff --git a/scio-core/src/main/scala/com/spotify/scio/ScioContext.scala b/scio-core/src/main/scala/com/spotify/scio/ScioContext.scala index a5aa85eed2..46f75a1c44 100644 --- a/scio-core/src/main/scala/com/spotify/scio/ScioContext.scala +++ b/scio-core/src/main/scala/com/spotify/scio/ScioContext.scala @@ -139,7 +139,7 @@ object ScioContext { // scalastyle:off number.of.methods class ScioContext private[scio] (val options: PipelineOptions, private var artifacts: List[String]) - extends TransformNameable[ScioContext] { + extends TransformNameable { private implicit val context: ScioContext = this diff --git a/scio-core/src/main/scala/com/spotify/scio/util/MultiJoin.scala b/scio-core/src/main/scala/com/spotify/scio/util/MultiJoin.scala index f42fe9c24d..0da52734a6 100644 --- a/scio-core/src/main/scala/com/spotify/scio/util/MultiJoin.scala +++ b/scio-core/src/main/scala/com/spotify/scio/util/MultiJoin.scala @@ -29,16 +29,12 @@ package com.spotify.scio.util import com.google.cloud.dataflow.sdk.transforms.join.{CoGroupByKey, KeyedPCollectionTuple} import com.google.cloud.dataflow.sdk.values.TupleTag import com.google.common.collect.Lists -import com.spotify.scio.values.{CallSiteNameProvider, ConstNameProvider, SCollection, TransformNameProvider} +import com.spotify.scio.values.{SCollection, TransformNameable} import scala.collection.JavaConverters._ import scala.reflect.ClassTag -object MultiJoin { - - private var tfNameProvider: TransformNameProvider = CallSiteNameProvider - - def withName(name: String): MultiJoin.type = {tfNameProvider = new ConstNameProvider(name); this} +object MultiJoin extends TransformNameable { def toOptions[T](xs: Iterator[T]): Iterator[Option[T]] = if (xs.isEmpty) Iterator(None) else xs.map(Option(_)) @@ -48,7 +44,7 @@ object MultiJoin { .of(tagA, a.toKV.internal) .and(tagB, b.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala)) } @@ -61,7 +57,7 @@ object MultiJoin { .and(tagB, b.toKV.internal) .and(tagC, c.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala)) } @@ -75,7 +71,7 @@ object MultiJoin { .and(tagC, c.toKV.internal) .and(tagD, d.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala)) } @@ -90,7 +86,7 @@ object MultiJoin { .and(tagD, d.toKV.internal) .and(tagE, e.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala)) } @@ -106,7 +102,7 @@ object MultiJoin { .and(tagE, e.toKV.internal) .and(tagF, f.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala)) } @@ -123,7 +119,7 @@ object MultiJoin { .and(tagF, f.toKV.internal) .and(tagG, g.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala)) } @@ -141,7 +137,7 @@ object MultiJoin { .and(tagG, g.toKV.internal) .and(tagH, h.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala)) } @@ -160,7 +156,7 @@ object MultiJoin { .and(tagH, h.toKV.internal) .and(tagI, i.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala)) } @@ -180,7 +176,7 @@ object MultiJoin { .and(tagI, i.toKV.internal) .and(tagJ, j.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala)) } @@ -201,7 +197,7 @@ object MultiJoin { .and(tagJ, j.toKV.internal) .and(tagK, k.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala)) } @@ -223,7 +219,7 @@ object MultiJoin { .and(tagK, k.toKV.internal) .and(tagL, l.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala)) } @@ -246,7 +242,7 @@ object MultiJoin { .and(tagL, l.toKV.internal) .and(tagM, m.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala)) } @@ -270,7 +266,7 @@ object MultiJoin { .and(tagM, m.toKV.internal) .and(tagN, n.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala, result.getAll(tagN).asScala)) } @@ -295,7 +291,7 @@ object MultiJoin { .and(tagN, n.toKV.internal) .and(tagO, o.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala, result.getAll(tagN).asScala, result.getAll(tagO).asScala)) } @@ -321,7 +317,7 @@ object MultiJoin { .and(tagO, o.toKV.internal) .and(tagP, p.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala, result.getAll(tagN).asScala, result.getAll(tagO).asScala, result.getAll(tagP).asScala)) } @@ -348,7 +344,7 @@ object MultiJoin { .and(tagP, p.toKV.internal) .and(tagQ, q.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala, result.getAll(tagN).asScala, result.getAll(tagO).asScala, result.getAll(tagP).asScala, result.getAll(tagQ).asScala)) } @@ -376,7 +372,7 @@ object MultiJoin { .and(tagQ, q.toKV.internal) .and(tagR, r.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala, result.getAll(tagN).asScala, result.getAll(tagO).asScala, result.getAll(tagP).asScala, result.getAll(tagQ).asScala, result.getAll(tagR).asScala)) } @@ -405,7 +401,7 @@ object MultiJoin { .and(tagR, r.toKV.internal) .and(tagS, s.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala, result.getAll(tagN).asScala, result.getAll(tagO).asScala, result.getAll(tagP).asScala, result.getAll(tagQ).asScala, result.getAll(tagR).asScala, result.getAll(tagS).asScala)) } @@ -435,7 +431,7 @@ object MultiJoin { .and(tagS, s.toKV.internal) .and(tagT, t.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala, result.getAll(tagN).asScala, result.getAll(tagO).asScala, result.getAll(tagP).asScala, result.getAll(tagQ).asScala, result.getAll(tagR).asScala, result.getAll(tagS).asScala, result.getAll(tagT).asScala)) } @@ -466,7 +462,7 @@ object MultiJoin { .and(tagT, t.toKV.internal) .and(tagU, u.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala, result.getAll(tagN).asScala, result.getAll(tagO).asScala, result.getAll(tagP).asScala, result.getAll(tagQ).asScala, result.getAll(tagR).asScala, result.getAll(tagS).asScala, result.getAll(tagT).asScala, result.getAll(tagU).asScala)) } @@ -498,7 +494,7 @@ object MultiJoin { .and(tagU, u.toKV.internal) .and(tagV, v.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv => + a.context.wrap(keyed).withName(this.tfName).map { kv => val (key, result) = (kv.getKey, kv.getValue) (key, (result.getAll(tagA).asScala, result.getAll(tagB).asScala, result.getAll(tagC).asScala, result.getAll(tagD).asScala, result.getAll(tagE).asScala, result.getAll(tagF).asScala, result.getAll(tagG).asScala, result.getAll(tagH).asScala, result.getAll(tagI).asScala, result.getAll(tagJ).asScala, result.getAll(tagK).asScala, result.getAll(tagL).asScala, result.getAll(tagM).asScala, result.getAll(tagN).asScala, result.getAll(tagO).asScala, result.getAll(tagP).asScala, result.getAll(tagQ).asScala, result.getAll(tagR).asScala, result.getAll(tagS).asScala, result.getAll(tagT).asScala, result.getAll(tagU).asScala, result.getAll(tagV).asScala)) } @@ -510,7 +506,7 @@ object MultiJoin { .of(tagA, a.toKV.internal) .and(tagB, b.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { b <- result.getAll(tagB).asScala.iterator @@ -526,7 +522,7 @@ object MultiJoin { .and(tagB, b.toKV.internal) .and(tagC, c.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { c <- result.getAll(tagC).asScala.iterator @@ -544,7 +540,7 @@ object MultiJoin { .and(tagC, c.toKV.internal) .and(tagD, d.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { d <- result.getAll(tagD).asScala.iterator @@ -564,7 +560,7 @@ object MultiJoin { .and(tagD, d.toKV.internal) .and(tagE, e.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { e <- result.getAll(tagE).asScala.iterator @@ -586,7 +582,7 @@ object MultiJoin { .and(tagE, e.toKV.internal) .and(tagF, f.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { f <- result.getAll(tagF).asScala.iterator @@ -610,7 +606,7 @@ object MultiJoin { .and(tagF, f.toKV.internal) .and(tagG, g.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { g <- result.getAll(tagG).asScala.iterator @@ -636,7 +632,7 @@ object MultiJoin { .and(tagG, g.toKV.internal) .and(tagH, h.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { h <- result.getAll(tagH).asScala.iterator @@ -664,7 +660,7 @@ object MultiJoin { .and(tagH, h.toKV.internal) .and(tagI, i.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { i <- result.getAll(tagI).asScala.iterator @@ -694,7 +690,7 @@ object MultiJoin { .and(tagI, i.toKV.internal) .and(tagJ, j.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { j <- result.getAll(tagJ).asScala.iterator @@ -726,7 +722,7 @@ object MultiJoin { .and(tagJ, j.toKV.internal) .and(tagK, k.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { k <- result.getAll(tagK).asScala.iterator @@ -760,7 +756,7 @@ object MultiJoin { .and(tagK, k.toKV.internal) .and(tagL, l.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { l <- result.getAll(tagL).asScala.iterator @@ -796,7 +792,7 @@ object MultiJoin { .and(tagL, l.toKV.internal) .and(tagM, m.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { m <- result.getAll(tagM).asScala.iterator @@ -834,7 +830,7 @@ object MultiJoin { .and(tagM, m.toKV.internal) .and(tagN, n.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { n <- result.getAll(tagN).asScala.iterator @@ -874,7 +870,7 @@ object MultiJoin { .and(tagN, n.toKV.internal) .and(tagO, o.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { o <- result.getAll(tagO).asScala.iterator @@ -916,7 +912,7 @@ object MultiJoin { .and(tagO, o.toKV.internal) .and(tagP, p.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { p <- result.getAll(tagP).asScala.iterator @@ -960,7 +956,7 @@ object MultiJoin { .and(tagP, p.toKV.internal) .and(tagQ, q.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { q <- result.getAll(tagQ).asScala.iterator @@ -1006,7 +1002,7 @@ object MultiJoin { .and(tagQ, q.toKV.internal) .and(tagR, r.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { r <- result.getAll(tagR).asScala.iterator @@ -1054,7 +1050,7 @@ object MultiJoin { .and(tagR, r.toKV.internal) .and(tagS, s.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { s <- result.getAll(tagS).asScala.iterator @@ -1104,7 +1100,7 @@ object MultiJoin { .and(tagS, s.toKV.internal) .and(tagT, t.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { t <- result.getAll(tagT).asScala.iterator @@ -1156,7 +1152,7 @@ object MultiJoin { .and(tagT, t.toKV.internal) .and(tagU, u.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { u <- result.getAll(tagU).asScala.iterator @@ -1210,7 +1206,7 @@ object MultiJoin { .and(tagU, u.toKV.internal) .and(tagV, v.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { v <- result.getAll(tagV).asScala.iterator @@ -1245,7 +1241,7 @@ object MultiJoin { .of(tagA, a.toKV.internal) .and(tagB, b.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { b <- toOptions(result.getAll(tagB).asScala.iterator) @@ -1261,7 +1257,7 @@ object MultiJoin { .and(tagB, b.toKV.internal) .and(tagC, c.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { c <- toOptions(result.getAll(tagC).asScala.iterator) @@ -1279,7 +1275,7 @@ object MultiJoin { .and(tagC, c.toKV.internal) .and(tagD, d.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { d <- toOptions(result.getAll(tagD).asScala.iterator) @@ -1299,7 +1295,7 @@ object MultiJoin { .and(tagD, d.toKV.internal) .and(tagE, e.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { e <- toOptions(result.getAll(tagE).asScala.iterator) @@ -1321,7 +1317,7 @@ object MultiJoin { .and(tagE, e.toKV.internal) .and(tagF, f.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { f <- toOptions(result.getAll(tagF).asScala.iterator) @@ -1345,7 +1341,7 @@ object MultiJoin { .and(tagF, f.toKV.internal) .and(tagG, g.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { g <- toOptions(result.getAll(tagG).asScala.iterator) @@ -1371,7 +1367,7 @@ object MultiJoin { .and(tagG, g.toKV.internal) .and(tagH, h.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { h <- toOptions(result.getAll(tagH).asScala.iterator) @@ -1399,7 +1395,7 @@ object MultiJoin { .and(tagH, h.toKV.internal) .and(tagI, i.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { i <- toOptions(result.getAll(tagI).asScala.iterator) @@ -1429,7 +1425,7 @@ object MultiJoin { .and(tagI, i.toKV.internal) .and(tagJ, j.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { j <- toOptions(result.getAll(tagJ).asScala.iterator) @@ -1461,7 +1457,7 @@ object MultiJoin { .and(tagJ, j.toKV.internal) .and(tagK, k.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { k <- toOptions(result.getAll(tagK).asScala.iterator) @@ -1495,7 +1491,7 @@ object MultiJoin { .and(tagK, k.toKV.internal) .and(tagL, l.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { l <- toOptions(result.getAll(tagL).asScala.iterator) @@ -1531,7 +1527,7 @@ object MultiJoin { .and(tagL, l.toKV.internal) .and(tagM, m.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { m <- toOptions(result.getAll(tagM).asScala.iterator) @@ -1569,7 +1565,7 @@ object MultiJoin { .and(tagM, m.toKV.internal) .and(tagN, n.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { n <- toOptions(result.getAll(tagN).asScala.iterator) @@ -1609,7 +1605,7 @@ object MultiJoin { .and(tagN, n.toKV.internal) .and(tagO, o.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { o <- toOptions(result.getAll(tagO).asScala.iterator) @@ -1651,7 +1647,7 @@ object MultiJoin { .and(tagO, o.toKV.internal) .and(tagP, p.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { p <- toOptions(result.getAll(tagP).asScala.iterator) @@ -1695,7 +1691,7 @@ object MultiJoin { .and(tagP, p.toKV.internal) .and(tagQ, q.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { q <- toOptions(result.getAll(tagQ).asScala.iterator) @@ -1741,7 +1737,7 @@ object MultiJoin { .and(tagQ, q.toKV.internal) .and(tagR, r.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { r <- toOptions(result.getAll(tagR).asScala.iterator) @@ -1789,7 +1785,7 @@ object MultiJoin { .and(tagR, r.toKV.internal) .and(tagS, s.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { s <- toOptions(result.getAll(tagS).asScala.iterator) @@ -1839,7 +1835,7 @@ object MultiJoin { .and(tagS, s.toKV.internal) .and(tagT, t.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { t <- toOptions(result.getAll(tagT).asScala.iterator) @@ -1891,7 +1887,7 @@ object MultiJoin { .and(tagT, t.toKV.internal) .and(tagU, u.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { u <- toOptions(result.getAll(tagU).asScala.iterator) @@ -1945,7 +1941,7 @@ object MultiJoin { .and(tagU, u.toKV.internal) .and(tagV, v.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { v <- toOptions(result.getAll(tagV).asScala.iterator) @@ -1980,7 +1976,7 @@ object MultiJoin { .of(tagA, a.toKV.internal) .and(tagB, b.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { b <- toOptions(result.getAll(tagB).asScala.iterator) @@ -1996,7 +1992,7 @@ object MultiJoin { .and(tagB, b.toKV.internal) .and(tagC, c.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { c <- toOptions(result.getAll(tagC).asScala.iterator) @@ -2014,7 +2010,7 @@ object MultiJoin { .and(tagC, c.toKV.internal) .and(tagD, d.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { d <- toOptions(result.getAll(tagD).asScala.iterator) @@ -2034,7 +2030,7 @@ object MultiJoin { .and(tagD, d.toKV.internal) .and(tagE, e.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { e <- toOptions(result.getAll(tagE).asScala.iterator) @@ -2056,7 +2052,7 @@ object MultiJoin { .and(tagE, e.toKV.internal) .and(tagF, f.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { f <- toOptions(result.getAll(tagF).asScala.iterator) @@ -2080,7 +2076,7 @@ object MultiJoin { .and(tagF, f.toKV.internal) .and(tagG, g.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { g <- toOptions(result.getAll(tagG).asScala.iterator) @@ -2106,7 +2102,7 @@ object MultiJoin { .and(tagG, g.toKV.internal) .and(tagH, h.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { h <- toOptions(result.getAll(tagH).asScala.iterator) @@ -2134,7 +2130,7 @@ object MultiJoin { .and(tagH, h.toKV.internal) .and(tagI, i.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { i <- toOptions(result.getAll(tagI).asScala.iterator) @@ -2164,7 +2160,7 @@ object MultiJoin { .and(tagI, i.toKV.internal) .and(tagJ, j.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { j <- toOptions(result.getAll(tagJ).asScala.iterator) @@ -2196,7 +2192,7 @@ object MultiJoin { .and(tagJ, j.toKV.internal) .and(tagK, k.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { k <- toOptions(result.getAll(tagK).asScala.iterator) @@ -2230,7 +2226,7 @@ object MultiJoin { .and(tagK, k.toKV.internal) .and(tagL, l.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { l <- toOptions(result.getAll(tagL).asScala.iterator) @@ -2266,7 +2262,7 @@ object MultiJoin { .and(tagL, l.toKV.internal) .and(tagM, m.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { m <- toOptions(result.getAll(tagM).asScala.iterator) @@ -2304,7 +2300,7 @@ object MultiJoin { .and(tagM, m.toKV.internal) .and(tagN, n.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { n <- toOptions(result.getAll(tagN).asScala.iterator) @@ -2344,7 +2340,7 @@ object MultiJoin { .and(tagN, n.toKV.internal) .and(tagO, o.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { o <- toOptions(result.getAll(tagO).asScala.iterator) @@ -2386,7 +2382,7 @@ object MultiJoin { .and(tagO, o.toKV.internal) .and(tagP, p.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { p <- toOptions(result.getAll(tagP).asScala.iterator) @@ -2430,7 +2426,7 @@ object MultiJoin { .and(tagP, p.toKV.internal) .and(tagQ, q.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { q <- toOptions(result.getAll(tagQ).asScala.iterator) @@ -2476,7 +2472,7 @@ object MultiJoin { .and(tagQ, q.toKV.internal) .and(tagR, r.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { r <- toOptions(result.getAll(tagR).asScala.iterator) @@ -2524,7 +2520,7 @@ object MultiJoin { .and(tagR, r.toKV.internal) .and(tagS, s.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { s <- toOptions(result.getAll(tagS).asScala.iterator) @@ -2574,7 +2570,7 @@ object MultiJoin { .and(tagS, s.toKV.internal) .and(tagT, t.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { t <- toOptions(result.getAll(tagT).asScala.iterator) @@ -2626,7 +2622,7 @@ object MultiJoin { .and(tagT, t.toKV.internal) .and(tagU, u.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { u <- toOptions(result.getAll(tagU).asScala.iterator) @@ -2680,7 +2676,7 @@ object MultiJoin { .and(tagU, u.toKV.internal) .and(tagV, v.toKV.internal) .apply("CoGroupByKey", CoGroupByKey.create()) - a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv => + a.context.wrap(keyed).withName(this.tfName).flatMap { kv => val (key, result) = (kv.getKey, kv.getValue) for { v <- toOptions(result.getAll(tagV).asScala.iterator) diff --git a/scio-core/src/main/scala/com/spotify/scio/values/PCollectionWrapper.scala b/scio-core/src/main/scala/com/spotify/scio/values/PCollectionWrapper.scala index d6bcdd1c75..cdf5de82f8 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/PCollectionWrapper.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/PCollectionWrapper.scala @@ -24,8 +24,7 @@ import com.spotify.scio.{Implicits, ScioContext} import scala.reflect.ClassTag -private[values] trait PCollectionWrapper[T] { - this: TransformNameable[_ <: PCollectionWrapper[T]] => +private[values] trait PCollectionWrapper[T] extends TransformNameable { import Implicits._ diff --git a/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala b/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala index 42be114440..d61b079d29 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala @@ -98,7 +98,7 @@ object SCollection { * @groupname transform Transformations * @groupname window Windowing Operations */ -sealed trait SCollection[T] extends PCollectionWrapper[T] with TransformNameable[SCollection[T]] { +sealed trait SCollection[T] extends PCollectionWrapper[T] { import TupleFunctions._ diff --git a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithAccumulator.scala b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithAccumulator.scala index 85c382e82b..c3ba299fba 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithAccumulator.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithAccumulator.scala @@ -29,7 +29,7 @@ import scala.reflect.ClassTag */ class SCollectionWithAccumulator[T: ClassTag] private[values] (val internal: PCollection[T], val context: ScioContext, acc: Seq[Accumulator[_]]) - extends PCollectionWrapper[T] with TransformNameable[SCollectionWithAccumulator[T]] { + extends PCollectionWrapper[T] { protected val ct: ClassTag[T] = implicitly[ClassTag[T]] diff --git a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithFanout.scala b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithFanout.scala index 8927c6f751..b0a21c0c7d 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithFanout.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithFanout.scala @@ -32,7 +32,7 @@ import scala.reflect.ClassTag class SCollectionWithFanout[T: ClassTag] private[values] (val internal: PCollection[T], val context: ScioContext, private val fanout: Int) - extends PCollectionWrapper[T] with TransformNameable[SCollectionWithFanout[T]] { + extends PCollectionWrapper[T] { protected val ct: ClassTag[T] = implicitly[ClassTag[T]] diff --git a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithHotKeyFanout.scala b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithHotKeyFanout.scala index 0f7db78781..0685965429 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithHotKeyFanout.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithHotKeyFanout.scala @@ -32,7 +32,7 @@ import scala.reflect.ClassTag class SCollectionWithHotKeyFanout[K: ClassTag, V: ClassTag] (private val self: PairSCollectionFunctions[K, V], private val hotKeyFanout: Either[K => Int, Int]) - extends TransformNameable[SCollectionWithHotKeyFanout[K, V]] { + extends TransformNameable { private def withFanout[K, I, O](combine: Combine.PerKey[K, I, O]) : PerKeyWithHotKeyFanout[K, I, O] = this.hotKeyFanout match { @@ -43,7 +43,7 @@ class SCollectionWithHotKeyFanout[K: ClassTag, V: ClassTag] combine.withHotKeyFanout(f) } - override def withName(name: String): SCollectionWithHotKeyFanout[K, V] = { + override def withName(name: String): this.type = { self.self.withName(name) this } diff --git a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideInput.scala b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideInput.scala index 92ea2c3ce7..dcae4da52d 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideInput.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideInput.scala @@ -34,7 +34,7 @@ import scala.util.Try class SCollectionWithSideInput[T: ClassTag] private[values] (val internal: PCollection[T], val context: ScioContext, sides: Iterable[SideInput[_]]) - extends PCollectionWrapper[T] with TransformNameable[SCollectionWithSideInput[T]] { + extends PCollectionWrapper[T] { protected val ct: ClassTag[T] = implicitly[ClassTag[T]] diff --git a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideOutput.scala b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideOutput.scala index 8b228e2109..aa29e348cc 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideOutput.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/SCollectionWithSideOutput.scala @@ -35,7 +35,7 @@ class SCollectionWithSideOutput[T: ClassTag] private[values] (val internal: PCollection[T], val context: ScioContext, sides: Iterable[SideOutput[_]]) - extends PCollectionWrapper[T] with TransformNameable[SCollectionWithSideOutput[T]] { + extends PCollectionWrapper[T] { protected val ct: ClassTag[T] = implicitly[ClassTag[T]] diff --git a/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala b/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala index 2758608d55..d5542b97dd 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala @@ -19,7 +19,7 @@ package com.spotify.scio.values import com.spotify.scio.util.CallSites -trait TransformNameable[T <: TransformNameable[T]] { this: T => +trait TransformNameable { private var nameProvider: TransformNameProvider = CallSiteNameProvider def tfName: String = { @@ -28,7 +28,7 @@ trait TransformNameable[T <: TransformNameable[T]] { this: T => n } - def withName(name: String): T = { + def withName(name: String): this.type = { require(nameProvider.getClass != classOf[ConstNameProvider], s"withName() has already been used to set '${tfName}' as the name for the next transform.") nameProvider = new ConstNameProvider(name) @@ -36,12 +36,12 @@ trait TransformNameable[T <: TransformNameable[T]] { this: T => } } -private[scio] trait TransformNameProvider { +private trait TransformNameProvider { def name: String } -private[scio] object CallSiteNameProvider extends TransformNameProvider { +private object CallSiteNameProvider extends TransformNameProvider { def name: String = CallSites.getCurrent } -private[scio] class ConstNameProvider(val name: String) extends TransformNameProvider +private class ConstNameProvider(val name: String) extends TransformNameProvider diff --git a/scio-core/src/main/scala/com/spotify/scio/values/WindowedSCollection.scala b/scio-core/src/main/scala/com/spotify/scio/values/WindowedSCollection.scala index 46f5f2fe1e..f4311efbdd 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/WindowedSCollection.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/WindowedSCollection.scala @@ -40,7 +40,7 @@ case class WindowedValue[T](value: T, timestamp: Instant, window: BoundedWindow, class WindowedSCollection[T: ClassTag] private[values] (val internal: PCollection[T], val context: ScioContext) - extends PCollectionWrapper[T] with TransformNameable[WindowedSCollection[T]] { + extends PCollectionWrapper[T] { protected val ct: ClassTag[T] = implicitly[ClassTag[T]] diff --git a/scripts/multijoin.py b/scripts/multijoin.py index 79883a8c90..1f995426ee 100755 --- a/scripts/multijoin.py +++ b/scripts/multijoin.py @@ -63,7 +63,7 @@ def cogroup(out, n): print >> out, ' .and(tag%s, %s.toKV.internal)' % (x, x.lower()) print >> out, ' .apply("CoGroupByKey", CoGroupByKey.create())' - print >> out, ' a.context.wrap(keyed).withName(this.tfNameProvider.name).map { kv =>' + print >> out, ' a.context.wrap(keyed).withName(this.tfName).map { kv =>' print >> out, ' val (key, result) = (kv.getKey, kv.getValue)' print >> out, ' (key, (%s))' % ', '.join('result.getAll(tag%s).asScala' % x for x in vals) # NOQA print >> out, ' }' @@ -87,7 +87,7 @@ def join(out, n): print >> out, ' .and(tag%s, %s.toKV.internal)' % (x, x.lower()) print >> out, ' .apply("CoGroupByKey", CoGroupByKey.create())' - print >> out, ' a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv =>' + print >> out, ' a.context.wrap(keyed).withName(this.tfName).flatMap { kv =>' print >> out, ' val (key, result) = (kv.getKey, kv.getValue)' print >> out, ' for {' for x in reversed(vals): @@ -113,7 +113,7 @@ def left(out, n): print >> out, ' .and(tag%s, %s.toKV.internal)' % (x, x.lower()) print >> out, ' .apply("CoGroupByKey", CoGroupByKey.create())' - print >> out, ' a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv =>' + print >> out, ' a.context.wrap(keyed).withName(this.tfName).flatMap { kv =>' print >> out, ' val (key, result) = (kv.getKey, kv.getValue)' print >> out, ' for {' for (i, x) in enumerate(reversed(vals)): @@ -142,7 +142,7 @@ def outer(out, n): print >> out, ' .and(tag%s, %s.toKV.internal)' % (x, x.lower()) print >> out, ' .apply("CoGroupByKey", CoGroupByKey.create())' - print >> out, ' a.context.wrap(keyed).withName(this.tfNameProvider.name).flatMap { kv =>' + print >> out, ' a.context.wrap(keyed).withName(this.tfName).flatMap { kv =>' print >> out, ' val (key, result) = (kv.getKey, kv.getValue)' print >> out, ' for {' for (i, x) in enumerate(reversed(vals)): @@ -186,16 +186,12 @@ def main(out): import com.google.cloud.dataflow.sdk.transforms.join.{CoGroupByKey, KeyedPCollectionTuple} # NOQA import com.google.cloud.dataflow.sdk.values.TupleTag import com.google.common.collect.Lists - import com.spotify.scio.values.{CallSiteNameProvider, ConstNameProvider, SCollection, TransformNameProvider} + import com.spotify.scio.values.{SCollection, TransformNameable} import scala.collection.JavaConverters._ import scala.reflect.ClassTag - object MultiJoin { - - private var tfNameProvider: TransformNameProvider = CallSiteNameProvider - - def withName(name: String): MultiJoin.type = {tfNameProvider = new ConstNameProvider(name); this} + object MultiJoin extends TransformNameable { def toOptions[T](xs: Iterator[T]): Iterator[Option[T]] = if (xs.isEmpty) Iterator(None) else xs.map(Option(_)) ''').replace(' # NOQA', '').lstrip('\n') From 407b4e617eb970ceddb36c9f82df85ffa89425e3 Mon Sep 17 00:00:00 2001 From: Andrew Martin Date: Tue, 17 Jan 2017 14:11:27 -0500 Subject: [PATCH 06/11] Accumulator names should still use CallSites.getCurrentName --- .../main/scala/com/spotify/scio/accumulators/package.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/scio-core/src/main/scala/com/spotify/scio/accumulators/package.scala b/scio-core/src/main/scala/com/spotify/scio/accumulators/package.scala index f9b7053c41..b52ae8fecd 100644 --- a/scio-core/src/main/scala/com/spotify/scio/accumulators/package.scala +++ b/scio-core/src/main/scala/com/spotify/scio/accumulators/package.scala @@ -60,7 +60,7 @@ package object accumulators { /** Count elements with an automatically created accumulator. */ def accumulateCount: SCollection[T] = - self.accumulateCount(self.context.sumAccumulator[Long](self.tfName)) + self.accumulateCount(self.context.sumAccumulator[Long](CallSites.getCurrentName)) /** Count positive and negative results in a filter with the given accumulators. */ def accumulateCountFilter(accPos: Accumulator[Long], @@ -79,8 +79,8 @@ package object accumulators { /** Count positive and negative results in a filter with automatically created accumulators. */ def accumulateCountFilter(f: T => Boolean): SCollection[T] = { - val accPos = self.context.sumAccumulator[Long]("Positive#" + self.tfName) - val accNeg = self.context.sumAccumulator[Long]("Negative#" + self.tfName) + val accPos = self.context.sumAccumulator[Long]("Positive#" + CallSites.getCurrentName) + val accNeg = self.context.sumAccumulator[Long]("Negative#" + CallSites.getCurrentName) self.accumulateCountFilter(accPos, accNeg)(f) } From fa29b258179258cc5b23b52a3b02aabec7697287 Mon Sep 17 00:00:00 2001 From: Andrew Martin Date: Tue, 17 Jan 2017 14:37:23 -0500 Subject: [PATCH 07/11] Fix transform names for union and unionAll --- .../src/main/scala/com/spotify/scio/values/SCollection.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala b/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala index d61b079d29..3416ff88c1 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/SCollection.scala @@ -60,7 +60,7 @@ object SCollection { def unionAll[T: ClassTag](scs: Iterable[SCollection[T]]): SCollection[T] = { val o = PCollectionList .of(scs.map(_.internal).asJava) - .apply("FlattenList", Flatten.pCollections()) + .apply("UnionAll", Flatten.pCollections()) scs.head.context.wrap(o) } @@ -163,7 +163,7 @@ sealed trait SCollection[T] extends PCollectionWrapper[T] { def union(that: SCollection[T]): SCollection[T] = { val o = PCollectionList .of(internal).and(that.internal) - .apply("FlattenList", Flatten.pCollections()) + .apply(this.tfName, Flatten.pCollections()) context.wrap(o) } From 3139408d80e7cfeb48ab60a8b617b54b091db415 Mon Sep 17 00:00:00 2001 From: Andrew Martin Date: Tue, 17 Jan 2017 14:37:59 -0500 Subject: [PATCH 08/11] Ignore internal transform names in DoubleSCollectionFunctions for now. --- .../spotify/scio/values/DoubleSCollectionFunctions.scala | 8 ++++---- .../com/spotify/scio/values/NamedTransformTest.scala | 5 ++++- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/scio-core/src/main/scala/com/spotify/scio/values/DoubleSCollectionFunctions.scala b/scio-core/src/main/scala/com/spotify/scio/values/DoubleSCollectionFunctions.scala index a6d8e911df..06f19c69a9 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/DoubleSCollectionFunctions.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/DoubleSCollectionFunctions.scala @@ -39,17 +39,17 @@ class DoubleSCollectionFunctions(self: SCollection[Double]) { // def sum: SCollection[Double] = this.stats().map(_.sum) /** Compute the standard deviation of this SCollection's elements. */ - def stdev: SCollection[Double] = self.transform(_.stats.withName("StdDev").map(_.stdev)) + def stdev: SCollection[Double] = self.transform(_.stats.map(_.stdev)) /** Compute the variance of this SCollection's elements. */ - def variance: SCollection[Double] = self.transform(_.stats.withName("Variance").map(_.variance)) + def variance: SCollection[Double] = self.transform(_.stats.map(_.variance)) /** * Compute the sample standard deviation of this SCollection's elements (which corrects for bias * in estimating the standard deviation by dividing by N-1 instead of N). */ def sampleStdev: SCollection[Double] = self.transform { - _.stats.withName("SampleStdDev").map(_.sampleStdev) + _.stats.map(_.sampleStdev) } /** @@ -57,7 +57,7 @@ class DoubleSCollectionFunctions(self: SCollection[Double]) { * estimating the variance by dividing by N-1 instead of N). */ def sampleVariance: SCollection[Double] = self.transform { - _.stats.withName("SampleVariance").map(_.sampleVariance) + _.stats.map(_.sampleVariance) } // Ported from org.apache.spark.rdd.DoubleRDDFunctions diff --git a/scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala b/scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala index 8102d265f7..3bc769ce99 100644 --- a/scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala @@ -42,7 +42,7 @@ class NamedTransformTest extends PipelineSpec { runWithContext { sc => val p = sc.parallelize(Seq(1.0, 2.0, 3.0, 4.0, 5.0)) .withName("CalcVariance").variance - assertTransformName(p, "CalcVariance/Variance") + assertOuterTransformName(p, "CalcVariance") } } @@ -147,4 +147,7 @@ class NamedTransformTest extends PipelineSpec { private def assertTransformName(p: PCollectionWrapper[_], tfName: String) = p.internal.getProducingTransformInternal.getFullName shouldBe tfName + + private def assertOuterTransformName(p: PCollectionWrapper[_], tfName: String) = + p.internal.getProducingTransformInternal.getFullName.split("/").head shouldBe tfName } From 9d8bde8e6499d8751b6678bc100401f401d7d526 Mon Sep 17 00:00:00 2001 From: Andrew Martin Date: Tue, 17 Jan 2017 14:44:21 -0500 Subject: [PATCH 09/11] Make tfName scio-private --- .../main/scala/com/spotify/scio/values/TransformNameable.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala b/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala index d5542b97dd..5cc9f54fcf 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/TransformNameable.scala @@ -22,7 +22,7 @@ import com.spotify.scio.util.CallSites trait TransformNameable { private var nameProvider: TransformNameProvider = CallSiteNameProvider - def tfName: String = { + private[scio] def tfName: String = { val n = nameProvider.name nameProvider = CallSiteNameProvider n From 49737c04786812c701a4cf9919c91359a285548e Mon Sep 17 00:00:00 2001 From: Andrew Martin Date: Tue, 17 Jan 2017 14:44:58 -0500 Subject: [PATCH 10/11] Better test helper function names. --- .../scio/values/NamedTransformTest.scala | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala b/scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala index 3bc769ce99..9b80ab42ad 100644 --- a/scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala +++ b/scio-test/src/test/scala/com/spotify/scio/values/NamedTransformTest.scala @@ -25,7 +25,7 @@ class NamedTransformTest extends PipelineSpec { "ScioContext" should "support custom transform name" in { runWithContext { sc => val p = sc.withName("ReadInput").parallelize(Seq("a", "b", "c")) - assertTransformName(p, "ReadInput/Read(InMemorySource)") + assertTransformNameEquals(p, "ReadInput/Read(InMemorySource)") } } @@ -34,7 +34,7 @@ class NamedTransformTest extends PipelineSpec { val p = sc.parallelize(Seq(1, 2, 3, 4, 5)) .map(_ * 3) .withName("OnlyEven").filter(_ % 2 == 0) - assertTransformName(p, "OnlyEven/Filter") + assertTransformNameEquals(p, "OnlyEven/Filter") } } @@ -42,7 +42,7 @@ class NamedTransformTest extends PipelineSpec { runWithContext { sc => val p = sc.parallelize(Seq(1.0, 2.0, 3.0, 4.0, 5.0)) .withName("CalcVariance").variance - assertOuterTransformName(p, "CalcVariance") + assertOuterTransformNameEquals(p, "CalcVariance") } } @@ -50,7 +50,7 @@ class NamedTransformTest extends PipelineSpec { runWithContext { sc => val p = sc.parallelize(Seq(("a", 1), ("b", 2), ("c", 3))) .withName("SumPerKey").sumByKey - assertTransformName(p, "SumPerKey/KvToTuple") + assertTransformNameEquals(p, "SumPerKey/KvToTuple") } } @@ -64,7 +64,7 @@ class NamedTransformTest extends PipelineSpec { c.addValue(intSum, n) n } - assertTransformName(p, "TripleSum") + assertTransformNameEquals(p, "TripleSum") } } @@ -72,7 +72,7 @@ class NamedTransformTest extends PipelineSpec { runWithContext { sc => val p = sc.parallelize(Seq(1, 2, 3)).withFanout(10) .withName("Sum").sum - assertTransformName(p, "Sum/Values/Values") + assertTransformNameEquals(p, "Sum/Values/Values") } } @@ -80,7 +80,7 @@ class NamedTransformTest extends PipelineSpec { runWithContext { sc => val p = sc.parallelize(Seq(("a", 1), ("b", 2), ("c", 3))).withHotKeyFanout(10) .withName("Sum").sumByKey - assertTransformName(p, "Sum/KvToTuple") + assertTransformNameEquals(p, "Sum/KvToTuple") } } @@ -90,7 +90,7 @@ class NamedTransformTest extends PipelineSpec { val p2 = sc.parallelize(Seq(1, 2, 3)).asListSideInput val s = p1.withSideInputs(p2) .withName("GetX").filter((x, s) => x == "a") - assertTransformName(s, "GetX") + assertTransformNameEquals(s, "GetX") } } @@ -100,8 +100,8 @@ class NamedTransformTest extends PipelineSpec { val p2 = SideOutput[String]() val (main, side) = p1.withSideOutputs(p2) .withName("MakeSideOutput").map { (x, s) => s.output(p2, x + "2"); x + "1" } - assertTransformName(main, "MakeSideOutput") - assertTransformName(side(p2), "MakeSideOutput") + assertTransformNameEquals(main, "MakeSideOutput") + assertTransformNameEquals(side(p2), "MakeSideOutput") } } @@ -110,7 +110,7 @@ class NamedTransformTest extends PipelineSpec { val p = sc.parallelize(Seq(1, 2, 3, 4, 5)) .toWindowed .withName("Triple").map(x => x.withValue(x.value * 3)) - assertTransformName(p, "Triple") + assertTransformNameEquals(p, "Triple") } } @@ -119,7 +119,7 @@ class NamedTransformTest extends PipelineSpec { val p1 = sc.parallelize(Seq(("a", 1), ("a", 2), ("b", 3), ("c", 4))) val p2 = sc.parallelize(Seq(("a", 11), ("b", 12), ("b", 13), ("d", 14))) val p = MultiJoin.withName("JoinEm").left(p1, p2) - assertTransformName(p, "JoinEm") + assertTransformNameEquals(p, "JoinEm") } } @@ -131,9 +131,9 @@ class NamedTransformTest extends PipelineSpec { .withName("MyTransform").map(_ * 3) val p3 = p1 .withName("MyTransform").map(_ * 4) - assertTransformName(p1, "MyTransform") - assertTransformName(p2, "MyTransform2") - assertTransformName(p3, "MyTransform3") + assertTransformNameEquals(p1, "MyTransform") + assertTransformNameEquals(p2, "MyTransform2") + assertTransformNameEquals(p3, "MyTransform3") } } @@ -145,9 +145,9 @@ class NamedTransformTest extends PipelineSpec { " as the name for the next transform." } - private def assertTransformName(p: PCollectionWrapper[_], tfName: String) = + private def assertTransformNameEquals(p: PCollectionWrapper[_], tfName: String) = p.internal.getProducingTransformInternal.getFullName shouldBe tfName - private def assertOuterTransformName(p: PCollectionWrapper[_], tfName: String) = + private def assertOuterTransformNameEquals(p: PCollectionWrapper[_], tfName: String) = p.internal.getProducingTransformInternal.getFullName.split("/").head shouldBe tfName } From 0d2fa9e71ecaabb90ad00ed07320dd67f93bc61f Mon Sep 17 00:00:00 2001 From: Andrew Martin Date: Tue, 17 Jan 2017 15:15:39 -0500 Subject: [PATCH 11/11] Fix formatting --- .../spotify/scio/values/DoubleSCollectionFunctions.scala | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/scio-core/src/main/scala/com/spotify/scio/values/DoubleSCollectionFunctions.scala b/scio-core/src/main/scala/com/spotify/scio/values/DoubleSCollectionFunctions.scala index 06f19c69a9..f34dea0644 100644 --- a/scio-core/src/main/scala/com/spotify/scio/values/DoubleSCollectionFunctions.scala +++ b/scio-core/src/main/scala/com/spotify/scio/values/DoubleSCollectionFunctions.scala @@ -48,17 +48,13 @@ class DoubleSCollectionFunctions(self: SCollection[Double]) { * Compute the sample standard deviation of this SCollection's elements (which corrects for bias * in estimating the standard deviation by dividing by N-1 instead of N). */ - def sampleStdev: SCollection[Double] = self.transform { - _.stats.map(_.sampleStdev) - } + def sampleStdev: SCollection[Double] = self.transform(_.stats.map(_.sampleStdev)) /** * Compute the sample variance of this SCollection's elements (which corrects for bias in * estimating the variance by dividing by N-1 instead of N). */ - def sampleVariance: SCollection[Double] = self.transform { - _.stats.map(_.sampleVariance) - } + def sampleVariance: SCollection[Double] = self.transform(_.stats.map(_.sampleVariance)) // Ported from org.apache.spark.rdd.DoubleRDDFunctions