├── .circleci ├── config.yml └── hasNotChanged ├── .floo ├── .flooignore ├── .gitignore ├── .java-version ├── .jvmopts ├── .scalafix.conf ├── .scalafmt.conf ├── LICENSE ├── README.md ├── bench └── src │ └── main │ └── scala │ └── io │ └── univalence │ └── sparktools │ └── bench │ └── ParkaBench.scala ├── build.sbt ├── centrifuge ├── QuickStartAutobuild.md ├── README.md ├── _config.yml └── src │ ├── main │ └── scala │ │ ├── Macros.scala │ │ ├── centrifuge │ │ └── sql │ │ │ ├── CoGroupX.scala │ │ │ ├── Explore.scala │ │ │ └── centrifuge_sql.scala │ │ ├── compare │ │ └── compare.scala │ │ ├── generics.scala │ │ └── io │ │ └── univalence │ │ └── centrifuge │ │ ├── Executor.scala │ │ ├── Model.scala │ │ ├── SparkLens.scala │ │ ├── SparkNarrow.scala │ │ ├── package.scala │ │ └── util │ │ └── TaskLimiter.scala │ └── test │ ├── resources │ └── log4j.properties │ ├── scala-2.11 │ └── MyUDF.scala │ ├── scala-2.12 │ └── MyUDF.scala │ └── scala │ ├── Applicative.sc │ ├── ApplicativePlusShapeless.sc │ ├── ExecutorTest.scala │ ├── Old.scala │ ├── TotoMachinApp.scala │ ├── compare │ ├── StringSpecification.scala │ ├── StringUtilsTest.scala │ └── compareStringsTest.scala │ ├── io │ └── univalence │ │ ├── DeltaQA.scala │ │ ├── IntroScala.scala │ │ ├── ModelScalaCheck.scala │ │ ├── ModelTest.scala │ │ ├── SparkLensTest.scala │ │ ├── SparkNarrowTest.scala │ │ ├── TestSparkVersion.scala │ │ ├── autobuild │ │ └── Quickstart.scala │ │ ├── catsContrib.scala │ │ └── centrifuge │ │ ├── ExecutorTest.scala │ │ └── sql │ │ ├── AnnotationTest.scala │ │ └── Explore.scala │ └── psug │ ├── Part0.scala │ ├── Part1.scala │ ├── Part2.scala │ ├── Part3.scala │ └── Part4.scala ├── fenek ├── README.md └── src │ ├── main │ └── scala │ │ └── io │ │ └── univalence │ │ └── fenek │ │ ├── Expr.scala │ │ ├── JsonInterpreter.scala │ │ ├── Query.scala │ │ ├── analysis │ │ └── Sources.scala │ │ ├── generic │ │ └── GenericExpr.scala │ │ └── package.scala │ └── test │ ├── resources │ └── log4j.properties │ └── scala │ └── io │ └── univalence │ └── fenek │ ├── DebugInterpreter.scala │ ├── DslTest.scala │ ├── JsonInterpreterTest.scala │ └── beta │ └── StaticAnalysis.scala ├── logo.png ├── parka ├── README.md └── src │ ├── main │ └── scala │ │ └── io │ │ └── univalence │ │ └── parka │ │ ├── CompressMap.scala │ │ ├── ConsoleSize.scala │ │ ├── Constraintor.scala │ │ ├── Histogram.scala │ │ ├── MonoidGen.scala │ │ ├── Parka.scala │ │ ├── Printer.scala │ │ ├── RowBasedMap.scala │ │ ├── Serde.scala │ │ └── Structure.scala │ └── test │ ├── ressources │ ├── leftTest.csv │ └── rightTest.csv │ └── scala │ └── io │ └── univalence │ └── parka │ ├── CompressMapTest.scala │ ├── ConstraintorTest.scala │ ├── DeltaTest.scala │ ├── DescribeTest.scala │ ├── HistogramTest.scala │ ├── KModes2Test.scala │ ├── ParkaTest.scala │ └── PrinterTest.scala ├── plumbus ├── README.md └── src │ ├── main │ └── scala │ │ └── io │ │ └── univalence │ │ └── plumbus │ │ ├── cc2map.scala │ │ ├── cogroup.scala │ │ ├── compress │ │ ├── CompressDump.scala │ │ └── GenerateSQL.scala │ │ ├── functions.scala │ │ ├── internal │ │ └── CleanFromRow.scala │ │ └── test │ │ └── SparkTestLike.scala │ └── test │ ├── resources │ └── log4j.properties │ └── scala │ └── io │ └── univalence │ └── plumbus │ ├── Cc2mapTest.scala │ ├── CogroupTest.scala │ ├── CompressDumpTest.scala │ └── functionsTest.scala ├── project ├── ScalaSettings.scala ├── build.properties └── plugins.sbt ├── scalafix ├── build.sbt ├── input │ └── src │ │ └── main │ │ └── scala │ │ └── fix │ │ └── Fenek2to3All.scala ├── output │ └── src │ │ └── main │ │ └── scala │ │ └── fix │ │ └── Fenek2to3All.scala ├── project │ ├── build.properties │ └── plugins.sbt ├── readme.md ├── rules │ └── src │ │ └── main │ │ ├── resources │ │ └── META-INF │ │ │ └── services │ │ │ └── scalafix.v1.Rule │ │ └── scala │ │ └── fix │ │ └── Fenek2to3.scala └── tests │ └── src │ └── test │ └── scala │ └── fix │ └── RuleSuite.scala ├── schema └── src │ ├── main │ └── scala │ │ └── io │ │ └── univalence │ │ └── schema │ │ └── Schema.scala │ └── test │ └── scala │ └── io │ └── univalence │ └── schema │ └── SchemaTest.scala ├── site ├── README.md ├── project │ └── build.properties └── src │ └── main │ ├── resources │ └── microsite │ │ ├── css │ │ └── custom.css │ │ ├── data │ │ └── menu.yml │ │ └── img │ │ ├── favicon.png │ │ ├── favicon16x16.png │ │ ├── favicon32x32.png │ │ ├── favicon64x64.png │ │ ├── jumbotron_pattern.png │ │ ├── navbar_brand.png │ │ ├── navbar_brand2x.png │ │ ├── sidebar_brand.png │ │ └── sidebar_brand2x.png │ └── tut │ ├── fenek │ └── index.md │ ├── index.md │ ├── parka │ ├── index.md │ ├── parka_analysis.md │ ├── parka_analysis_datasetinfo.md │ ├── parka_analysis_inner.md │ ├── parka_analysis_outer.md │ ├── parka_printer.md │ ├── parka_question.md │ ├── parka_serde.md │ ├── parka_structures.md │ ├── parka_structures_delta.md │ ├── parka_structures_describe.md │ ├── parka_structures_inner.md │ └── parka_structures_outer.md │ ├── plumbus │ └── index.md │ ├── spark-test │ ├── features.md │ ├── getting_started.md │ ├── index.md │ └── overview.md │ └── spark-zio │ ├── getting_started.md │ └── index.md ├── spark-test ├── README.md └── src │ ├── main │ └── scala │ │ ├── io │ │ └── univalence │ │ │ └── sparktest │ │ │ ├── Schema2CaseClass.scala │ │ │ ├── SparkTest.scala │ │ │ ├── SparkTestSQLImplicits.scala │ │ │ ├── SparkTestSession.scala │ │ │ ├── ValueComparison.scala │ │ │ └── internal │ │ │ └── DatasetUtils.scala │ │ └── org │ │ └── apache │ │ └── spark │ │ └── sql │ │ └── SparkTestExt.scala │ └── test │ ├── resources │ ├── jsonTest.json │ ├── jsonTest2.json │ └── log4j.properties │ └── scala │ └── io │ └── univalence │ └── sparktest │ ├── DataFrameComparisonTest.scala │ ├── DataSetComparisonTest.scala │ ├── GettingStartedTest.scala │ ├── RDDComparisonTest.scala │ ├── ReadFromJsonTest.scala │ ├── ValueComparisonTest.scala │ ├── bug │ ├── DataFrameEqualityBugSparkTraining.scala │ └── DataFrameNoCommonColumnsButIsEqual.scala │ └── featurematching │ ├── SparkFastTest.scala │ └── SparkTestingBase.scala ├── spark-zio ├── README.md └── src │ ├── main │ └── scala │ │ └── io │ │ └── univalence │ │ └── sparkzio │ │ ├── CircuitTap.scala │ │ ├── Iterator.scala │ │ └── SparkEnv.scala │ └── test │ ├── resources │ ├── log4j.properties │ └── toto │ │ └── toto.txt │ └── scala │ └── io │ └── univalence │ └── sparkzio │ ├── CircuitTapTest.scala │ ├── IteratorTest.scala │ ├── ProtoMapWithEffet.scala │ ├── SparkEnvImplicitClassTest.scala │ └── ZDynamicConcurrency.scala └── typedpath ├── README.md └── src ├── main └── scala │ └── io │ └── univalence │ ├── schema │ └── SchemaComparator.scala │ └── typedpath │ ├── Key.scala │ └── strings.scala └── test ├── resources └── log4j.properties └── scala └── io └── univalence ├── schema └── SchemaComparatorTest.scala └── typedpath ├── FenekSyntaxTest.scala ├── IndexTest.scala └── KeySpec.scala /.circleci/hasNotChanged: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | LAST_SUCCESSFUL_BUILD_URL="https://circleci.com/api/v1.1/project/github/$CIRCLE_PROJECT_USERNAME/$CIRCLE_PROJECT_REPONAME/tree/$CIRCLE_BRANCH?filter=successful&limit=40" 4 | 5 | echo "LAST_SUCCESSFUL_BUILD_URL: " $LAST_SUCCESSFUL_BUILD_URL 6 | 7 | LAST_SUCCESSFUL_COMMIT=`curl -Ss $LAST_SUCCESSFUL_BUILD_URL |jq -r ".[]|select(.build_parameters.CIRCLE_JOB == \"$CIRCLE_JOB\")|.[\"vcs_revision\"]" |head -n 1` 8 | echo "LAST_SUCCESSFUL_COMMIT: " $LAST_SUCCESSFUL_COMMIT 9 | 10 | 11 | for arg in "$@" 12 | do 13 | if [[ $(git diff $CIRCLE_SHA1..$LAST_SUCCESSFUL_COMMIT --name-status | grep "$arg") != "" ]] 14 | then 15 | exit 255 16 | fi 17 | done 18 | exit 0 -------------------------------------------------------------------------------- /.floo: -------------------------------------------------------------------------------- 1 | { 2 | "url": "https://floobits.com/ahoy-jon/spark-tools" 3 | } -------------------------------------------------------------------------------- /.flooignore: -------------------------------------------------------------------------------- 1 | extern 2 | node_modules 3 | tmp 4 | vendor 5 | .idea/workspace.xml 6 | .idea/misc.xml 7 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | .attach_pid* 2 | .DS_Store 3 | *.iml 4 | target/ 5 | .idea/ 6 | .metals/ 7 | .bloop/ 8 | *.log 9 | report/ 10 | -------------------------------------------------------------------------------- /.java-version: -------------------------------------------------------------------------------- 1 | 1.8 2 | -------------------------------------------------------------------------------- /.jvmopts: -------------------------------------------------------------------------------- 1 | -Xms512M 2 | -Xmx4096M 3 | -Xss2M 4 | -XX:MaxMetaspaceSize=1024M 5 | -------------------------------------------------------------------------------- /.scalafmt.conf: -------------------------------------------------------------------------------- 1 | maxColumn = 120 2 | align = more 3 | includeCurlyBraceInSelectChains = false 4 | danglingParentheses = true 5 | 6 | rewrite { 7 | rules = [ 8 | SortImports, 9 | PreferCurlyFors, 10 | RedundantBraces, 11 | SortModifiers] 12 | redundantBraces.stringInterpolation = true 13 | } 14 | 15 | continuationIndent { 16 | defnSite = 2 17 | } 18 | 19 | spaces { 20 | inImportCurlyBraces = true 21 | } 22 | 23 | optIn { 24 | annotationNewlines = true 25 | } 26 | 27 | align { 28 | tokens.add = ["at", "=>", "->", "<-", "%", "%%", ":=", "=", "<<-", "&&"] 29 | } 30 | 31 | rewriteTokens = { "⇒": "=>", "←" : "<-", "→": "->"} 32 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | ![Spark-tools Logo](./logo.png) 2 | 3 | [![CircleCI](https://circleci.com/gh/univalence/spark-tools.svg?style=svg)](https://circleci.com/gh/univalence/spark-tools) 4 | 5 | Spark-tools is a set of tools dedicated to Spark and aims to 6 | make the life of data engineers easier. 7 | 8 | ## The tools 9 | 10 | * [Spark-Test](https://github.com/univalence/spark-tools/tree/master/spark-test), testing tools for Dataframe and Dataset from Spark 11 | * [Parka](https://github.com/univalence/spark-tools/tree/master/parka), data quality on Spark 12 | * [Fenek](https://github.com/univalence/spark-tools/tree/master/fenek), a DSL for *semi-typed* transformation in Scala for Spark 13 | * [Plumbus](https://github.com/univalence/spark-tools/tree/master/plumbus), light misc things for Spark 14 | * [Spark-ZIO](https://github.com/univalence/spark-tools/tree/master/spark-zio), Spark in ZIO environment 15 | -------------------------------------------------------------------------------- /bench/src/main/scala/io/univalence/sparktools/bench/ParkaBench.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.sparktools.bench 2 | 3 | import cats.kernel.Monoid 4 | import io.univalence.parka.{ MonoidGen, RowBasedMap } 5 | import org.openjdk.jmh.annotations._ 6 | 7 | import scala.collection.mutable 8 | import scala.reflect.ClassTag 9 | 10 | object MapMonoidBench { 11 | 12 | @State(Scope.Benchmark) 13 | abstract class AbstractBenchmark(val maps: Maps, val monoid: Monoid[Map[String, Int]]) { 14 | 15 | @Benchmark 16 | def m1_and_m2(): Unit = 17 | monoid.combine(maps.m1, maps.m2) 18 | 19 | @Benchmark 20 | def times2(): Unit = 21 | monoid.combine(maps.m1, maps.m1) 22 | 23 | } 24 | 25 | trait Maps { 26 | val m1: Map[String, Int] 27 | val m2: Map[String, Int] 28 | } 29 | def text(string: String): Map[String, Int] = 30 | string.split(' ').groupBy(x => x).mapValues(_.length).toMap 31 | 32 | object map1 extends Maps { 33 | val m1: Map[String, Int] = text("abc abc def def ghi def jkl mno") 34 | val m2: Map[String, Int] = text("def def ghi def jkl mno mno") 35 | } 36 | 37 | object map2 extends Maps { 38 | val m1: RowBasedMap[String, Int] = RowBasedMap.toColbaseMap(map1.m1) 39 | val m2: RowBasedMap[String, Int] = RowBasedMap.toColbaseMap(map1.m2) 40 | } 41 | 42 | import com.twitter.algebird.Ring._ 43 | val monoid2: Monoid[Map[String, Int]] = MapMonoid.mapMonoid2 44 | val monoid1: Monoid[Map[String, Int]] = MapMonoid.mapMonoidInit 45 | val monoid3: Monoid[Map[String, Int]] = MapMonoid.mapMonoid3 46 | 47 | class Map1Monoid1 extends AbstractBenchmark(map1, monoid1) 48 | class Map1Monoid2 extends AbstractBenchmark(map1, monoid2) 49 | class Map2Monoid2 extends AbstractBenchmark(map2, monoid2) 50 | class Map2Monoid3 extends AbstractBenchmark(map2, monoid3) 51 | 52 | } 53 | 54 | object MapMonoid { 55 | 56 | def mapMonoidInit[K, V: Monoid]: Monoid[Map[K, V]] = 57 | MonoidGen( 58 | Map.empty, 59 | (m1, m2) => { 60 | (m1.keySet ++ m2.keySet) 61 | .map( 62 | k => 63 | k -> (m1.get(k) match { 64 | case None => m2.getOrElse(k, Monoid[V].empty) 65 | case Some(x) => 66 | m2.get(k) match { 67 | case Some(y) => Monoid[V].combine(x, y) 68 | case None => x 69 | } 70 | }) 71 | ) 72 | .toMap 73 | } 74 | ) 75 | 76 | def mapMonoid2[K, V: Monoid]: Monoid[Map[K, V]] = new Monoid[Map[K, V]] { 77 | @inline 78 | override def empty: Map[K, V] = Map.empty 79 | 80 | @inline 81 | override def combine(m1: Map[K, V], m2: Map[K, V]): Map[K, V] = { 82 | val m1k = m1.keySet 83 | m1.map({ 84 | case (k, v) => 85 | (k, m2.get(k).fold(v)(v2 => Monoid[V].combine(v, v2))) 86 | }) ++ m2.filterKeys(k => !m1k(k)) 87 | } 88 | } 89 | 90 | def mapMonoid3[K, V: Monoid]: Monoid[Map[K, V]] = new Monoid[Map[K, V]] { 91 | 92 | val valueMonoid: Monoid[V] = Monoid[V] 93 | 94 | val proxiedMonoid: Monoid[Map[K, V]] = mapMonoid2[K, V] 95 | 96 | override def empty: Map[K, V] = RowBasedMap.empty 97 | 98 | override def combine(x: Map[K, V], y: Map[K, V]): Map[K, V] = 99 | (x, y) match { 100 | case (cx: RowBasedMap[K, V], cy: RowBasedMap[K, V]) => cx.combine(cy) 101 | case _ => proxiedMonoid.combine(x, y) 102 | } 103 | } 104 | } 105 | 106 | object App { 107 | 108 | def main(args: Array[String]): Unit = 109 | new MapMonoidBench.Map2Monoid3().m1_and_m2() 110 | 111 | } 112 | -------------------------------------------------------------------------------- /centrifuge/QuickStartAutobuild.md: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | ```scala 5 | 6 | case class Hello(name: String, greet: Boolean) 7 | 8 | object Hello { 9 | @autoBuildResult 10 | def build:Result[Hello] = ??? 11 | } 12 | 13 | ```scala 14 | Error:(__, __) signature not matching to build Hello, use : 15 | @autoBuildResult 16 | def build(name : Result[String], 17 | greet : Result[Boolean]):Result[Hello] = MacroMarker.generated_applicative 18 | @autoBuildResult 19 | ``` -------------------------------------------------------------------------------- /centrifuge/_config.yml: -------------------------------------------------------------------------------- 1 | theme: jekyll-theme-cayman -------------------------------------------------------------------------------- /centrifuge/src/main/scala/centrifuge/sql/CoGroupX.scala: -------------------------------------------------------------------------------- 1 | package centrifuge.sql 2 | 3 | import org.apache.spark.rdd.CoGroupedRDD 4 | import org.apache.spark.rdd.RDD 5 | import org.apache.spark.sql.SparkSession 6 | import org.apache.spark.HashPartitioner 7 | import org.apache.spark.Partitioner 8 | import shapeless._ 9 | import shapeless.ops.hlist.Tupler 10 | import shapeless.ops.traversable.FromTraversable 11 | 12 | import scala.language.higherKinds 13 | import scala.reflect.ClassTag 14 | 15 | trait CheckCogroup[K, H <: HList] extends Serializable { 16 | type Res 17 | 18 | def toSeq(h: H): List[RDD[(K, _)]] 19 | } 20 | 21 | object CheckCogroup { 22 | type Aux[K, H <: HList, R] = CheckCogroup[K, H] { type Res = R } 23 | 24 | implicit def rFromNil[K]: CheckCogroup.Aux[K, HNil, HNil] = 25 | new CheckCogroup[K, HNil] { 26 | type Res = HNil 27 | 28 | override def toSeq(h: HNil): List[RDD[(K, _)]] = Nil 29 | } 30 | 31 | implicit def rCons[K, V, REST <: HList, RES <: HList]( 32 | implicit removeFirstT: CheckCogroup.Aux[K, REST, RES] 33 | ): CheckCogroup.Aux[K, RDD[(K, V)] :: REST, Seq[V] :: RES] = 34 | new CheckCogroup[K, RDD[(K, V)] :: REST] { 35 | type Res = Seq[V] :: RES 36 | 37 | //override def toSeq(h: ::[RDD[(K, V)], REST]): List[RDD[(K, _)]] = h.head.map(identity) :: removeFirstT.toSeq(h.tail) 38 | override def toSeq(h: ::[RDD[(K, V)], REST]): List[RDD[(K, _)]] = 39 | h.head.asInstanceOf[RDD[(K, _)]] :: removeFirstT.toSeq(h.tail) 40 | } 41 | } 42 | 43 | trait CoGroupN[K, H] extends Serializable { 44 | type Res <: (K, _) 45 | 46 | def arrayToRes(k: K, array: Array[Iterable[_]]): Res 47 | 48 | def toSeq(h: H): List[RDD[(K, _)]] 49 | } 50 | 51 | object CoGroupN { 52 | type Aux[K, H, R] = CoGroupN[K, H] { type Res = R } 53 | 54 | //FOR THE LULZ, comme si quelqu'un allait faire un group by comme ça ! 55 | //On peut supprimer ce cas, dans le cas d'un group by, le cogroupN reverra un RDD[(K,(Seq[V], Unit))] 56 | implicit def groupByCase[K, V]: CoGroupN.Aux[K, RDD[(K, V)] :: HNil, (K, Seq[V])] = 57 | new CoGroupN[K, RDD[(K, V)] :: HNil] { 58 | type Res = (K, Seq[V]) 59 | 60 | override def arrayToRes(k: K, array: Array[Iterable[_]]): (K, Seq[V]) = 61 | (k, array.head.toSeq.asInstanceOf[Seq[V]]) 62 | 63 | override def toSeq(h: ::[RDD[(K, V)], HNil]): List[RDD[(K, _)]] = 64 | List(h.head.asInstanceOf[RDD[(K, _)]]) 65 | } 66 | 67 | implicit def cogroupNByCase[K, H1 <: RDD[_], HRest <: HList, ABC <: HList, R <: Product]( 68 | implicit 69 | // verifie que c'est une séquence de RDD[(K,V)] :: RDD[(K,VV)] :: ... 70 | // et permet d'extraire le type ABC : Seq[V] :: Seq[VV] :: ... 71 | checkCoGroup: CheckCogroup.Aux[K, H1 :: HRest, ABC], 72 | // permet de passer de ABC au tuple R (Seq[V], Seq[VV], ... ) 73 | tupler: Tupler.Aux[ABC, R], 74 | // permet d'extraire ABC depuis un Array[Iterable[_]] 75 | fromTraversable: FromTraversable[ABC] 76 | ): CoGroupN.Aux[K, H1 :: HRest, (K, R)] = new CoGroupN[K, H1 :: HRest] { 77 | type Res = (K, R) 78 | 79 | override def arrayToRes(k: K, array: Array[Iterable[_]]): Res = 80 | (k, tupler.apply(fromTraversable(array.map(_.toSeq)).get)) 81 | 82 | override def toSeq(h: ::[H1, HRest]): List[RDD[(K, _)]] = 83 | checkCoGroup.toSeq(h) 84 | } 85 | 86 | /* 87 | Usage : cogroupN(rdd1 :: rdd2 :: rdd3 :: HNil) 88 | */ 89 | def cogroupN[K, H <: HList, Res](h: H, part: Partitioner = new HashPartitioner(1024))( 90 | implicit 91 | ctK: ClassTag[K], 92 | ctRes: ClassTag[Res], 93 | //ToTraversable.Aux[H,List,RDD[(K,_)]] don't work 94 | coGroupAble: CoGroupN.Aux[K, H, Res] 95 | ): RDD[Res] = 96 | new CoGroupedRDD(coGroupAble.toSeq(h), part) 97 | .map((coGroupAble.arrayToRes _).tupled) 98 | 99 | } 100 | 101 | object CoGroupX { 102 | 103 | def main(args: Array[String]): Unit = { 104 | val ss = 105 | SparkSession.builder().appName("toto").master("local[*]").getOrCreate() 106 | 107 | val d = ss.sparkContext.makeRDD(Seq("a" -> "b", "a" -> "c", "b" -> "d")) 108 | 109 | val f = d.mapValues(_ => 1) 110 | 111 | val n: RDD[(String, (Seq[String], Seq[String], Seq[String]))] = 112 | CoGroupN.cogroupN(d :: d :: d :: HNil) 113 | 114 | n.collect().foreach(println) 115 | 116 | val n1: RDD[(String, (Seq[String], Seq[String]))] = 117 | CoGroupN.cogroupN(d :: d :: HNil) 118 | 119 | val n2: RDD[(String, Seq[String])] = CoGroupN.cogroupN(d :: HNil) 120 | 121 | val n3: RDD[(String, (Seq[String], Seq[Int], Seq[Int], Seq[String]))] = 122 | CoGroupN.cogroupN(d :: f :: f :: d :: HNil) 123 | 124 | n3.collect.foreach(println) 125 | } 126 | 127 | } 128 | -------------------------------------------------------------------------------- /centrifuge/src/main/scala/compare/compare.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.utils 2 | 3 | object StringUtils { 4 | 5 | def compareStrings(a: String, b: String): Double = { 6 | def frequencies[T](s1: Seq[T]): Map[T, Int] = 7 | s1.groupBy(identity).mapValues(_.size) 8 | 9 | def joinMap[K, V](m1: Map[K, V], m2: Map[K, V]): Map[K, (Option[V], Option[V])] = 10 | //todo : optimize 11 | (m1.keySet ++ m2.keySet).toSeq.map(k => (k, (m1.get(k), m2.get(k)))).toMap 12 | 13 | // identical strings 14 | if (a == b) return 1 15 | 16 | val pairs1: Seq[String] = wordLetterPairs(a) 17 | val pairs2: Seq[String] = wordLetterPairs(b) 18 | 19 | val union = pairs1.length + pairs2.length 20 | 21 | val inter = 22 | joinMap(frequencies(pairs1), frequencies(pairs2)).values.map { 23 | case (Some(l), Some(r)) => Math.min(l, r) 24 | case _ => 0 25 | }.sum 26 | 27 | inter.toDouble * 2 / union 28 | } 29 | 30 | def letterPairs(str: String): Seq[String] = 31 | str.sliding(2, 1).toSeq 32 | 33 | def wordLetterPairs(str: String): Seq[String] = 34 | str.split("\\s").flatMap(letterPairs) 35 | 36 | } 37 | -------------------------------------------------------------------------------- /centrifuge/src/main/scala/generics.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.autobuild.struct 2 | 3 | import shapeless.labelled._ 4 | import shapeless._ 5 | 6 | import scala.collection.immutable 7 | import scala.language.higherKinds 8 | 9 | trait TypeName[T] { 10 | def name: String 11 | } 12 | 13 | object TypeName { 14 | import scala.reflect.runtime.universe.TypeTag 15 | 16 | implicit def fromTypeTag[T](implicit typeTag: TypeTag[T]): TypeName[T] = 17 | new TypeName[T] { 18 | override def name: String = typeTag.tpe.toString 19 | } 20 | } 21 | 22 | trait PathAwareness[T[_]] { 23 | def injectPrefix[A](prefix: String)(t: T[A]): T[A] 24 | } 25 | 26 | object DefaultPathAwareness { 27 | implicit def defaultPathAwareness[App[_]] = new PathAwareness[App] { 28 | override def injectPrefix[A](prefix: String)(t: App[A]): App[A] = t 29 | } 30 | } 31 | 32 | trait FieldsNonRecur[L] { 33 | def fieldnames: List[(String, String)] 34 | 35 | } 36 | 37 | trait LowPriorityFieldsNonRecur { 38 | implicit def caseClassFields[F, G]( 39 | implicit gen: LabelledGeneric.Aux[F, G], 40 | encode: Lazy[FieldsNonRecur[G]] 41 | ): FieldsNonRecur[F] = 42 | new FieldsNonRecur[F] { 43 | override def fieldnames: List[(String, String)] = encode.value.fieldnames 44 | } 45 | 46 | implicit def hcon[K <: Symbol, H, T <: HList]( 47 | implicit 48 | key: Witness.Aux[K], 49 | tv: TypeName[H], 50 | tailEncode: Lazy[FieldsNonRecur[T]] 51 | ): FieldsNonRecur[FieldType[K, H] :: T] = 52 | new FieldsNonRecur[FieldType[K, H] :: T] { 53 | override def fieldnames: List[(String, String)] = 54 | (key.value.name, tv.name) :: tailEncode.value.fieldnames 55 | } 56 | } 57 | 58 | object FieldsNonRecur extends LowPriorityFieldsNonRecur { 59 | implicit def hnil[L <: HNil]: FieldsNonRecur[L] = new FieldsNonRecur[L] { 60 | override def fieldnames: List[(String, String)] = Nil 61 | } 62 | 63 | def fieldnames[A](implicit tmr: FieldsNonRecur[A]): Seq[(String, String)] = 64 | tmr.fieldnames 65 | } 66 | -------------------------------------------------------------------------------- /centrifuge/src/main/scala/io/univalence/centrifuge/Executor.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.centrifuge 2 | 3 | import java.util.concurrent.TimeUnit 4 | import monix.eval.Task 5 | import monix.eval.TaskCircuitBreaker 6 | import org.apache.spark.sql.Dataset 7 | import org.apache.spark.sql.Encoder 8 | import scala.concurrent.Await 9 | import scala.concurrent.duration.Duration 10 | import scala.util.Failure 11 | import scala.util.Success 12 | import scala.util.Try 13 | 14 | case class ExecutionSummary(nbFailure: Long) 15 | 16 | object RetryDs { 17 | def retryDs[A, C, B](in: Dataset[A])(run: A => Try[C])(integrate: (A, Try[C]) => B)( 18 | nbGlobalAttemptMax: Int, 19 | circuitBreakerMaxFailure: Int = 10 20 | )( 21 | implicit 22 | encoderA: Encoder[A], 23 | encoderB: Encoder[B], 24 | encoderI: Encoder[(Option[A], LocalExecutionStatus, B)] 25 | ): (Dataset[B], ExecutionSummary) = { 26 | import monix.execution.Scheduler.Implicits.global 27 | Await.result( 28 | retryDsWithTask(in)(a => Task(run(a).get))(integrate)( 29 | nbGlobalAttemptMax = nbGlobalAttemptMax, 30 | circuitBreakerMaxFailure = Some(circuitBreakerMaxFailure) 31 | ).runAsync, 32 | Duration.Inf 33 | ) 34 | } 35 | 36 | private def toExecutionStat(les: LocalExecutionStatus): ExecutionStat = 37 | if (les) 0 else 1 38 | 39 | private def addExecutionState(es1: ExecutionStat, es2: ExecutionStat): ExecutionStat = es1 + es2 40 | 41 | private def initExecutionSummary(fes: ExecutionStat): ExecutionSummary = 42 | ExecutionSummary(fes) 43 | 44 | private def localExecutionFromError(e: Throwable): LocalExecutionStatus = 45 | false 46 | 47 | private val successExecution: LocalExecutionStatus = true 48 | 49 | private type LocalExecutionStatus = Boolean 50 | 51 | private type ExecutionStat = Long 52 | 53 | def retryDsWithTask[A, C, B](in: Dataset[A])(run: A => Task[C])(integrate: (A, Try[C]) => B)( 54 | nbGlobalAttemptMax: Int, 55 | circuitBreakerMaxFailure: Option[Int] = Option(10) 56 | )( 57 | implicit 58 | encoderA: Encoder[A], 59 | encoderB: Encoder[B], 60 | encoderI: Encoder[(Option[A], LocalExecutionStatus, B)] 61 | ): Task[(Dataset[B], ExecutionSummary)] = { 62 | 63 | type M = (Option[A], LocalExecutionStatus, B) 64 | 65 | def newCircuitBreaker: Option[TaskCircuitBreaker] = 66 | circuitBreakerMaxFailure.map(n => TaskCircuitBreaker(n, Duration(1, TimeUnit.HOURS))) 67 | 68 | def aToM(a: A, endo: Task[C] => Task[C]): M = { 69 | import monix.execution.Scheduler.Implicits.global 70 | val tried: Try[C] = Try(Await.result(endo(run(a)).runAsync, Duration.Inf)) 71 | 72 | tried match { 73 | case Failure(e) => 74 | (Some(a), localExecutionFromError(e), integrate(a, tried)) 75 | case Success(_) => (None, successExecution, integrate(a, tried)) 76 | } 77 | } 78 | 79 | def loopTheLoop( 80 | mAndEs: (Dataset[M], ExecutionSummary), 81 | attemptRemaining: Int 82 | ): Task[(Dataset[M], ExecutionSummary)] = 83 | if (mAndEs._2.nbFailure == 0 || attemptRemaining <= 0) 84 | Task.pure(mAndEs) 85 | else { 86 | Task { 87 | val (ds, _) = mAndEs 88 | val newDs: Dataset[(Option[A], LocalExecutionStatus, B)] = 89 | ds.mapPartitions(iterator => { 90 | val circuitBreaker = newCircuitBreaker 91 | iterator.map({ 92 | case (Some(a), _, _) => 93 | aToM(a, x => circuitBreaker.fold(x)(breaker => breaker.protect(x))) 94 | case x => x 95 | }) 96 | }) 97 | newDs.persist() 98 | val es: ExecutionSummary = dsToEs(newDs) 99 | (newDs, es) 100 | }.flatMap(x => loopTheLoop(x, attemptRemaining - 1)) 101 | } 102 | 103 | def dsToEs(ds: Dataset[M]): ExecutionSummary = { 104 | import ds.sparkSession.implicits._ 105 | initExecutionSummary(ds.map(_._2).rdd.map(toExecutionStat).reduce(addExecutionState)) 106 | } 107 | 108 | Task({ 109 | val init: Dataset[M] = { 110 | in.mapPartitions(iterator => { 111 | val circuitBreaker = newCircuitBreaker 112 | iterator.map(x => aToM(x, t => circuitBreaker.fold(t)(breaker => breaker.protect(t)))) 113 | }) 114 | } 115 | 116 | init.persist() 117 | (init, dsToEs(init)) 118 | }).flatMap(x => loopTheLoop(x, nbGlobalAttemptMax - 1)) 119 | .map({ case (a, b) => (a.map(_._3), b) }) 120 | 121 | } 122 | } 123 | -------------------------------------------------------------------------------- /centrifuge/src/main/scala/io/univalence/centrifuge/Model.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.centrifuge 2 | 3 | import scala.util.Failure 4 | import scala.util.Success 5 | import scala.util.Try 6 | 7 | case class Result[+T]( 8 | value: Option[T], 9 | annotations: Vector[Annotation] 10 | ) { 11 | 12 | def addPathPart(s: String): Result[T] = 13 | mapAnnotations(x => x.copy(onField = x.onField.map(s + _).orElse(Some(s)))) 14 | 15 | def mapAnnotations(f: Annotation => Annotation): Result[T] = 16 | Result(value, annotations.map(f)) 17 | 18 | def prependAnnotations(xs: Vector[AnnotationSql]): Result[T] = 19 | Result(value, xs ++ annotations) 20 | 21 | def isEmpty: Boolean = value.isEmpty 22 | 23 | def isPure: Boolean = value.isDefined && annotations.isEmpty 24 | 25 | def hasAnnotations: Boolean = annotations.nonEmpty 26 | 27 | def map[U](f: T => U): Result[U] = Result(value.map(f), annotations) 28 | 29 | def map2[U, V](result: Result[U])(f: (T, U) => V): Result[V] = 30 | Result((value, result.value) match { 31 | case (Some(t), Some(u)) => Some(f(t, u)) 32 | case _ => None 33 | }, annotations ++ result.annotations) 34 | 35 | def flatMap[U](f: T => Result[U]): Result[U] = 36 | value match { 37 | case None => this.asInstanceOf[Result[U]] 38 | case Some(v) => 39 | val r = f(v) 40 | r.copy(annotations = annotations ++ r.annotations) 41 | } 42 | 43 | def filter(f: T => Boolean): Result[T] = Result(value.filter(f), annotations) 44 | 45 | def get: T = 46 | value.getOrElse(throw new Exception("empty result : " + annotations.mkString("|"))) 47 | 48 | def toTry: Try[T] = Try(get) 49 | 50 | def toEither: Either[Vector[Annotation], T] = 51 | value.fold(Left(annotations).asInstanceOf[Either[Vector[Annotation], T]])(Right(_)) 52 | } 53 | 54 | object Result { 55 | 56 | def fromTry[T](tr: Try[T])(expToString: Throwable => String): Result[T] = 57 | tr match { 58 | case Success(t) => pure(t) 59 | case Failure(e) => fromError(expToString(e)) 60 | } 61 | 62 | def fromEither[L, R](either: Either[L, R])(leftToString: L => String): Result[R] = 63 | either.fold(leftToString.andThen(fromError), pure) 64 | 65 | def pure[T](t: T): Result[T] = 66 | Result( 67 | value = Some(t), 68 | annotations = Vector.empty 69 | ) 70 | 71 | def fromError(error: String): Result[Nothing] = 72 | Result( 73 | value = None, 74 | annotations = Vector(Annotation.fromString(msg = error, error = true)) 75 | ) 76 | 77 | def fromWarning[T](t: T, warning: String): Result[T] = 78 | Result( 79 | value = Some(t), 80 | annotations = Vector(Annotation.fromString(msg = warning, error = false)) 81 | ) 82 | } 83 | 84 | case class Annotation( 85 | message: String, 86 | onField: Option[String] = None, 87 | fromFields: Vector[String] = Vector.empty, 88 | isError: Boolean, 89 | count: Long = 1L 90 | ) { 91 | 92 | def this(message: String, onField: Option[String], fromFields: Seq[String], isError: Boolean, count: Long) = { 93 | this(message, onField, fromFields.toVector, isError, count) 94 | } 95 | } 96 | 97 | object Annotation { 98 | def fromString( 99 | msg: String, 100 | error: Boolean 101 | ): Annotation = 102 | Annotation( 103 | message = msg, 104 | isError = error, 105 | count = 1L 106 | ) 107 | 108 | def missingField(fieldName: String): Annotation = 109 | Annotation(message = "MISSING_VALUE", onField = Some(fieldName), isError = true) 110 | } 111 | -------------------------------------------------------------------------------- /centrifuge/src/main/scala/io/univalence/centrifuge/SparkLens.scala: -------------------------------------------------------------------------------- 1 | package io.univalence 2 | 3 | import org.apache.spark.rdd.RDD 4 | import org.apache.spark.sql.DataFrame 5 | import org.apache.spark.sql.Row 6 | import org.apache.spark.sql.catalyst.expressions.GenericRow 7 | import org.apache.spark.sql.types._ 8 | 9 | import scala.util.matching.Regex 10 | 11 | sealed trait Prefix 12 | case object PrefixArray extends Prefix 13 | case class PrefixName(name: String) extends Prefix 14 | 15 | object SparkLens { 16 | 17 | /*implicit class Regex(sc: StringContext) { 18 | def r = new util.matching.Regex(sc.parts.mkString, sc.parts.tail.map(_ => "x"): _*) 19 | }*/ 20 | 21 | type Path = Seq[Prefix] 22 | 23 | def pathToStr(path: Path): String = 24 | path 25 | .map({ 26 | case PrefixName(name) => name 27 | case PrefixArray => "[]" 28 | }) 29 | .mkString("/") 30 | 31 | def lensRegExp( 32 | df: DataFrame 33 | )(fieldSelect: (String, DataType) => Boolean, transform: (Any, DataType) => Any): DataFrame = 34 | lens(df)({ case (p, dt) => fieldSelect(pathToStr(p), dt) }, transform) 35 | 36 | type Jump = Seq[Option[Int]] 37 | 38 | def lens(df: DataFrame)(fieldSelect: (Path, DataType) => Boolean, transform: (Any, DataType) => Any): DataFrame = { 39 | 40 | val schema = df.schema 41 | 42 | def matchJump(prefix: Jump = Vector.empty, path: Path = Nil, dataType: DataType): Seq[(Jump, DataType)] = { 43 | 44 | val first: Option[(Jump, DataType)] = 45 | if (fieldSelect(path, dataType)) Some(prefix -> dataType) else None 46 | 47 | val recur: Seq[(Jump, DataType)] = dataType match { 48 | case StructType(fields) => 49 | fields.zipWithIndex.flatMap({ 50 | case (s, i) => 51 | val j = prefix :+ Some(i) 52 | val newPath = path :+ PrefixName(s.name) 53 | 54 | matchJump(j, newPath, s.dataType) 55 | }) 56 | 57 | case ArrayType(dt, _) => 58 | val j = prefix :+ None 59 | matchJump(j, path :+ PrefixArray, dt) 60 | case _ => Vector.empty 61 | } 62 | first.toSeq ++ recur 63 | } 64 | 65 | val toTx: Seq[(Jump, DataType)] = 66 | matchJump(Vector.empty, Vector.empty, schema) 67 | 68 | val res: RDD[Row] = df.rdd.map { gen => 69 | toTx.foldLeft(gen)({ 70 | case (r, (j, dt)) => 71 | update(j, r, a => transform(a, dt)).asInstanceOf[Row] 72 | }) 73 | } 74 | 75 | df.sparkSession.createDataFrame(res, schema) 76 | 77 | } 78 | 79 | private def update(j: Jump, r: Any, f: Any => Any): Any = 80 | j.toList match { 81 | case Nil => f(r) 82 | case x :: xs if r == null => null 83 | case None :: xs => r.asInstanceOf[Seq[Any]].map(x => update(xs, x, f)) 84 | case Some(i) :: xs => 85 | val row = r.asInstanceOf[Row] 86 | val s = row.toSeq 87 | new GenericRow(s.updated(i, update(xs, s(i), f)).toArray) 88 | } 89 | 90 | } 91 | 92 | /* //TODO Use a tree 93 | sealed trait JumpTree 94 | 95 | case class Root(childs: Seq[BranchTree]) extends JumpTree 96 | case class BranchTree(pos: Option[Int], childs: Seq[BranchTree]) extends JumpTree 97 | */ 98 | -------------------------------------------------------------------------------- /centrifuge/src/main/scala/io/univalence/centrifuge/SparkNarrow.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.centrifuge 2 | 3 | import org.apache.spark.sql.types.DataType 4 | import org.apache.spark.sql.types.StructType 5 | 6 | sealed trait SType { 7 | def typeName: String 8 | } 9 | 10 | case class SOption(arg: SType) extends SType { 11 | override def typeName: String = s"Option[${arg.typeName}]" 12 | } 13 | case class SClass(name: String) extends SType { 14 | override def typeName: String = name 15 | } 16 | 17 | case class SCC(names: Seq[String], args: Seq[(String, SType)]) extends SType { 18 | 19 | def classDef: String = 20 | s"case class ${names.last}(${args.map({ case (n, t) => s"$n:${t.typeName}" }).mkString(",")} )" 21 | 22 | override def typeName: String = names.mkString(".") 23 | } 24 | 25 | object Sparknarrow { 26 | 27 | def dataTypeToTypeName(dataType: DataType): String = 28 | dataType.simpleString.capitalize match { 29 | case "Date" => "java.sql.Date" 30 | case "Int" => "scala.Int" 31 | case x => s"java.lang.$x" 32 | } 33 | 34 | def basicCC(schema: StructType, pck: Option[String] = None, name: String = "_Cc"): SCC = 35 | SCC( 36 | names = pck.toSeq ++ List(name), 37 | schema.map(strucField => { 38 | strucField.name -> SOption(SClass(dataTypeToTypeName(strucField.dataType))) 39 | }) 40 | ) 41 | 42 | } 43 | -------------------------------------------------------------------------------- /centrifuge/src/main/scala/io/univalence/centrifuge/package.scala: -------------------------------------------------------------------------------- 1 | package io.univalence 2 | 3 | import org.apache.spark.sql.Dataset 4 | import org.apache.spark.sql.SparkSession 5 | import org.apache.spark.sql.centrifuge_sql._ 6 | 7 | package object centrifuge { 8 | 9 | type AnnotationSql = Annotation 10 | 11 | object AnnotationSql { 12 | 13 | def apply( 14 | msg: String, 15 | onField: String, 16 | fromFields: Vector[String], 17 | isError: Boolean, 18 | count: Long 19 | ): Annotation = Annotation( 20 | message = msg, 21 | isError = isError, 22 | count = count, 23 | onField = Some(onField), 24 | fromFields = fromFields 25 | ) 26 | 27 | } 28 | 29 | object implicits { 30 | implicit def QADFOps[T](dataframe: Dataset[T]): QADF = 31 | new QADF(dataframe.toDF()) 32 | implicit def sparkSessionOps(ss: SparkSession): QATools = new QATools(ss) 33 | } 34 | 35 | } 36 | -------------------------------------------------------------------------------- /centrifuge/src/main/scala/io/univalence/centrifuge/util/TaskLimiter.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.centrifuge.util 2 | 3 | //from https://gist.github.com/alexandru/623fe6c587d73e89a8f14de284ca1e2d 4 | 5 | import monix.eval.Task 6 | import java.util.concurrent.TimeUnit 7 | import scala.concurrent.duration._ 8 | 9 | /** Request limiter for APIs that have quotas per second, minute, hour, etc. 10 | * 11 | * {{{ 12 | * // Rate-limits to 100 requests per second 13 | * val limiter = TaskLimiter(TimeUnit.SECONDS, limit = 100) 14 | * 15 | * limiter.request(myTask) 16 | * }}} 17 | */ 18 | final class TaskLimiter(period: TimeUnit, limit: Int) { 19 | import monix.execution.atomic.Atomic 20 | import TaskLimiter.State 21 | 22 | @transient private[this] val state = Atomic(State(0, period, 0, limit)) 23 | 24 | def request[A](task: Task[A]): Task[A] = 25 | Task.deferAction { ec => 26 | val now = ec.currentTimeMillis() 27 | state.transformAndExtract(_.request(now)) match { 28 | case None => task 29 | case Some(delay) => 30 | // Recursive call, retrying request after delay 31 | request(task).delayExecution(delay) 32 | } 33 | } 34 | } 35 | 36 | object TaskLimiter { 37 | 38 | /** Builder for [[TaskLimiter]]. */ 39 | def apply(period: TimeUnit, limit: Int): TaskLimiter = 40 | new TaskLimiter(period, limit) 41 | 42 | /** Timestamp specified in milliseconds since epoch, 43 | * as returned by `System.currentTimeMillis` 44 | */ 45 | type Timestamp = Long 46 | 47 | /** Internal state of [[TaskLimiter]]. */ 48 | final case class State(window: Long, period: TimeUnit, requested: Int, limit: Int) { 49 | private def periodMillis = 50 | TimeUnit.MILLISECONDS.convert(1, period) 51 | 52 | def request(now: Timestamp): (Option[FiniteDuration], State) = { 53 | val periodMillis = this.periodMillis 54 | val currentWindow = now / periodMillis 55 | 56 | if (currentWindow != window) 57 | (None, copy(window = currentWindow, requested = 1)) 58 | else if (requested < limit) 59 | (None, copy(requested = requested + 1)) 60 | else { 61 | val nextTS = (currentWindow + 1) * periodMillis 62 | val sleep = nextTS - now 63 | (Some(sleep.millis), this) 64 | } 65 | } 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /centrifuge/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | log4j.rootCategory=WARN, console 2 | log4j.appender.console=org.apache.log4j.ConsoleAppender 3 | log4j.appender.console.target=System.err 4 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 5 | log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n -------------------------------------------------------------------------------- /centrifuge/src/test/scala-2.11/MyUDF.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql 2 | 3 | import org.apache.spark.sql.catalyst.expressions.Expression 4 | import org.apache.spark.sql.catalyst.expressions.Literal 5 | import org.apache.spark.sql.types.LongType 6 | import org.apache.spark.sql.types.TimestampType 7 | 8 | object MyUDF { 9 | 10 | private def myTimestampCast(xs: Seq[Expression]): Expression = { 11 | val expSource = xs.head 12 | expSource.dataType match { 13 | case LongType => 14 | new Column(expSource).divide(Literal(1000)).cast(TimestampType).expr 15 | case TimestampType => 16 | /* WARNING */ 17 | expSource 18 | } 19 | } 20 | 21 | def register(sparkSession: SparkSession): Unit = 22 | sparkSession.sessionState.functionRegistry 23 | .registerFunction("toTs", myTimestampCast) 24 | 25 | } 26 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala-2.12/MyUDF.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql 2 | 3 | import org.apache.spark.sql.catalyst.FunctionIdentifier 4 | import org.apache.spark.sql.catalyst.expressions.Expression 5 | import org.apache.spark.sql.catalyst.expressions.Literal 6 | import org.apache.spark.sql.types.LongType 7 | import org.apache.spark.sql.types.TimestampType 8 | 9 | object MyUDF { 10 | 11 | private def myTimestampCast(xs: Seq[Expression]): Expression = { 12 | val expSource = xs.head 13 | expSource.dataType match { 14 | case LongType => 15 | new Column(expSource).divide(Literal(1000)).cast(TimestampType).expr 16 | case TimestampType => 17 | /* WARNING */ 18 | expSource 19 | } 20 | } 21 | 22 | def register(sparkSession: SparkSession): Unit = 23 | sparkSession.sessionState.functionRegistry 24 | .registerFunction(FunctionIdentifier("toTs",None), myTimestampCast) 25 | 26 | } 27 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/Applicative.sc: -------------------------------------------------------------------------------- 1 | import io.univalence.autobuild._ 2 | import scala.language.higherKinds 3 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/ApplicativePlusShapeless.sc: -------------------------------------------------------------------------------- 1 | import shapeless._ 2 | import shapeless.labelled._ 3 | import shapeless.syntax.singleton 4 | import shapeless.syntax.singleton._ 5 | 6 | import scala.language.higherKinds 7 | 8 | trait Applicative[M[_]] { 9 | def point[A](a: A): M[A] 10 | def ap[A, B](m1: M[A])(m2: M[A ⇒ B]): M[B] 11 | //derived 12 | def map[A, B](m1: M[A])(f: A ⇒ B): M[B] = ap(m1)(point(f)) 13 | def apply[A, B, C](m1: M[A], m2: M[B])(f: (A, B) ⇒ C): M[C] = 14 | ap(m2)(map(m1)(f.curried)) 15 | 16 | } 17 | 18 | case class Ahoy(name: String, y: Int) 19 | 20 | val oName: Option[String] = Option("hello") // None 21 | val oY: Option[String] = None 22 | 23 | object f extends Poly1 { 24 | implicit def kv[K, V, A[_]](implicit app: Applicative[A]): Case.Aux[ 25 | FieldType[K, A[V]], 26 | A[FieldType[K, V]] 27 | ] = 28 | at(a ⇒ app.map(a.asInstanceOf[A[V]])(field[K](_))) 29 | } 30 | 31 | val a = 'name ->> Option("hello") :: 'y ->> Option(1) :: HNil 32 | val b = Option('name ->> "hello") :: Option('y ->> 1) :: HNil 33 | def assertTypedEquals[A](expected: A, actual: A): Unit = 34 | assert(expected == actual) 35 | val kName = Witness.`'name` 36 | val kY = Witness.`'y` 37 | type Res = 38 | Option[FieldType[kName.T, String]] :: Option[FieldType[kY.T, Int]] :: HNil 39 | implicit val optionAppInstance: Applicative[Option] = new Applicative[Option] { 40 | override def point[A](a: A): Option[A] = Option(a) 41 | 42 | override def ap[A, B](m1: Option[A])(m2: Option[(A) ⇒ B]): Option[B] = 43 | m2.flatMap(f ⇒ m1.map(f)) 44 | } 45 | 46 | a.map(f) 47 | trait Apply2[FH, OutT] { 48 | type Out 49 | def apply(fh: FH, outT: OutT): Out 50 | } 51 | 52 | object Apply2 { 53 | type Aux[FH, OutT, Out0] = Apply2[FH, OutT] { type Out = Out0 } 54 | 55 | implicit def apply2[F[_], H, T <: HList]( 56 | implicit 57 | app: Applicative[F]): Aux[F[H], F[T], F[H :: T]] = 58 | new Apply2[F[H], F[T]] { 59 | type Out = F[H :: T] 60 | 61 | def apply(fh: F[H], outT: F[T]): Out = app.apply(fh, outT) { 62 | _ :: _ 63 | } 64 | } 65 | } 66 | 67 | trait Sequencer[L <: HList] { 68 | type Out 69 | def apply(in: L): Out 70 | } 71 | 72 | object Sequencer { 73 | type Aux[L <: HList, Out0] = Sequencer[L] { type Out = Out0 } 74 | 75 | implicit def consSequencerAux[FH, FT <: HList, OutT](implicit 76 | st: Aux[FT, OutT], 77 | ap: Apply2[FH, OutT]): Aux[FH :: FT, ap.Out] = 78 | new Sequencer[FH :: FT] { 79 | type Out = ap.Out 80 | def apply(in: FH :: FT): Out = 81 | ap(in.head, st(in.tail)) // un.TC.apply2(un(in.head), st(in.tail)) { _ :: _ } 82 | } 83 | 84 | implicit def nilSequencerAux[F[_]: Applicative]: Aux[HNil, F[HNil]] = 85 | new Sequencer[HNil] { 86 | type Out = F[HNil] 87 | def apply(in: HNil): F[HNil] = the[Applicative[F]].point(HNil: HNil) 88 | } 89 | } 90 | 91 | val sequenced = the[Sequencer[Res]].apply(b) 92 | val generic = the[LabelledGeneric[Ahoy]] 93 | 94 | the[Applicative[Option]].map(sequenced)(generic.from) 95 | 96 | def zip[A[_], X, Y](x: A[X])(y: A[Y])(implicit app: Applicative[A]) = 97 | app.apply(x, y)((x, y) ⇒ (x, y)) 98 | 99 | zip(Option("A"))(zip(Option("B"))(zip(Option("C"))(Some(null)))) 100 | 101 | /* 102 | def build[A[_]](name:A[String],y:A[Int])(implicit app:Applicative[A]) = { 103 | 104 | val mapped = ('name ->> name :: 'y ->> y :: HNil).map(f) 105 | 106 | app.map(the[Sequencer[mapped.type]].apply(mapped))(generic.from) 107 | } 108 | 109 | 110 | build(Option("A"),Option(2)) 111 | */ 112 | 113 | //assertTypedEquals[Res](b,a.map(f)) 114 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/ExecutorTest.scala: -------------------------------------------------------------------------------- 1 | 2 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/Old.scala: -------------------------------------------------------------------------------- 1 | import io.univalence.centrifuge.Result 2 | 3 | case class Ahoy(name: String, i: Int, l: String) 4 | 5 | object testBuilder { 6 | 7 | //@autoBuildResult 8 | def build( 9 | name: Result[String], 10 | i: Result[Int], 11 | l: Result[String] 12 | ): Result[Ahoy] = { 13 | 14 | val _1 = name.addPathPart("name") 15 | val _2 = i.addPathPart("i") 16 | val _3 = l.addPathPart("l") 17 | 18 | Result( 19 | value = (_1.value, _2.value, _3.value) match { 20 | case (Some(s1), Some(s2), Some(s3)) => 21 | Some(Ahoy(name = s1, i = s2, l = s3)) 22 | case _ => None 23 | }, 24 | annotations = _1.annotations ++ _2.annotations ++ _3.annotations 25 | ) 26 | } 27 | 28 | } 29 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/TotoMachinApp.scala: -------------------------------------------------------------------------------- 1 | package totopackage 2 | 3 | object TotoMachinApp { 4 | def main(args: Array[String]): Unit = { 5 | //println(("Hello") 6 | 7 | } 8 | } 9 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/compare/StringSpecification.scala: -------------------------------------------------------------------------------- 1 | package compare 2 | 3 | import io.univalence.utils.StringUtils 4 | import org.scalacheck.Gen 5 | import org.scalacheck.Properties 6 | import org.scalacheck.Prop._ 7 | 8 | object StringSpecification extends Properties("StringUtils") { 9 | 10 | private val asciiLetter: Gen[Char] = 11 | Gen.oneOf((0 to 127).map(_.toChar).filter(_.isLetterOrDigit)) 12 | 13 | private val asciiLetterString: Gen[String] = 14 | Gen.listOf(asciiLetter).map(_.mkString) 15 | 16 | def isAsciiLetter(c: Char): Boolean = c.isLetter && c <= 127 17 | 18 | property("letterPairs") = forAll(asciiLetterString) { a: String => 19 | (a.size > 1) ==> 20 | (StringUtils 21 | .letterPairs(a) 22 | .map(_.head) 23 | .mkString == a.dropRight(1)) 24 | } 25 | 26 | property("compareStrings should be 1 for identical strings") = forAll { a: String => 27 | StringUtils.compareStrings(a, a) == 1 28 | } 29 | 30 | property("compareStrings") = forAll { (a: String, b: String) => 31 | val result = StringUtils.compareStrings(a, b) 32 | 33 | (a != b) ==> (result < 1 && result >= 0) 34 | } 35 | 36 | } 37 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/compare/StringUtilsTest.scala: -------------------------------------------------------------------------------- 1 | package compare 2 | 3 | import io.univalence.utils.StringUtils 4 | import org.scalatest.FunSuite 5 | 6 | class StringUtilsTest extends FunSuite { 7 | 8 | test("letterPairs") { 9 | assert(StringUtils.letterPairs("abc").toSeq == Seq("ab", "bc")) 10 | } 11 | 12 | } 13 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/compare/compareStringsTest.scala: -------------------------------------------------------------------------------- 1 | 2 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/io/univalence/DeltaQA.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.deltaqa.kpialgebra 2 | 3 | import org.apache.spark.rdd.RDD 4 | import org.apache.spark.SparkConf 5 | import org.apache.spark.SparkContext 6 | import shapeless.contrib.spire._ 7 | import spire.algebra._ 8 | import spire.implicits._ 9 | 10 | import scala.reflect.ClassTag 11 | 12 | case class DeltaPart[T: AdditiveMonoid]( 13 | count: Long, 14 | part: T 15 | ) 16 | 17 | case class DeltaCommon[T: AdditiveMonoid]( 18 | count: Long, 19 | countZero: Long, 20 | diff: T, 21 | error: T, 22 | left: T, 23 | right: T 24 | ) 25 | 26 | case class Delta[L: AdditiveMonoid, R: AdditiveMonoid, C: AdditiveMonoid]( 27 | left: DeltaPart[L], 28 | right: DeltaPart[R], 29 | common: DeltaCommon[C] 30 | ) 31 | 32 | object KpiAlgebra { 33 | 34 | def computeCommon[LRC: AdditiveAbGroup: MultiplicativeSemigroup](left: LRC, right: LRC): DeltaCommon[LRC] = { 35 | val diff = left - right 36 | val error = diff * diff 37 | DeltaCommon( 38 | count = 1, 39 | countZero = if (diff == Monoid.additive[LRC].id) 1 else 0, 40 | diff = diff, 41 | error = error, 42 | left = left, 43 | right = right 44 | ) 45 | } 46 | 47 | def monoid[LM: AdditiveMonoid, RM: AdditiveMonoid, LRC: AdditiveMonoid]: Monoid[Delta[LM, RM, LRC]] = 48 | Monoid.additive[Delta[LM, RM, LRC]] 49 | 50 | def compare[ 51 | K: ClassTag, 52 | L: ClassTag, 53 | R: ClassTag, 54 | LM: AdditiveMonoid: ClassTag, 55 | RM: AdditiveMonoid: ClassTag, 56 | LRC: AdditiveAbGroup: MultiplicativeSemigroup: ClassTag 57 | ]( 58 | left: RDD[(K, L)], 59 | right: RDD[(K, R)] 60 | )(flm: L => LM, frm: R => RM, flc: L => LRC, frc: R => LRC): Delta[LM, RM, LRC] = { 61 | 62 | val map: RDD[Delta[LM, RM, LRC]] = left 63 | .fullOuterJoin(right) 64 | .map({ 65 | case (_, (Some(l), None)) => 66 | monoid[LM, RM, LRC].id 67 | .copy(left = DeltaPart(count = 1, part = flm(l))) 68 | case (_, (None, Some(r))) => 69 | monoid[LM, RM, LRC].id 70 | .copy(right = DeltaPart(count = 1, part = frm(r))) 71 | case (_, (Some(l), Some(r))) => 72 | monoid[LM, RM, LRC].id.copy(common = computeCommon(flc(l), frc(r))) 73 | }) 74 | 75 | map.reduce((x, y) => monoid[LM, RM, LRC].op(x, y)) 76 | } 77 | } 78 | 79 | case class KpiLeaf(l1: Long, l2: Long, l3: Long) 80 | 81 | object KpiAlgebraTest { 82 | 83 | def main(args: Array[String]) { 84 | val sc = new SparkContext(new SparkConf().setMaster("local[*]").setAppName("smoketest")) 85 | 86 | val parallelize: RDD[(Int, Int)] = sc.parallelize((1 to 4).zipWithIndex) 87 | 88 | /*println((( 89 | KpiAlgebra.compare( 90 | left = parallelize, 91 | right = parallelize 92 | )( 93 | flm = identity, 94 | frm = identity, 95 | flc = identity, 96 | frc = identity 97 | ) 98 | )*/ 99 | 100 | // Delta(DeltaPart(0,0),DeltaPart(0,0),DeltaCommon(4,4,0,0,6,6)) 101 | 102 | val p2: RDD[(Int, KpiLeaf)] = 103 | sc.parallelize((1 to 4)).map(_ -> KpiLeaf(1, 2, 3)) 104 | 105 | import spire.implicits._ 106 | import shapeless.contrib.spire._ 107 | 108 | ////println(((KpiAlgebra.compare(p2, p2)(identity, identity, identity, identity)) 109 | 110 | } 111 | } 112 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/io/univalence/ModelScalaCheck.scala: -------------------------------------------------------------------------------- 1 | package io.univalence 2 | 3 | import io.univalence.centrifuge.Result 4 | import io.univalence.centrifuge._ 5 | import org.scalacheck.Prop.forAll 6 | import org.scalacheck.Prop.all 7 | import org.scalacheck.Properties 8 | 9 | object ModelScalaCheck extends Properties("String") { 10 | 11 | property("isNotPure") = forAll { a: String => 12 | all( 13 | !Result(Some(a), Vector(Annotation("msg", Some("oF"), Vector("fF"), false, 1))).isPure, 14 | !Result(None, Vector(Annotation(a, Some("oF"), Vector("fF"), false, 1))).isPure, 15 | !Result(Some(a), Vector(Annotation(a, Some("oF"), Vector("fF"), false, 1))).isPure, 16 | !Result(Some("msg"), Vector(Annotation(a, Some("oF"), Vector("fF"), false, 1))).isPure 17 | ) 18 | } 19 | property("isPure") = forAll { a: String => 20 | Result(Some(a), Vector()).isPure 21 | } 22 | property("filter") = forAll { a: String => 23 | Result(Some(a), Vector(Annotation(a, Some("oF"), Vector("fF"), false, 1))) 24 | .filter(_.contains(a)) == Result( 25 | Some(a), 26 | Vector(Annotation(a, Some("oF"), Vector("fF"), false, 1)) 27 | ) 28 | } 29 | property("map") = forAll { a: String => 30 | all( 31 | Result(Some(a), Vector(Annotation("msg", Some("oF"), Vector("fF"), false, 1))) 32 | .map(_.toString) == Result( 33 | Some(a), 34 | Vector(Annotation("msg", Some("oF"), Vector("fF"), false, 1)) 35 | ), 36 | Result(Some(a), Vector(Annotation(a, Some("oF"), Vector("fF"), false, 1))) 37 | .map(_.toString) == Result( 38 | Some(a), 39 | Vector(Annotation(a, Some("oF"), Vector("fF"), false, 1)) 40 | ), 41 | Result(Some("msg"), Vector(Annotation(a, Some("oF"), Vector("fF"), false, 1))) == Result( 42 | Some("msg"), 43 | Vector(Annotation(a, Some("oF"), Vector("fF"), false, 1)) 44 | ) 45 | ) 46 | } 47 | 48 | } 49 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/io/univalence/SparkLensTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence 2 | 3 | import org.apache.spark.SparkConf 4 | import org.apache.spark.sql.DataFrame 5 | import org.apache.spark.sql.SparkSession 6 | import org.apache.spark.sql.types.ArrayType 7 | import org.apache.spark.sql.types.StringType 8 | import io.univalence.SparkLens._ 9 | import org.scalatest.FunSuite 10 | 11 | case class Toto(name: String, age: Int) 12 | 13 | case class Tata(toto: Toto) 14 | 15 | class SparkLensTest extends FunSuite { 16 | 17 | val conf: SparkConf = new SparkConf() 18 | conf.setAppName("yo") 19 | conf.setMaster("local[*]") 20 | 21 | implicit val ss: SparkSession = SparkSession.builder.config(conf).getOrCreate 22 | 23 | import ss.implicits._ 24 | 25 | test("testLensRegExp change string") { 26 | assert(lensRegExp(ss.createDataFrame(Seq(Toto("a", 1))))({ 27 | case ("name", StringType) => true 28 | case _ => false 29 | }, { case (a: String, d) => a.toUpperCase }).as[Toto].first() == Toto("A", 1)) 30 | } 31 | 32 | test("change Int") { 33 | assert(lensRegExp(ss.createDataFrame(Seq(Tata(Toto("a", 1)))))({ 34 | case ("toto/age", _) => true 35 | case _ => false 36 | }, { case (a: Int, d) => a + 1 }).as[Tata].first() == Tata(Toto("a", 2))) 37 | } 38 | 39 | ignore("null to nil") { 40 | 41 | val df: DataFrame = ss.read.parquet("/home/phong/daily_gpp_20180705") 42 | 43 | val yoho: DataFrame = lensRegExp(df)({ 44 | case (_, ArrayType(_, _)) => true 45 | case _ => false 46 | }, (a, b) => if (a == null) Nil else a) 47 | 48 | } 49 | 50 | } 51 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/io/univalence/SparkNarrowTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence 2 | 3 | import java.net.URLClassLoader 4 | import java.sql.Date 5 | 6 | import io.univalence.centrifuge.Sparknarrow 7 | import org.apache.spark.SparkConf 8 | import org.apache.spark.sql.types._ 9 | import org.apache.spark.sql.Encoders 10 | import org.apache.spark.sql.SparkSession 11 | import org.scalatest.FunSuite 12 | 13 | case class Person(name: String, age: Int, date: Date) 14 | 15 | class SparknarrowTest extends FunSuite { 16 | 17 | val conf: SparkConf = new SparkConf() 18 | conf.setAppName("yo") 19 | conf.set("spark.sql.caseSensitive", "true") 20 | conf.setMaster("local[2]") 21 | 22 | implicit val ss: SparkSession = SparkSession.builder.config(conf).getOrCreate 23 | import ss.implicits._ 24 | 25 | test("testBasicCC") { 26 | 27 | val classDef = Sparknarrow.basicCC(Encoders.product[Person].schema).classDef 28 | checkDefinition(classDef) 29 | 30 | } 31 | 32 | def checkDefinition(scalaCode: String): Unit = { 33 | //TODO do a version for 2.11 and 2.12 34 | /* 35 | import scala.tools.reflect.ToolBox 36 | import scala.reflect.runtime.{ universe => ru } 37 | import ru._ 38 | 39 | val cl = this.getClass.getClassLoader 40 | 41 | val tb = ru.runtimeMirror(cl).mkToolBox() 42 | 43 | val parsed = tb.parse(scalaCode) 44 | tb.compile(parsed)() 45 | */ 46 | } 47 | 48 | test("play with scala eval") { 49 | 50 | val code = 51 | """ 52 | case class Tata(str: String) 53 | case class Toto(age: Int, tata: Tata) 54 | """ 55 | 56 | checkDefinition(code) 57 | checkDefinition(code) 58 | 59 | } 60 | 61 | ignore("printSchema StructType") { 62 | val yo = StructType( 63 | Seq( 64 | StructField("name", StringType), 65 | StructField("tel", ArrayType(StringType)) 66 | ) 67 | ) 68 | 69 | yo.printTreeString() 70 | } 71 | 72 | } 73 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/io/univalence/TestSparkVersion.scala: -------------------------------------------------------------------------------- 1 | package io.univalence 2 | 3 | import org.apache.spark.SparkContext 4 | import org.scalatest.FunSuite 5 | 6 | class TestSparkVersion extends FunSuite { 7 | 8 | test("version") { 9 | assert(SparkContext.getClass.getPackage.getImplementationVersion == "2.1.1") 10 | } 11 | 12 | } 13 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/io/univalence/autobuild/Quickstart.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.autobuild 2 | 3 | import io.univalence.centrifuge.Annotation 4 | import io.univalence.centrifuge.Result 5 | import org.scalatest.FunSuite 6 | 7 | class Quickstart extends FunSuite { 8 | 9 | test("quickstart autobuild") { 10 | 11 | case class Hello(name: String, greet: Boolean) 12 | 13 | //Should not compile 14 | /* 15 | object Hello { 16 | @autoBuildResult 17 | def build:Result[Hello] = ??? 18 | } 19 | */ 20 | 21 | val name = "Edgar Allan Poe" 22 | val pureStrResult = Result.pure(name) 23 | val pureBoolResult = Result.pure(true) 24 | 25 | val errorAnnotation = 26 | Annotation.fromString(msg = "errorAnnotation", error = true) 27 | val regularAnnotation = 28 | Annotation.fromString(msg = "regularAnnotation", error = false) 29 | val errorStrResult = Result(None, Vector(errorAnnotation)) 30 | val errorBoolResult = Result(None, Vector(errorAnnotation)) 31 | val regularStrResult = 32 | Result(Some("regularStrResult"), Vector(regularAnnotation)) 33 | val regularBoolResult = Result(Some(true), Vector(regularAnnotation)) 34 | 35 | object Hello { 36 | @autoBuildResult 37 | def build( 38 | name: Result[String], 39 | greet: Result[Boolean] 40 | ): Result[Hello] = MacroMarker.generated_applicative 41 | 42 | } 43 | 44 | // pure pure is pure 45 | assert(Hello.build(pureStrResult, pureBoolResult).isPure) 46 | 47 | // 48 | val errorBuild = Hello.build(errorStrResult, errorBoolResult) 49 | 50 | assert(errorBuild.annotations.size == 4) 51 | assert(errorBuild.isEmpty) 52 | assert(!errorBuild.isPure) 53 | assert(errorBuild.value == None) 54 | 55 | val warnBuild = Hello.build(regularStrResult, regularBoolResult) 56 | 57 | assert(warnBuild.annotations.size == 2) 58 | assert(!warnBuild.isEmpty) 59 | assert(!warnBuild.isPure) 60 | assert(warnBuild.value == Some(Hello("regularStrResult", true))) 61 | 62 | /*assert(warnBuild == Result(Some(Hello("regularStrResult", true)), 63 | Vector(Annotation("regularAnnotation", Some("nameregularAnnotation"), Vector("regularAnnotation"), false, 1), 64 | Annotation("regularAnnotation", Some("greetregularAnnotation"), Vector("regularAnnotation"), false, 1))))*/ 65 | 66 | //print(Hello.build(errorStrResult, pureBoolResult)) 67 | } 68 | 69 | } 70 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/io/univalence/catsContrib.scala: -------------------------------------------------------------------------------- 1 | package io.univalence 2 | 3 | import cats.Eq 4 | import cats.Monad 5 | import io.univalence.centrifuge.Result 6 | 7 | import scala.annotation.tailrec 8 | 9 | object CatsContrib { 10 | 11 | implicit val resultMonad: Monad[Result] = new Monad[Result] { 12 | 13 | override def pure[A](x: A): Result[A] = Result.pure(x) 14 | 15 | @tailrec 16 | override def tailRecM[A, B](a: A)(f: (A) => Result[Either[A, B]]): Result[B] = 17 | f(a) match { 18 | case r if r.isEmpty => r.asInstanceOf[Result[B]] 19 | case r @ Result(None, xs) => r.asInstanceOf[Result[B]] 20 | case Result(Some(Left(a2)), xs) => 21 | tailRecM(a2)(f.andThen(x => x.prependAnnotations(xs))) 22 | case r @ Result(Some(Right(b)), xs) => Result(Some(b), xs) 23 | } 24 | 25 | override def flatMap[A, B](fa: Result[A])(f: (A) => Result[B]): Result[B] = 26 | fa.flatMap(f) 27 | } 28 | 29 | implicit def equResul[T]: Eq[Result[T]] = Eq.fromUniversalEquals 30 | 31 | } 32 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/io/univalence/centrifuge/ExecutorTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.centrifuge 2 | 3 | import java.util.concurrent.TimeUnit 4 | 5 | import io.univalence.Toto 6 | import monix.eval.Task 7 | import org.apache.spark.SparkConf 8 | import org.apache.spark.sql.SparkSession 9 | import org.scalatest.FunSuite 10 | 11 | import scala.concurrent.Await 12 | import scala.concurrent.duration.Duration 13 | import scala.util.Failure 14 | import scala.util.Success 15 | import scala.util.Try 16 | 17 | class ExecutorTest extends FunSuite { 18 | 19 | val conf: SparkConf = new SparkConf() 20 | conf.setAppName("yo") 21 | conf.set("spark.sql.caseSensitive", "true") 22 | conf.setMaster("local[2]") 23 | 24 | val ss: SparkSession = SparkSession.builder.config(conf).getOrCreate 25 | 26 | import ss.implicits._ 27 | 28 | //TODO too slow 29 | ignore("spark") { 30 | val totoes: Seq[Toto] = Seq(Toto("a", 1), Toto("b", 2)) 31 | val ds = ss.createDataset(totoes) 32 | 33 | assert( 34 | RetryDs 35 | .retryDs(ds)(run = x => Try(1))({ case (a, Success(1)) => a })(nbGlobalAttemptMax = 1000) 36 | ._1 37 | .collect() 38 | .toSeq == totoes 39 | ) 40 | 41 | val startDate: Long = System.currentTimeMillis() 42 | 43 | val wait = 5000 44 | 45 | val (resDs, ex) = RetryDs.retryDs(ds)( 46 | run = x => 47 | if (System.currentTimeMillis() - startDate > wait) Try(2 -> 1) 48 | else Failure(new Exception("too soon")) 49 | )({ case (a, _) => a })(1000) 50 | 51 | assert(resDs.collect().toSeq == totoes) 52 | assert(ex.nbFailure == 0) 53 | 54 | assert(System.currentTimeMillis() - startDate > wait) 55 | } 56 | 57 | ignore("implementation should work on pure function") { 58 | val (ds, _) = 59 | RetryDs.retryDs(ss.createDataset(Seq(1, 2, 3)))(x => Try(x + 1))({ 60 | case (a, Success(c)) => c 61 | })(1) 62 | 63 | assert(ds.collect().toList == List(2, 3, 4)) 64 | } 65 | 66 | ignore("circuit breaker should the stop calling the task for current partition after 10 executions in error") { 67 | 68 | val ds = ss.createDataset(1 to 19).coalesce(1) 69 | 70 | RetryDs.retryDs(ds)(CircuitBreakerMutable.f)({ case (a, _) => a })(1000, circuitBreakerMaxFailure = 10) 71 | 72 | ////println(((CircuitBreakerMutable.calls) 73 | 74 | val res = (1 to 10) ++ (1 to 10) ++ (11 to 19) ++ (11 to 19) 75 | 76 | assert(CircuitBreakerMutable.calls == res) 77 | 78 | } 79 | 80 | ignore("should work with a timeout") { 81 | 82 | val ds = ss.createDataset(List(1, 2, 5, 10, 100, 200, 500, 1000)) 83 | 84 | val t = 85 | RetryDs.retryDsWithTask(ds)(x => Task(Thread.sleep(x)).timeout(Duration(10, TimeUnit.MILLISECONDS)))( 86 | (x, y) => (x, y.isSuccess) 87 | )(2, None) 88 | 89 | import monix.execution.Scheduler.Implicits.global 90 | val res = Await.result(t.runAsync, Duration.Inf)._1.collect().toList 91 | 92 | assert(res.forall({ 93 | case (a, true) => a <= 10 94 | case (a, false) => a > 10 95 | })) 96 | 97 | } 98 | 99 | } 100 | 101 | object CircuitBreakerMutable { 102 | 103 | @transient var calls: Seq[Int] = Vector.empty 104 | 105 | @transient var shouldPassNext: Set[Int] = Set.empty 106 | 107 | def f(i: Int): Try[Int] = { 108 | 109 | calls = calls :+ i 110 | val res = if (shouldPassNext(i)) { 111 | Try(i) 112 | } else { 113 | shouldPassNext = shouldPassNext + i 114 | Failure(new Exception("not available")) 115 | } 116 | res 117 | } 118 | 119 | } 120 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/io/univalence/centrifuge/sql/Explore.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.centrifuge.sql 2 | 3 | import io.univalence.centrifuge.Result 4 | import org.apache.spark.sql.SparkSession 5 | import org.apache.spark.sql.expressions.UserDefinedFunction 6 | 7 | object Explore { 8 | 9 | def to_age(i: Int): Result[Int] = {} match { 10 | case _ if i < 0 => Result.fromError("BELOW_ZERO") 11 | case _ if i <= 13 => Result.fromWarning(i, "UNDER_13") 12 | case _ if i >= 130 => Result.fromError("OVER_130") 13 | case _ => Result.pure(i) 14 | } 15 | 16 | def non_empty_string(str: String): Result[String] = 17 | str match { 18 | //case None => Result.fromError("NULL_VALUE") 19 | case "" => Result.fromError("EMPTY_STRING") 20 | case _ => Result.pure(str) 21 | } 22 | 23 | def main(args: Array[String]): Unit = { 24 | 25 | import io.univalence.centrifuge.implicits._ 26 | 27 | val ss = 28 | SparkSession.builder().appName("test").master("local[*]").getOrCreate() 29 | 30 | ss.registerTransformation("to_age", to_age) 31 | ss.registerTransformation("non_empty_string", non_empty_string) 32 | 33 | import ss.sqlContext.implicits._ 34 | 35 | ss.sparkContext 36 | .makeRDD(Seq(Person("Joe", 12), Person("Joseph", 50), Person("", -1))) 37 | .toDS() 38 | .createTempView("person") 39 | 40 | val df = ss 41 | .sql("select non_empty_string(name) as person_name, to_age(age) as person_age from person") 42 | .includeAnnotations 43 | 44 | //println((df.schema.toString()) 45 | 46 | System.exit(0) 47 | 48 | df.toJSON.collect() //.foreach(//println() 49 | 50 | //println((df.queryExecution.toString()) 51 | 52 | df.printSchema() 53 | 54 | df.includeSources 55 | 56 | System.exit(0) 57 | //println((df.queryExecution.analyzed.toJSON) 58 | 59 | val df2 = df 60 | 61 | //df2.show(false) 62 | 63 | df2.printSchema() 64 | df2.createTempView("toto") 65 | 66 | //BLOG IDEA, AUTOMATICALY FLATTEN STRUCTURE IN SPARK 67 | ss.sql( 68 | "select col.msg, col.isError, col.count, age,name from (select explode(annotations) as col, age, name from toto) t" 69 | ) 70 | //.show(false) 71 | 72 | ss.sql("select non_empty_string(name) as person_name from person").includeAnnotations 73 | //.show(false) 74 | } 75 | 76 | } 77 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/psug/Part0.scala: -------------------------------------------------------------------------------- 1 | package psug.part0 2 | 3 | import scala.language.higherKinds 4 | 5 | class Part0 { 6 | 7 | /** HELLO PSUG !!! 8 | * 9 | * MY NAME IS AHOY-JON 10 | */ 11 | } 12 | 13 | trait Functor[F[_]] { 14 | def map[A, B](fa: F[A])(f: A => B): F[B] 15 | } 16 | 17 | trait Applicative[M[_]] extends Functor[M] { 18 | def point[A](a: A): M[A] 19 | 20 | def zip[A, B](ma: M[A], mb: M[B]): M[(A, B)] 21 | 22 | def ap[A, B](ma: M[A], mb: M[A => B]): M[B] = 23 | map(zip(ma, mb))(t => t._2(t._1)) 24 | 25 | def apply2[A, B, C](ma: M[A], mb: M[B])(f: (A, B) => C): M[C] = 26 | ap(mb, map(ma)(x => (b: B) => f(x, b))) 27 | } 28 | 29 | trait Monads[M[_]] extends Applicative[M] { 30 | def flatMap[A, B](ma: M[A])(f: A => M[B]): M[B] 31 | } 32 | 33 | case class IronSuit[T](value: T) { 34 | def map[B](f: T => B): IronSuit[B] = IronSuit(f(value)) 35 | 36 | def flatMap[B](f: T => IronSuit[B]): IronSuit[B] = f(value) 37 | 38 | } 39 | 40 | object Functor { 41 | 42 | implicit val ironSuitInstance: Monads[IronSuit] = new Monads[IronSuit] { 43 | override def flatMap[A, B](ma: IronSuit[A])(f: (A) => IronSuit[B]): IronSuit[B] = f(ma.value) 44 | 45 | override def point[A](a: A): IronSuit[A] = IronSuit(a) 46 | 47 | override def map[A, B](fa: IronSuit[A])(f: (A) => B): IronSuit[B] = 48 | IronSuit(f(fa.value)) 49 | 50 | override def ap[A, B](ma: IronSuit[A], mb: IronSuit[(A) => B]): IronSuit[B] = 51 | point(mb.value(ma.value)) 52 | 53 | override def zip[A, B](ma: IronSuit[A], mb: IronSuit[B]): IronSuit[(A, B)] = 54 | ??? 55 | } 56 | 57 | implicit class monadOps[M[_], A](fa: M[A])(implicit monad: Monads[M]) { 58 | def map[B](f: A => B) = monad.map(fa)(f) 59 | 60 | def flatMap[B](f: A => M[B]): M[B] = monad.flatMap(fa)(f) 61 | 62 | def apply2[B, C](mb: M[B])(f: (A, B) => C): M[C] = monad.apply2(fa, mb)(f) 63 | } 64 | 65 | } 66 | 67 | object Test2 { 68 | def main(args: Array[String]) { 69 | 70 | import Functor._ 71 | 72 | //println((for (x ← IronSuit(1); y ← IronSuit("a")) yield (x + y)) 73 | 74 | //println((IronSuit(1).apply2(IronSuit("a"))(_ + _)) 75 | 76 | //println((Option(1).map(_ + 1)) 77 | 78 | //println((Option(-1).flatMap(x => if (x < 0) None else Some("abc"))) 79 | 80 | } 81 | } 82 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/psug/Part1.scala: -------------------------------------------------------------------------------- 1 | package psug 2 | 3 | trait Applicative[M[_]] { 4 | def point[A](a: A): M[A] 5 | 6 | def ap[A, B](m1: M[A])(m2: M[A => B]): M[B] 7 | 8 | // def flatMap[A,B](m1:M[A])(f:A => M[B]):M[B] 9 | 10 | //derived 11 | def map[A, B](m1: M[A])(f: A => B): M[B] = ap(m1)(point(f)) 12 | 13 | def apply2[A, B, C](m1: M[A], m2: M[B])(f: (A, B) => C): M[C] = 14 | ap(m2)(map(m1)(f.curried)) 15 | 16 | def zip[A, B](m1: M[A], m2: M[B]): M[(A, B)] = { 17 | def tpl(x: A): B => (A, B) = (x, _) 18 | 19 | ap(m2)(map(m1)(tpl)) 20 | } 21 | } 22 | 23 | trait ApplicativeAlternative[M[_]] { 24 | def point[A](a: A): M[A] 25 | 26 | def zip[A, B](m1: M[A], m2: M[B]): M[(A, B)] 27 | 28 | def map[A, B](m1: M[A])(f: A => B): M[B] 29 | 30 | // def flatMap[A,B](m1:M[A])(f:A => M[B]):M[B] 31 | //derived 32 | def ap[A, B](m1: M[A])(m2: M[A => B]): M[B] = 33 | map(zip(m1, m2))(t => t._2(t._1)) 34 | 35 | def apply2[A, B, C](m1: M[A], m2: M[B])(f: (A, B) => C): M[C] = 36 | map(zip(m1, m2))(f.tupled) 37 | } 38 | 39 | object IronSuit { 40 | implicit val applicativeInst: Applicative[IronSuit] = 41 | new Applicative[IronSuit] { 42 | override def point[A](a: A): IronSuit[A] = IronSuit(a) 43 | 44 | override def ap[A, B](m1: IronSuit[A])(m2: IronSuit[(A) => B]): IronSuit[B] = IronSuit(m2.value(m1.value)) 45 | } 46 | implicit val applicativeAltInst: ApplicativeAlternative[IronSuit] = 47 | new ApplicativeAlternative[IronSuit] { 48 | override def point[A](a: A): IronSuit[A] = IronSuit(a) 49 | 50 | override def zip[A, B](m1: IronSuit[A], m2: IronSuit[B]): IronSuit[(A, B)] = 51 | IronSuit((m1.value, m2.value)) 52 | 53 | override def map[A, B](m1: IronSuit[A])(f: (A) => B): IronSuit[B] = 54 | IronSuit(f(m1.value)) 55 | } 56 | } 57 | 58 | case class IronSuit[T](value: T) 59 | 60 | object Test { 61 | 62 | val is1 = IronSuit(1) 63 | val isA = IronSuit("A") 64 | 65 | def main(args: Array[String]) { 66 | 67 | implicitly[Applicative[IronSuit]].apply2(is1, isA)(_ + _) 68 | 69 | implicitly[ApplicativeAlternative[IronSuit]].apply2(is1, isA)(_ + _) 70 | 71 | } 72 | } 73 | 74 | case class MyValidation[T](nominal: Option[T], err: Option[String]) 75 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/psug/Part2.scala: -------------------------------------------------------------------------------- 1 | package psug.part2 2 | 3 | import scala.language.higherKinds 4 | import scalaz.Applicative 5 | 6 | case class Hello(name: String, i: Int) 7 | 8 | object Hello { 9 | 10 | def build[A[_]: Applicative](name: A[String], i: A[Int]): A[Hello] = { 11 | import scalaz.syntax.applicative._ 12 | (name |@| i)(Hello.apply) 13 | 14 | } 15 | 16 | def main(args: Array[String]) { 17 | import scalaz.std.option._ 18 | 19 | //println((build(name = Option("ahoy"), i = Option(-1))) 20 | 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/psug/Part3.scala: -------------------------------------------------------------------------------- 1 | package psug.shapeless 2 | 3 | //import shapeless.contrib.scalaz.Sequencer 4 | import shapeless.labelled._ 5 | import shapeless.ops.hlist.Mapper 6 | import shapeless.syntax.singleton._ 7 | import shapeless._ 8 | 9 | import scala.language.higherKinds 10 | import scalaz._ 11 | import scalaz.std.option._ 12 | 13 | /* 14 | object Builder { 15 | 16 | /* 17 | example de construction applicative présentée au PSUG avec scalaz et shapeless 18 | */ 19 | 20 | object transFormField extends Poly1 { 21 | implicit def kv[K, V, A[_]](implicit app: Applicative[A]): Case.Aux[FieldType[K, A[V]], A[FieldType[K, V]]] = 22 | at(a => app.map(a.asInstanceOf[A[V]])(field[K](_))) 23 | } 24 | 25 | def build0[In <: HList, Out <: HList](in: In)(implicit map: Mapper.Aux[transFormField.type, In, Out]): Out = 26 | map.apply(in) 27 | 28 | def build1[In <: HList, Out](in: In)(implicit sequencer: Sequencer.Aux[In, Out]): Out = sequencer.apply(in) 29 | 30 | def build2[A[_], In, Out](in: A[In])(implicit f: Functor[A], gen: LabelledGeneric.Aux[Out, In]): A[Out] = 31 | f.map(in)(gen.from) 32 | 33 | def build[In <: HList, In1 <: HList, Out1, A[_], In2, Out2](in: In)(implicit 34 | map: Mapper.Aux[transFormField.type, In, In1], 35 | sequencer: Sequencer.Aux[In1, Out1], 36 | un: Unpack1[Out1, A, In2], 37 | f: Functor[A], 38 | gen: LabelledGeneric.Aux[Out2, In2]): A[Out2] = 39 | build2(build1(build0(in)).asInstanceOf[A[In2]]) 40 | 41 | } 42 | 43 | case class Yolo(i: Int) 44 | 45 | object Test { 46 | 47 | type namek = Witness.`'name`.T 48 | 49 | def assertTypedEquals[A](expected: A): Unit = () 50 | 51 | val ex = 'name ->> Option("hello") 52 | assertTypedEquals[FieldType[namek, Option[String]] :: HNil](ex :: HNil) 53 | 54 | assertTypedEquals[Option[FieldType[namek, String]] :: HNil]( 55 | Builder.transFormField.apply('name ->> Option("hello")) :: HNil 56 | ) 57 | 58 | case class Ahoy(name: String, y: Int) 59 | 60 | import Builder._ 61 | 62 | val o1: Option[Ahoy] = build2(build1(build0('name ->> Option("hello") :: 'y ->> Option(1) :: HNil))) 63 | 64 | val o2: Option[Ahoy] = build('name ->> Option("hello") :: 'y ->> Option(1) :: HNil) 65 | 66 | val y: Option[Yolo] = build('i ->> Option(1) :: HNil) 67 | 68 | def main(args: Array[String]) { 69 | //println((y) 70 | } 71 | 72 | } 73 | */ 74 | -------------------------------------------------------------------------------- /centrifuge/src/test/scala/psug/Part4.scala: -------------------------------------------------------------------------------- 1 | /*package psug.part4 2 | 3 | import io.univalence.autobuild.autoBuildResult 4 | 5 | 6 | case class HelloHello(name: String, i: Int) 7 | 8 | 9 | case class Annotation(position: String, error: String) { 10 | def addPathPath(s: String): Annotation = this.copy(position = s + position) 11 | } 12 | 13 | 14 | case class Result[T](nominal: Option[T], annotations: List[Annotation]) { 15 | def addPathPart(s: String): Result[T] = this.copy(annotations = annotations.map(_.addPathPath(s))) 16 | } 17 | 18 | object Result { 19 | 20 | 21 | def point[A](a:A) = Result(nominal = Some(a), Nil) 22 | 23 | implicit class ROps[A](ra: Result[A]) { 24 | def app[B](f: Result[A => B]): Result[B] = ??? 25 | 26 | def map[B](f: A => B): Result[B] = ??? 27 | 28 | } 29 | 30 | } 31 | 32 | 33 | object HelloHello { 34 | 35 | 36 | @autoBuildResult 37 | def build(name: Result[String], 38 | i: Result[Int]): Result[HelloHello] = { 39 | 40 | val _1 = name.addPathPart("name") 41 | val _2 = i.addPathPart("i") 42 | 43 | Result(nominal = (_1.nominal, _2.nominal) match { 44 | case (Some(s1), Some(s2)) => Some(HelloHello(name = s1, i = s2)) 45 | case _ => None 46 | }, annotations = _1.annotations ::: _2.annotations) 47 | } 48 | } 49 | 50 | object TestTest { 51 | 52 | def main(args: Array[String]) { 53 | 54 | import io.univalence.excelsius._ 55 | HelloHello.build(Result(None, Nil), Result(Some(1), List(Annotation("", "c'est pas bon")))).excel() 56 | } 57 | }*/ 58 | -------------------------------------------------------------------------------- /fenek/README.md: -------------------------------------------------------------------------------- 1 | Fenek 2 | ====================== 3 | 4 | [ ![Download](https://api.bintray.com/packages/univalence/univalence-jvm/fenek/images/download.svg) ](https://bintray.com/univalence/univalence-jvm/fenek/_latestVersion) 5 | 6 | ## Installation 7 | 8 | ### Stable version 9 | 10 | Version for scala 2.11.X : 11 | 12 | ```scala 13 | libraryDependencies += "io.univalence" % "fenek_2.11" % "733d698b" 14 | ``` 15 | 16 | ### Latest version 17 | 18 | If you want to get the very last version of this library you can still download it using bintray here : https://bintray.com/univalence/univalence-jvm/fenek 19 | 20 | Here is an example using ```version 0.3+79-4936e981``` that work with ```scala 2.11.X```: 21 | 22 | ```scala 23 | resolvers += "fenek" at "http://dl.bintray.com/univalence/univalence-jvm" 24 | libraryDependencies += "io.univalence" %% "fenek" % "0.3+79-4936e981" 25 | ``` 26 | -------------------------------------------------------------------------------- /fenek/src/main/scala/io/univalence/fenek/Query.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.fenek 2 | 3 | import io.univalence.fenek.Expr.Struct 4 | 5 | sealed trait Query { 6 | final def andWhere(expr: Expr[Boolean]): Query = Where(this, expr) 7 | //final def orWhere(expr: Expr[Boolean]):Query = OrWhere(this,expr) 8 | final def union(query: Query): Query = Union(this, query) 9 | } 10 | 11 | case class Select(projection: Struct) extends Query 12 | 13 | case class Union(left: Query, right: Query) extends Query 14 | 15 | case class Where(query: Query, where: Expr[Boolean]) extends Query 16 | 17 | object Query { 18 | implicit def structToQuery(struct: Struct): Query = Select(struct) 19 | } 20 | -------------------------------------------------------------------------------- /fenek/src/main/scala/io/univalence/fenek/analysis/Sources.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.fenek.analysis 2 | 3 | import io.univalence.fenek.Expr 4 | import io.univalence.fenek.Expr.UntypedExpr 5 | import io.univalence.fenek.generic.GenericExpr 6 | import io.univalence.typedpath.Key 7 | 8 | object Sources { 9 | 10 | //TODO Test and redo 11 | def getSources(expr: UntypedExpr): Vector[Key] = 12 | /* 13 | def loop(genericExpr: GenericExpr, suffix: Vector[String] = Vector.empty): Vector[Path] = 14 | genericExpr.expr.value match { 15 | case Expr.Ops.SelectField(name, source) => 16 | loop(GenericExpr(source), name +: suffix) 17 | case Expr.Ops.RootField(name) => Vector(Path(name +: suffix)) 18 | case _ => 19 | for { 20 | sourceline <- genericExpr.sources.toVector 21 | source <- sourceline.value 22 | x <- loop(source) 23 | } yield x 24 | 25 | } 26 | 27 | loop(GenericExpr(expr))*/ 28 | ??? 29 | } 30 | -------------------------------------------------------------------------------- /fenek/src/main/scala/io/univalence/fenek/generic/GenericExpr.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.fenek.generic 2 | 3 | import io.univalence.fenek.Expr.Ops.Lit 4 | import io.univalence.fenek.Expr.{ StructField, UntypedExpr } 5 | 6 | sealed trait GenericExpr { 7 | import GenericExpr.{ Named, OneOrMore } 8 | def expr: Named[UntypedExpr] 9 | def sources: Seq[Named[OneOrMore[GenericExpr]]] 10 | def strs: Seq[Named[String]] 11 | def values: Seq[Named[Any]] 12 | } 13 | 14 | object GenericExpr { 15 | 16 | type OneOrMore[T] = ::[T] 17 | 18 | def OneOrMore[T](t: T, ts: T*): OneOrMore[T] = ::(t, ts.toList) 19 | 20 | case class Named[+T](name: String, value: T) 21 | 22 | def apply(sourceExpr: UntypedExpr): GenericExpr = { 23 | val named = toNamedSeq(sourceExpr) 24 | 25 | new GenericExpr { 26 | override lazy val expr: Named[UntypedExpr] = 27 | Named(sourceExpr.getClass.getName, sourceExpr) 28 | 29 | override lazy val sources: Seq[Named[OneOrMore[GenericExpr]]] = { 30 | val res = named.collect({ 31 | case Named(name, expr: UntypedExpr) => 32 | Named(name, OneOrMore(apply(expr))) 33 | // a Field could be a expression as well 34 | case Named(name, StructField(fname, source)) => 35 | Named(name + "." + fname, OneOrMore(apply(source))) 36 | case Named(name, (expr1: UntypedExpr, expr2: UntypedExpr)) => 37 | Named(name, OneOrMore(apply(expr1), apply(expr2))) 38 | }) 39 | 40 | res 41 | } 42 | 43 | override lazy val strs: Seq[Named[String]] = { 44 | named.collect({ case Named(name, s: String) => Named(name, s) }) 45 | } 46 | override lazy val values: Seq[Named[Any]] = { 47 | expr.value match { 48 | case Lit(value, _) => List(Named("value", value)) 49 | case _ => Nil 50 | } 51 | } 52 | } 53 | 54 | } 55 | 56 | private def toNamedSeq(entity: AnyRef): Seq[Named[Any]] = 57 | entity.getClass.getDeclaredFields.flatMap(field => { 58 | field.setAccessible(true) 59 | 60 | field.get(entity) match { 61 | case Seq() | None | Nil => Nil 62 | case Some(v) => List(Named(field.getName, v)) 63 | case xs: Seq[_] => xs.map(x => Named(field.getName, x)) 64 | case x => List(Named(field.getName, x)) 65 | } 66 | }) 67 | 68 | } 69 | -------------------------------------------------------------------------------- /fenek/src/main/scala/io/univalence/fenek/package.scala: -------------------------------------------------------------------------------- 1 | package io.univalence 2 | 3 | import io.univalence.fenek.Expr.{ StructField, UntypedExpr } 4 | import io.univalence.typedpath.Key 5 | 6 | package object fenek { 7 | implicit class fieldOps(name: String) { 8 | def <<-(expr: UntypedExpr): StructField = StructField(name, expr) 9 | //def <<-(path: Path): StructField = StructField(name, path) 10 | } 11 | 12 | implicit class booleanOps(_expr: Expr[Boolean]) { 13 | def or(expr: Expr[Boolean]): Expr[Boolean] = _expr <*> expr |> (_ || _) 14 | def and(expr: Expr[Boolean]): Expr[Boolean] = _expr <*> expr |> (_ && _) 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /fenek/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | log4j.rootCategory=WARN, console 2 | log4j.appender.console=org.apache.log4j.ConsoleAppender 3 | log4j.appender.console.target=System.err 4 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 5 | log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n 6 | -------------------------------------------------------------------------------- /fenek/src/test/scala/io/univalence/fenek/DslTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.fenek 2 | 3 | import io.univalence.fenek.Expr._ 4 | import org.scalatest.FunSuite 5 | 6 | class DslTest extends FunSuite { 7 | 8 | test("basic") { 9 | 10 | val x: Expr[Boolean] = true caseWhen (true -> true, false -> true) 11 | 12 | val y: UntypedExpr = 2.caseWhen(1 -> "b", Else -> 0) 13 | 14 | val z: UntypedExpr = 1.caseWhen(1 -> "a", 2 -> "b").as[String] |> (_.toUpperCase) 15 | 16 | } 17 | 18 | test("simple") { 19 | val expr: UntypedExpr = Expr.lit(0) 20 | 21 | val rule0: Expr.CaseWhenExpr[Any] = CaseWhenExpr(1 -> "a", 2 -> 1, 3 -> true) 22 | 23 | val rule4: Expr.CaseWhenExpr[Any] = CaseWhenExpr(1 -> 1, 2 -> 2, 3 -> "a") 24 | 25 | val rule5: Expr.CaseWhenExpr[Any] = CaseWhenExpr(1 -> 1, 2 -> 2, Else -> expr) 26 | 27 | val rule2: Expr.CaseWhenExpr[Int] = CaseWhenExpr(1 -> 2, 2 -> 3, Else -> 4) 28 | 29 | val rule1: Expr.CaseWhenExpr[Int] = CaseWhenExpr(true -> 2, false -> 3) 30 | 31 | val e1: Expr[Int] = lit(true) caseWhen rule1 caseWhen rule2 32 | 33 | val e2: Expr[Int] = lit(true) caseWhen 1 -> 2 34 | 35 | val e4: UntypedExpr = lit(true) caseWhen rule0 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /fenek/src/test/scala/io/univalence/fenek/beta/StaticAnalysis.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.fenek.beta 2 | 3 | import io.univalence.fenek.{ Else, Expr } 4 | import io.univalence.fenek.Expr.{ CaseWhen, Ops, UntypedExpr } 5 | import io.univalence.fenek.Expr.Ops.RootField 6 | 7 | object StaticAnalysis { 8 | 9 | sealed trait Movement 10 | 11 | object Movement { 12 | 13 | case object Up extends Movement 14 | 15 | case object Down extends Movement 16 | 17 | case object Same extends Movement 18 | 19 | } 20 | 21 | type Tree[T] = Seq[(Movement, T)] 22 | 23 | import Movement._ 24 | 25 | Seq(Same -> 1, Down -> 2, Down -> 3, Same -> 4, Up -> 5) 26 | 27 | /* 1 1 0 9 28 | - 2 2 1 6 29 | - 3 3 2 3 30 | - 4 4 4 5 31 | - 5 5 7 8 32 | */ 33 | 34 | /* 35 | lit(a) caseWhen (1 -> a + b | 2 -> a - b | Else -> 3) 36 | 37 | caseWhen 38 | - a 39 | - caseWhenExp 40 | - valueEqual 41 | - 1 42 | - + 43 | - a 44 | - b 45 | - valueEqual 46 | - 2 47 | - - 48 | - a 49 | - b 50 | - elseCase 51 | - 3 52 | */ 53 | 54 | case class PosExpr(level: Int, index: Int, expr: UntypedExpr) 55 | 56 | def staticAnalysis(expr: UntypedExpr): Seq[PosExpr] = { 57 | def loop(expr: UntypedExpr, pos: Int, index: Int): Seq[PosExpr] = { 58 | 59 | import Expr.Ops._ 60 | val toUnfold: Seq[UntypedExpr] = expr match { 61 | case cw: CaseWhen[Any] => 62 | Seq(Seq(cw.source), cw.cases.pairs.flatMap(t => Seq(t._1, t._2)), cw.cases.orElse.toList).flatten 63 | 64 | case x: RootField => Nil 65 | case l: Lit[_] => Nil 66 | 67 | case Map2(a, b, _, _) => Seq(a, b) 68 | 69 | case TypeCasted(a, _) => Seq(a) 70 | } 71 | 72 | def next(expr: UntypedExpr, index: Int): Seq[PosExpr] = 73 | loop(expr, pos + 1, index) 74 | 75 | val res = toUnfold.foldLeft[(Seq[PosExpr], Int)]((Nil, index + 1))( 76 | { 77 | case ((xs, i), e) => 78 | val ys = next(e, i) 79 | (xs ++ ys, i + ys.size) 80 | } 81 | ) 82 | 83 | PosExpr(pos, index, expr) +: res._1 84 | 85 | } 86 | 87 | loop(expr, pos = 0, 0) 88 | } 89 | 90 | def main(args: Array[String]): Unit = { 91 | 92 | import io.univalence.typedpath._ 93 | import io.univalence.fenek.Expr._ 94 | 95 | val ab = key"a".as[Int] <*> key"b".as[Int] 96 | 97 | val x = key"a".caseWhen(1 -> (ab |> (_ + _)), 2 -> (ab |> (_ - _)), Else -> 3) 98 | 99 | staticAnalysis(x).foreach({ 100 | case PosExpr(level, index, expr) => 101 | println(index.formatted("%03d ") + (" " * level) + expr) 102 | }) 103 | 104 | } 105 | 106 | } 107 | -------------------------------------------------------------------------------- /logo.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/univalence/spark-tools/12a158ea185f77156624bd5797316141461b514b/logo.png -------------------------------------------------------------------------------- /parka/README.md: -------------------------------------------------------------------------------- 1 | Parka 2 | ====================== 3 | 4 | [ ![Download](https://api.bintray.com/packages/univalence/univalence-jvm/parka/images/download.svg) ](https://bintray.com/univalence/univalence-jvm/parka/_latestVersion) 5 | 6 | Parka is a library about data quality of a Datasets in Scala. 7 | 8 | It implements DeltaQA for Datasets, comparing two Datasets to each other and notifying differences into Parka Analysis which is an object that contains the comparison’s data.. 9 | 10 | ## Table of content 11 | 12 | - [Installation](#installation) 13 | - [Usage](#usage) 14 | - [Support](#support) 15 | - [Authors](#authors) 16 | - [License](#license) 17 | - [Dependencies](#dependencies) 18 | - [Links](#links) 19 | 20 | ## Installation 21 | 22 | ### Stable version 23 | 24 | A stable version isn't available yet. 25 | 26 | ### Latest version 27 | 28 | If you want to get the very last version of this library you can still download it using bintray here : https://bintray.com/univalence/univalence-jvm/parka 29 | 30 | Here is an example using ```version 0.3+79-4936e981``` that work with ```scala 2.11.X```: 31 | 32 | ```scala 33 | resolvers += "parka" at "http://dl.bintray.com/univalence/univalence-jvm" 34 | libraryDependencies += "io.univalence" %% "parka" % "0.3+79-4936e981" 35 | ``` 36 | 37 | ## Usage 38 | 39 | The entry of Parka is the Parka Analysis object, this object contains all the information about the comparison between two Datasets. 40 | 41 | To get a Parka Analysis, first import parka and then generate the analysis from two Datasets as below : 42 | 43 | ```scala 44 | import io.univalence.parka.Parka 45 | 46 | val pa: ParkaAnalysis = Parka(df1, d2f)("key") 47 | //or 48 | 49 | val pa: ParkaAnalysis = Parka.withConfig(nPartition = 500)(df1, df2)("key1", "key2") 50 | 51 | ``` 52 | 53 | First give the two Datasets to compare to and then column(s) that are keys. then print the result in the console or export it in JSON. 54 | 55 | Here is an example : 56 | 57 | ```scala 58 | import io.univalence.parka.Printer 59 | 60 | println(Printer.printParkaResult(pa.result)) 61 | ``` 62 | ## Support 63 | 64 | If you have any problem/question don't hesitate to add a new issue. 65 | 66 | ## Authors 67 | 68 | Made with :heart: by Univalence's team. 69 | 70 | ## License 71 | 72 | Parka is licensed under the Apache License, Version 2.0 (the “License”); you may not use this software except in compliance with the License. 73 | 74 | 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. 75 | 76 | ## Dependencies 77 | 78 | * [algebird](https://github.com/twitter/algebird) - 0.13.4 --> Approximate data structure 79 | * [magnolia](https://github.com/propensive/magnolia) - 0.10.0 --> Generic derivation for typeclasses 80 | * [jline](https://github.com/jline/jline3) - 3.12.1 --> Console's size 81 | * [circe](https://github.com/circe/circe) - 0.11.1 --> Json serialization 82 | * [spark-test](https://github.com/univalence/spark-tools/tree/master/spark-test) - current --> Dataframe's testing tool 83 | 84 | ## Links 85 | 86 | * [Univalence Web site](https://www.univalence.io/) 87 | * [Microsite](https://univalence.github.io/spark-tools/parka/) 88 | * [Source code](https://github.com/univalence/spark-tools/tree/master/parka/src/main/scala/io/univalence/parka) 89 | * [Video](https://www.youtube.com/watch?v=t24sUF2zWLY) - DeltaQA introduction between [14:25](http://www.youtube.com/watch?v=t24sUF2zWLY&t=14m25s) and [28:10](http://www.youtube.com/watch?v=t24sUF2zWLY&t=28m10s) 90 | 91 | :star: Star us on GitHub — it helps! 92 | -------------------------------------------------------------------------------- /parka/src/main/scala/io/univalence/parka/ConsoleSize.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.parka 2 | 3 | import java.util.Optional 4 | import org.jline.terminal.{ Size, TerminalBuilder } 5 | 6 | case class ConsoleSize(columns: Int, rows: Int) 7 | 8 | object ConsoleSize { 9 | 10 | val default = ConsoleSize(80, 25) 11 | 12 | def get: ConsoleSize = 13 | if (isInsideTrueTerminal) fromJline(TerminalBuilder.terminal().getSize) 14 | else default 15 | 16 | private def fromJline(size: Size) = ConsoleSize(size.getColumns, size.getRows) 17 | 18 | private def isInsideTrueTerminal: Boolean = 19 | (for { 20 | inIntelliJ <- isInsideIntelliJ 21 | } yield isInsideEmacs || !inIntelliJ).getOrElse(false) 22 | 23 | private def isInsideEmacs: Boolean = System.getenv("INSIDE_EMACS") != null 24 | 25 | /** Detect IntelliJ Idea. 26 | * 27 | * Code according to JLine. 28 | * 29 | * @return None if the current JVM version can't get the parent process. Some(true) if IntelliJ IDEA has been 30 | * detected as parent process. 31 | */ 32 | private def isInsideIntelliJ: Option[Boolean] = 33 | try { 34 | // java.lang.ProcessHandle is defined in Java 9+ and not in Java 8 or before. So reflexion is mandatory here. 35 | val phClass = Class.forName("java.lang.ProcessHandle") 36 | val current = phClass.getMethod("current").invoke(null) 37 | val parent = phClass.getMethod("parent").invoke(current).asInstanceOf[Optional[AnyRef]].orElse(null) 38 | val infoMethod = phClass.getMethod("info") 39 | val info = infoMethod.invoke(parent) 40 | val command = 41 | infoMethod.getReturnType.getMethod("command").invoke(info).asInstanceOf[Optional[String]].orElse(null) 42 | 43 | Some(command != null && command.contains("idea")) 44 | } catch { 45 | case _: Throwable => None 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /parka/src/main/scala/io/univalence/parka/Constraintor.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.parka 2 | 3 | /** 4 | * The Constraintor Object is a way to analyse a Parla Result 5 | * This is convenient if you use Parka to verify that the new version of your project is working as expected 6 | * For example if your new version should not change anything about the data process then the parka result 7 | * should be empty. Instead of looking inside the parka result you can just set a constraint about your Parka Result 8 | * using the Constraintor. 9 | * Example: 10 | * Constraintor.respectConstraints(analysis)(isSimilar) should return Pass in case of nothing has changed. 11 | */ 12 | object Constraintor { 13 | sealed trait Status 14 | case object Pass extends Status 15 | case class Fail(constraints: Constraint*) extends Status 16 | 17 | sealed trait Constraint{ 18 | def ok(pa: ParkaAnalysis): Boolean 19 | } 20 | 21 | case object isSimilar extends Constraint { 22 | override def ok(pa: ParkaAnalysis): Boolean = 23 | noOuter.ok(pa) && noInner.ok(pa) 24 | } 25 | case object noOuter extends Constraint{ 26 | override def ok(pa: ParkaAnalysis): Boolean = 27 | pa.result.outer == Outer(Both(DescribeByRow(0, RowBasedMap.empty),DescribeByRow(0, RowBasedMap.empty))) 28 | } 29 | case object noInner extends Constraint{ 30 | override def ok(pa: ParkaAnalysis): Boolean = 31 | pa.result.inner.countRowNotEqual == 0 32 | } 33 | 34 | case class colChange(col: String) extends Constraint{ 35 | override def ok(pa: ParkaAnalysis): Boolean = 36 | pa.result.inner.byColumn.get(col).map(!_.error.equals(Describe.empty)).getOrElse(throw new Exception("Fail")) 37 | } 38 | 39 | case class colsChange(col: String*) extends Constraint{ 40 | override def ok(pa: ParkaAnalysis): Boolean = 41 | col.forall(colChange(_).ok(pa)) 42 | } 43 | 44 | case class colNotChange(col: String) extends Constraint{ 45 | override def ok(pa: ParkaAnalysis): Boolean = 46 | !colChange(col).ok(pa) 47 | } 48 | 49 | case class colsNotChange(col: String*) extends Constraint{ 50 | override def ok(pa: ParkaAnalysis): Boolean = 51 | col.forall(colNotChange(_).ok(pa)) 52 | } 53 | 54 | def respectConstraints(pa: ParkaAnalysis)(constraints: Constraint*): Status = { 55 | val verification = constraints.map(constraint => (constraint, constraint.ok(pa))).filter({case (_, ok) => ok == false}) 56 | verification.isEmpty match{ 57 | case true => Pass 58 | case false => Fail(verification.map(_._1): _*) 59 | } 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /parka/src/main/scala/io/univalence/parka/RowBasedMap.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.parka 2 | 3 | import cats.kernel.Monoid 4 | 5 | import scala.reflect.ClassTag 6 | 7 | /** 8 | * Implementation Note 9 | * It uses Array[Any] as {@link scala.collection.mutable.ArraySeq} 10 | * **/ 11 | class RowBasedMap[K, V] private (val keys_ : Array[Any], val values_ : Array[Any]) extends Map[K, V] with Serializable { 12 | 13 | override def +[B1 >: V](kv: (K, B1)): Map[K, B1] = 14 | keys_.indexOf(kv._1) match { 15 | case -1 => new RowBasedMap(keys_ :+ kv._1, values_ :+ kv._2) 16 | case i => new RowBasedMap(keys_, values_.updated(i, kv._2)) 17 | } 18 | 19 | override def get(key: K): Option[V] = 20 | keys_.indexOf(key) match { 21 | case -1 => None 22 | case i => Some(values_(i).asInstanceOf[V]) 23 | } 24 | 25 | override def iterator: Iterator[(K, V)] = keys_.zip(values_).toIterator.asInstanceOf[Iterator[(K, V)]] 26 | 27 | override def -(key: K): RowBasedMap[K, V] = 28 | keys_.indexOf(key) match { 29 | case -1 => this 30 | case i => 31 | import RowBasedMap._ 32 | new RowBasedMap(removeIndex(keys_, i), removeIndex(values_, i)) 33 | } 34 | 35 | def combine(right: RowBasedMap[K, V])(implicit monoid: Monoid[V]): RowBasedMap[K, V] = 36 | if (keys_ sameElements right.keys_) { 37 | val arr: Array[Any] = new Array(values_.length) 38 | for (i <- arr.indices) { 39 | arr(i) = monoid.combine(values_(i).asInstanceOf[V], right.values_(i).asInstanceOf[V]) 40 | } 41 | new RowBasedMap(keys_, arr) 42 | } else { 43 | 44 | if (right.values_.length > values_.length) { 45 | right.combine(this) 46 | } else { 47 | 48 | val indexMap: Array[Int] = new Array(right.values_.length) 49 | var extraValue: Int = 0 50 | 51 | for (i <- right.keys_.indices) { 52 | keys_.indexOf(right.keys_(i)) match { 53 | case -1 => 54 | extraValue += 1 55 | indexMap(i) = values_.length + extraValue 56 | case x => 57 | indexMap(i) = x 58 | } 59 | } 60 | 61 | val resKeys: Array[Any] = new Array(keys_.length + extraValue) 62 | val resValues: Array[Any] = new Array(keys_.length + extraValue) 63 | 64 | keys_.copyToArray(resKeys) 65 | values_.copyToArray(resValues) 66 | 67 | for (i <- indexMap.indices) { 68 | val j = indexMap(i) 69 | resKeys(j) = right.keys_(i) 70 | resValues(j) = monoid.combine(values_(j).asInstanceOf[V], right.values_(i).asInstanceOf[V]) 71 | } 72 | 73 | new RowBasedMap(resKeys, resValues) 74 | } 75 | 76 | } 77 | 78 | } 79 | 80 | object RowBasedMap { 81 | 82 | def empty[K, V]: RowBasedMap[K, V] = new RowBasedMap[K, V](Array.empty, Array.empty) 83 | 84 | implicit def toColbaseMap[K, V](map: Map[K, V]): RowBasedMap[K, V] = { 85 | val kvs = map.toArray 86 | new RowBasedMap(kvs.map(_._1).toArray, kvs.map(_._2).toArray) 87 | } 88 | 89 | def toColbaseMapFromSeq[K, V](seq: Seq[(K, V)]): RowBasedMap[K, V] = 90 | new RowBasedMap(seq.map(_._1).toArray, seq.map(_._2).toArray) 91 | 92 | def removeIndex[E: ClassTag](seq: Array[E], index: Int): Array[E] = 93 | (seq.take(index) ++ seq.drop(index + 1)).toArray 94 | 95 | def apply[K, V](keys_ : Seq[K], values_ : Seq[V]): RowBasedMap[K, V] = 96 | new RowBasedMap(keys_.toArray, values_.toArray) 97 | } 98 | -------------------------------------------------------------------------------- /parka/src/test/ressources/leftTest.csv: -------------------------------------------------------------------------------- 1 | key;value;eulav 2 | a;0;a 3 | b;1;b 4 | c;2;c 5 | d;3;d 6 | e;4;e 7 | f;5;f 8 | g;6;g 9 | h;7;h 10 | i;8;i 11 | j;9;j 12 | -------------------------------------------------------------------------------- /parka/src/test/ressources/rightTest.csv: -------------------------------------------------------------------------------- 1 | key;value;eulav 2 | a;0;a 3 | b;1;b 4 | c;2;c 5 | d;3;d 6 | e;4;e 7 | f;5;f 8 | g;6;g 9 | h;77;h 10 | i;8;ii 11 | j;99;jj 12 | -------------------------------------------------------------------------------- /parka/src/test/scala/io/univalence/parka/CompressMapTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.parka 2 | 3 | import cats.kernel.Semigroup 4 | import org.scalatest.FunSuite 5 | 6 | class CompressMapTest extends FunSuite { 7 | 8 | test("testApply") { 9 | 10 | val addInt = new Semigroup[Int] { 11 | override def combine(x: Int, y: Int): Int = x + y 12 | } 13 | 14 | assert(CompressMap(Map(Set(1, 2) -> 3, Set(2, 3) -> 2), 1)(addInt) == Map(Set(1, 2, 3) -> 5)) 15 | 16 | assert(CompressMap(Map(Set(1, 2) -> 3, Set(2, 3) -> 2), 2)(addInt) == Map(Set(1, 2) -> 3, Set(2, 3) -> 2)) 17 | 18 | assert( 19 | CompressMap(Map( 20 | Set(1, 2) -> 3, 21 | Set(2, 3) -> 2, 22 | Set(1, 2, 3) -> 0, 23 | Set(5, 6) -> 7, 24 | Set(6, 7) -> 1, 25 | Set(5, 6, 7) -> 0 26 | ), 27 | 2)(addInt) == Map(Set(1, 2, 3) -> 5, Set(5, 6, 7) -> 8) 28 | ) 29 | 30 | } 31 | 32 | } 33 | -------------------------------------------------------------------------------- /parka/src/test/scala/io/univalence/parka/DeltaTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.parka 2 | 3 | import org.scalatest.FunSuite 4 | import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks 5 | 6 | class DeltaTest extends FunSuite with ScalaCheckPropertyChecks { 7 | 8 | test("testApply") { 9 | forAll((xs: Seq[Boolean]) => { 10 | if (xs.size >= 2) { 11 | val (left, right) = xs.splitAt(xs.size / 2) 12 | val bb: Seq[(Boolean, Boolean)] = left.zip(right) 13 | 14 | import MonoidGen._ 15 | val monoid = gen[Delta] 16 | 17 | val delta: Delta = monoid.combineAll(bb.map({ case (l, r) => Delta(l, r) })) 18 | 19 | assert(delta.nEqual == bb.count({ case (l, r) => l == r })) 20 | assert(delta.nNotEqual == bb.count({ case (l, r) => l != r })) 21 | 22 | def count(left: Boolean, right: Boolean): Int = bb.count(_ == ((left, right))) 23 | 24 | val deltaBoolean = delta.asBoolean.get 25 | 26 | import deltaBoolean._ 27 | 28 | assert(ft == count(left = false, right = true)) 29 | assert(tf == count(left = true, right = false)) 30 | assert(tt == count(left = true, right = true)) 31 | assert(ff == count(left = false, right = false)) 32 | } 33 | }) 34 | 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /parka/src/test/scala/io/univalence/parka/DescribeTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.parka 2 | 3 | import org.scalatest.FunSuiteLike 4 | 5 | class DescribeTest extends FunSuiteLike { 6 | 7 | test("Describe enums") { 8 | val result = Describe("FR") 9 | 10 | assert(result.count == 1) 11 | assert(result.enums("value").estimate("FR") == 1) 12 | } 13 | 14 | } 15 | -------------------------------------------------------------------------------- /parka/src/test/scala/io/univalence/parka/KModes2Test.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.parka 2 | 3 | import org.scalatest.FunSuite 4 | 5 | import scala.collection.immutable.BitSet 6 | 7 | class KModes2Test extends FunSuite { 8 | 9 | test("testNewKModes2") { 10 | 11 | val kModes2 = KModes2.newKModes2(5, 10) 12 | 13 | assert(kModes2.centers.size == 5) 14 | 15 | assert(kModes2.centers.map(_.max).max <= 10) 16 | } 17 | 18 | test("fit") { 19 | val model = KModes2.fit((0 to 2).map(i => 1.0 -> BitSet(i)).toVector, 3, 10) 20 | 21 | assert(model.centers.size == 3) 22 | 23 | (0 to 2).foreach(i => { 24 | assert(model.centers.contains(BitSet(i))) 25 | }) 26 | } 27 | 28 | } 29 | -------------------------------------------------------------------------------- /parka/src/test/scala/io/univalence/parka/PrinterTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.parka 2 | import java.io.{ BufferedWriter, File, FileWriter } 3 | 4 | import io.univalence.sparktest.SparkTest 5 | import java.sql.{ Date, Timestamp } 6 | 7 | import io.circe.Json 8 | import io.univalence.schema.SchemaComparator.SchemaError 9 | import org.apache.spark.sql.{ DataFrame, Dataset, SparkSession } 10 | import org.scalactic.Prettifier 11 | import org.scalatest.{ FunSuite, Tag, Transformer } 12 | 13 | class PrinterTest extends FunSuite with SparkTest with HistogramTest { 14 | 15 | import MonoidGen._ 16 | 17 | implicit val default: Prettifier = Prettifier.default 18 | 19 | //Turn status to false if you want to ignore these tests 20 | private val status: Boolean = false 21 | 22 | private def run(testName: String, testTags: Tag*)(testFun: => Unit) { 23 | status match { 24 | case true => test(testName, testTags: _*)(testFun) 25 | case false => ignore(testName, testTags: _*)(testFun) 26 | } 27 | } 28 | 29 | run("Prettify test") { 30 | val left: Dataset[Element] = dataset( 31 | Element("0", 100), 32 | Element("1", 100), 33 | Element("2", 100), 34 | Element("3", 100), 35 | Element("4", 100), 36 | Element("5", 100), 37 | Element("6", 100), 38 | Element("7", 100), 39 | Element("8", 100), 40 | Element("9", 100) 41 | ) 42 | val right: Dataset[Element] = dataset( 43 | Element("0", 0), 44 | Element("1", 10), 45 | Element("2", 20), 46 | Element("3", 30), 47 | Element("4", 40), 48 | Element("5", 50), 49 | Element("6", 60), 50 | Element("7", 70), 51 | Element("8", 80), 52 | Element("9", 90), 53 | Element("10", 100) 54 | ) 55 | val result = Parka(left, right)("key").result 56 | val part = Printer.printParkaResult(result) 57 | println(Part.toString(part)) 58 | } 59 | 60 | run("Prettify test 2 column") { 61 | val left: Dataset[Element2] = dataset( 62 | Element2("0", 0, 0), 63 | Element2("1", 10, 0), 64 | Element2("2", 20, 0), 65 | Element2("3", 30, 10), 66 | Element2("4", 40, 0), 67 | Element2("5", 50, 20), 68 | Element2("6", 60, 0), 69 | Element2("7", 70, 30), 70 | Element2("8", 80, 0), 71 | Element2("9", 80, 30) 72 | ) 73 | val right: Dataset[Element2] = dataset( 74 | Element2("0", 0, 0), 75 | Element2("1", 10, 0), 76 | Element2("2", 20, 10), 77 | Element2("3", 30, 10), 78 | Element2("4", 40, 20), 79 | Element2("5", 50, 20), 80 | Element2("6", 60, 30), 81 | Element2("7", 70, 30), 82 | Element2("8", 80, 40), 83 | Element2("9", 90, 40), 84 | Element2("10", 100, 50) 85 | ) 86 | val result = Parka(left, right)("key").result 87 | val part = Printer.printParkaResult(result) 88 | println(Part.toString(part)) 89 | } 90 | 91 | run("Prettify test null") { 92 | val left = Seq( 93 | (1, "aaaa"), 94 | (2, "bbbb"), 95 | (3, null), 96 | (4, null) 97 | ).toDF("id", "str") 98 | 99 | val right = Seq( 100 | (1, null), 101 | (2, "bbbb"), 102 | (3, "cccc"), 103 | (4, null), 104 | (5, "eeee") 105 | ).toDF("id", "str") 106 | 107 | val result = Parka(left, right)("id").result 108 | val part = Printer.printParkaResult(result) 109 | println(Part.toString(part)) 110 | } 111 | 112 | run("Prettify test enum") { 113 | val left = Seq( 114 | (1, "FR"), 115 | (2, "FR"), 116 | (3, "DE"), 117 | (4, "DE") 118 | ).toDF("id", "str") 119 | 120 | val right = Seq( 121 | (1, "FR"), 122 | (2, "DE"), 123 | (3, "FR"), 124 | (4, "DE"), 125 | (5, "DE") 126 | ).toDF("id", "str") 127 | 128 | val result = Parka(left, right)("id").result 129 | val part = Printer.printParkaResult(result) 130 | println(Part.toString(part)) 131 | } 132 | } 133 | -------------------------------------------------------------------------------- /plumbus/README.md: -------------------------------------------------------------------------------- 1 | Spark Plumbus 2 | ====================== 3 | 4 | [ ![Download](https://api.bintray.com/packages/univalence/univalence-jvm/plumbus/images/download.svg) ](https://bintray.com/univalence/univalence-jvm/plumbus/_latestVersion) 5 | 6 | Spark Plumbus is a collection of tools for Scala Spark, that aims to 7 | improve the day to day life of data engineers. 8 | 9 | ## Installation 10 | 11 | ### Stable version 12 | 13 | Version for scala 2.11.X : 14 | 15 | ```scala 16 | libraryDependencies += "io.univalence" % "plumbus_2.11" % "0.2+220-639d2e4b" 17 | ``` 18 | 19 | ### Latest version 20 | 21 | If you want to get the very last version of this library you can still download it using bintray here : https://bintray.com/univalence/univalence-jvm/plumbus 22 | 23 | Here is an example using ```version 0.3+79-4936e981``` that work with ```scala 2.11.X```: 24 | 25 | ```scala 26 | resolvers += "plumbus" at "http://dl.bintray.com/univalence/univalence-jvm" 27 | libraryDependencies += "io.univalence" %% "plumbus" % "0.3+79-4936e981" % Test 28 | ``` 29 | 30 | ## FAQ 31 | 32 | ### Plumbus ? 33 | 34 | A Plumbus is an all-purpose home device. Everyone knows what it does, 35 | so there is no reason to explain it! For the rest, you can rely on [this 36 | Website](https://rickandmorty.fandom.com/wiki/Plumbus) . 37 | 38 | ## License 39 | 40 | Copyright 2019 Univalence.io 41 | 42 | Licensed under the Apache License, Version 2.0: 43 | http://www.apache.org/licenses/LICENSE-2.0 44 | -------------------------------------------------------------------------------- /plumbus/src/main/scala/io/univalence/plumbus/cc2map.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.plumbus 2 | 3 | import java.util.UUID 4 | 5 | import mercator.Monadic 6 | 7 | import scala.util.Try 8 | 9 | object cc2map { 10 | 11 | class Prefix private (private val names: String = "") extends AnyVal { 12 | def rekey(str: String): String = 13 | str 14 | .map(s => if (s.isUpper) "_" + s.toLower else s.toString) 15 | .mkString 16 | 17 | def add(name: String): Prefix = 18 | if (names.isEmpty) new Prefix(name) 19 | else new Prefix(names + "." + rekey(name)) 20 | 21 | def toKey: String = names 22 | 23 | } 24 | 25 | object Prefix { 26 | val empty: Prefix = new Prefix() 27 | } 28 | 29 | trait ToMap[A] { 30 | def toMap(a: A, prefix: Prefix = Prefix.empty): Map[String, Any] 31 | } 32 | 33 | object ToMap { 34 | 35 | import magnolia._ 36 | type Typeclass[T] = ToMap[T] 37 | 38 | private val _instance: ToMap[Any] = 39 | new ToMap[Any] { 40 | import scala.collection.immutable.Map.Map1 41 | override def toMap(t: Any, prefix: Prefix): Map[String, Any] = 42 | new Map1(prefix.toKey, t) 43 | } 44 | 45 | def instance[T]: ToMap[T] = _instance.asInstanceOf[ToMap[T]] 46 | 47 | implicit val str: ToMap[String] = instance 48 | implicit val int: ToMap[Int] = instance 49 | implicit val uuid: ToMap[UUID] = instance 50 | implicit val long: ToMap[Long] = instance 51 | implicit val boolean: ToMap[Boolean] = instance 52 | 53 | implicit def opt[T](implicit T: ToMap[T]): ToMap[Option[T]] = 54 | new ToMap[Option[T]] { 55 | override def toMap(t: Option[T], prefix: Prefix): Map[String, Any] = 56 | t match { 57 | case None => Map.empty 58 | case Some(x) => T.toMap(x, prefix) 59 | } 60 | } 61 | 62 | def combine[T](ctx: CaseClass[Typeclass, T]): ToMap[T] = 63 | new ToMap[T] { 64 | override def toMap(t: T, prefix: Prefix): Map[String, Any] = 65 | ctx.parameters 66 | .flatMap(param => { 67 | param.typeclass.toMap(param.dereference(t), prefix.add(param.label)) 68 | }) 69 | .toMap 70 | } 71 | 72 | def dispatch[T](ctx: SealedTrait[Typeclass, T]): ToMap[T] = 73 | new ToMap[T] { 74 | override def toMap(t: T, prefix: Prefix): Map[String, Any] = 75 | ctx.dispatch(t) { sub => 76 | sub.typeclass.toMap(sub.cast(t), prefix) 77 | } 78 | } 79 | 80 | implicit def gen[T]: ToMap[T] = macro Magnolia.gen[T] 81 | 82 | } 83 | 84 | trait FromMap[A] { 85 | def fromMap(map: Map[String, Any]): Try[A] 86 | } 87 | 88 | object FromMap { 89 | 90 | import magnolia._ 91 | type Typeclass[T] = FromMap[T] 92 | 93 | def instance[T]: FromMap[T] = new FromMap[T] { 94 | override def fromMap(map: Map[String, Any]): Try[T] = 95 | Try(map.head._2.asInstanceOf[T]) 96 | } 97 | 98 | implicit val str: FromMap[String] = instance 99 | implicit val int: FromMap[Int] = instance 100 | implicit val uuid: FromMap[UUID] = instance 101 | implicit val long: FromMap[Long] = instance 102 | implicit val boolean: FromMap[Boolean] = instance 103 | 104 | implicit def opt[T](implicit T: FromMap[T]): FromMap[Option[T]] = 105 | new FromMap[Option[T]] { 106 | override def fromMap(map: Map[String, Any]): Try[Option[T]] = 107 | if (map.isEmpty) Try(None) 108 | else { 109 | T.fromMap(map).map(Option.apply) 110 | } 111 | } 112 | 113 | def combine[T](ctx: CaseClass[Typeclass, T]): FromMap[T] = 114 | new FromMap[T] { 115 | override def fromMap(map: Map[String, Any]): Try[T] = 116 | ctx.constructMonadic(param => { 117 | param.typeclass.fromMap(map.filterKeys(_ == param.label)) 118 | }) 119 | } 120 | 121 | implicit def gen[T]: FromMap[T] = macro Magnolia.gen[T] 122 | } 123 | 124 | def toMap[CC: ToMap](cc: CC): Map[String, Any] = 125 | implicitly[ToMap[CC]].toMap(cc) 126 | 127 | def fromMap[CC: FromMap](map: Map[String, Any]): Try[CC] = 128 | implicitly[FromMap[CC]].fromMap(map) 129 | 130 | } 131 | -------------------------------------------------------------------------------- /plumbus/src/main/scala/io/univalence/plumbus/cogroup.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.plumbus 2 | 3 | import org.apache.spark.Partitioner 4 | import org.apache.spark.rdd.{ CoGroupedRDD, RDD } 5 | import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema 6 | import org.apache.spark.sql.types.{ ArrayType, StructField } 7 | import org.apache.spark.sql.{ types, DataFrame, Dataset, Encoder, KeyValueGroupedDataset, Row } 8 | 9 | import scala.reflect.ClassTag 10 | import scala.util.Try 11 | 12 | object cogroup { 13 | 14 | /** 15 | * Usage : 16 | * {{{ 17 | * import io.univalence.plumbus.cogroup._ 18 | * 19 | * val persons:Dataset[Person] = ??? 20 | * val addresses:Dataset[Address] = ??? 21 | * 22 | * persons.groupByKey(_.id).cogroup(addresses.groupByKey(_.personId)) 23 | * }}} 24 | * 25 | * @param kvgd 26 | * @tparam K 27 | * @tparam A 28 | */ 29 | implicit class KVGD[K, A](val kvgd: KeyValueGroupedDataset[K, A]) { 30 | def cogroup[B](right: KeyValueGroupedDataset[K, B]): Dataset[(K, Seq[A], Seq[B])] = 31 | //Use SparkAddOn ? 32 | ??? 33 | } 34 | 35 | def apply[A, B, K](left: Dataset[A], right: Dataset[B])(keyLeft: A => K, keyRight: B => K)( 36 | implicit encA: Encoder[A], 37 | encB: Encoder[B], 38 | encC: Encoder[K], 39 | enc: Encoder[(K, Seq[A], Seq[B])], 40 | ca: ClassTag[A], 41 | ck: ClassTag[K], 42 | cb: ClassTag[B] 43 | ): Dataset[(K, Seq[A], Seq[B])] = 44 | left.sparkSession.implicits 45 | .rddToDatasetHolder( 46 | RDD 47 | .rddToPairRDDFunctions(left.rdd.keyBy(keyLeft)) 48 | .cogroup(right.rdd.keyBy(keyRight)) 49 | .map({ case (k, (ia, ib)) => (k, ia.toSeq, ib.toSeq) }) 50 | ) 51 | .toDS 52 | 53 | def cogroupDf(group: DataFrame, namedSubGroup: (String, DataFrame)*)( 54 | byKey: String, 55 | partitioner: Partitioner = Partitioner.defaultPartitioner(group.rdd, namedSubGroup.map(_._2.rdd): _*) 56 | ): Try[DataFrame] = 57 | Try { 58 | val subGroup: Seq[DataFrame] = namedSubGroup.map(_._2) 59 | val allFrames: Seq[DataFrame] = group +: subGroup 60 | val allFramesKeyed: Seq[RDD[(String, Row)]] = 61 | allFrames.map(df => { 62 | val idx = df.columns.indexOf(byKey) 63 | df.rdd.keyBy(_.get(idx).toString) 64 | }) 65 | 66 | val cogroupRdd: CoGroupedRDD[String] = new CoGroupedRDD[String](allFramesKeyed, partitioner) 67 | 68 | val rowRdd: RDD[Row] = 69 | cogroupRdd.map(x => { 70 | val rows: Array[Seq[Row]] = x._2.asInstanceOf[Array[Iterable[Row]]].map(_.toSeq) 71 | val seq = rows.head.head.toSeq ++ rows.tail 72 | 73 | new GenericRowWithSchema(seq.toArray, null).asInstanceOf[Row] 74 | }) 75 | 76 | val schema = 77 | types.StructType( 78 | group.schema.fields 79 | ++ namedSubGroup.map { case (name, df) => StructField(name, ArrayType(df.schema)) } 80 | ) 81 | 82 | group.sparkSession.createDataFrame(rowRdd, schema) 83 | } 84 | 85 | } 86 | -------------------------------------------------------------------------------- /plumbus/src/main/scala/io/univalence/plumbus/compress/GenerateSQL.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.plumbus.compress 2 | 3 | import io.univalence.plumbus.compress.CompressDump._ 4 | import org.apache.spark.rdd.RDD 5 | import org.apache.spark.sql.types.{ ArrayType, StructType } 6 | import org.apache.spark.sql.{ DataFrame, Row } 7 | 8 | object GenerateSQL { 9 | 10 | def displayLigneFreqOverPkPerDump(df: DataFrame): Unit = { 11 | val pos = df.schema.fieldNames.indexOf(rowsName) 12 | val pos2 = df.schema 13 | .fields(pos) 14 | .dataType 15 | .asInstanceOf[ArrayType] 16 | .elementType 17 | .asInstanceOf[StructType] 18 | .fieldNames 19 | .indexOf(dtFinalColName) 20 | 21 | val rddF: RDD[(String, Int)] = 22 | df.rdd.flatMap( 23 | _.get(pos) 24 | .asInstanceOf[Seq[Row]] 25 | .flatMap( 26 | _.get(pos2).asInstanceOf[Seq[String]] 27 | ) 28 | .groupBy(x => x) 29 | .mapValues(_.size) 30 | ) 31 | 32 | rddF.countByValue().foreach(println) 33 | } 34 | 35 | def generateView(schema: StructType, tablename: String): String = { 36 | val keyFields: Vector[String] = schema.fieldNames.toVector.filterNot(_ == rowsName) 37 | 38 | val fields: Vector[String] = schema 39 | .fields(schema.fieldIndex(rowsName)) 40 | .dataType 41 | .asInstanceOf[ArrayType] 42 | .elementType 43 | .asInstanceOf[StructType] 44 | .fieldNames 45 | .filterNot(_ == dtFinalColName) 46 | .toVector 47 | 48 | val projectionFields: Seq[String] = 49 | keyFields ++ Seq("minDt", "maxDt", "minDt_prev", "maxDt_prev", "minDt_prev IS NULL as isInit") ++ fields.flatMap( 50 | name => { 51 | val name_prev = name + "_prev" 52 | Seq( 53 | s"""(minDt_prev IS NOT NULL) AND 54 | ( ($name <> $name_prev ) 55 | OR ($name_prev IS NULL AND $name IS NOT NULL ) 56 | OR ($name IS NULL AND $name_prev IS NOT NULL ) 57 | ) as ${name}_hasChanged""", 58 | name, 59 | name_prev 60 | ) 61 | } 62 | ) 63 | 64 | s""" 65 | select 66 | ${projectionFields.mkString(",\n")} 67 | from 68 | $tablename tbl, 69 | (select 70 | lineT2.*, 71 | LAG(minDt) OVER (order by minDt) as minDt_prev, 72 | LAG(maxDt) OVER (order by minDt) as maxDt_prev, 73 | 74 | ${fields.map(name => s"LAG($name) OVER (order by minDt) as ${name}_prev").mkString(",\n")} 75 | 76 | from 77 | (select 78 | lineT1.*, 79 | minDt, 80 | maxDt 81 | 82 | from 83 | tbl.$rowsName as lineT1, 84 | (select 85 | min(dts.item) as minDt, 86 | max(dts.item) as maxDt 87 | 88 | from lineT1.compressdumpdts as dts) as dtsE) as lineT2) as lineT3 89 | """ 90 | } 91 | 92 | /*def main(args: Array[String]): Unit = { 93 | val ss = SparkSession.builder() 94 | .master("local[1]") 95 | .appName("tigrou") 96 | .config("spark.executor.memory", "25g") 97 | .config("spark.driver.memory", "1g") 98 | .config("spark.driver.maxResultSize", "0") 99 | .config("spark.sql.autoBroadcastJoinThreshold", "-1") 100 | //.config("spark.executor.memoryOverhead", "6g") 101 | .getOrCreate() 102 | import ss.implicits._ 103 | 104 | val df = ss.read.parquet("/home/phong/IdeaProjects/gouache/CompressionLZParquet/yo8") 105 | displayLigneFreqOverPkPerDump(df) 106 | println(generateView(df.schema, "youpi")) 107 | }*/ 108 | 109 | } 110 | -------------------------------------------------------------------------------- /plumbus/src/main/scala/io/univalence/plumbus/internal/CleanFromRow.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.plumbus.internal 2 | import org.apache.spark.sql.Row 3 | 4 | trait CleanFromRow[A] extends Serializable { 5 | def clean(a: Any): A 6 | } 7 | 8 | object CleanFromRow { 9 | import magnolia._ 10 | import scala.reflect.ClassTag 11 | import language.experimental.macros 12 | 13 | type Typeclass[T] = CleanFromRow[T] 14 | 15 | private def instance[A]: Typeclass[A] = 16 | new Typeclass[A] { 17 | override def clean(a: Any): A = a.asInstanceOf[A] 18 | } 19 | 20 | // Instances to associate clean operation to basic types 21 | implicit val double: Typeclass[Double] = instance 22 | implicit val boolean: Typeclass[Boolean] = instance 23 | implicit val strCFR: Typeclass[String] = instance 24 | implicit val intCFR: Typeclass[Int] = instance 25 | implicit val longCFR: Typeclass[Long] = instance 26 | // add other typeclass instances for basic types... 27 | 28 | // Instance for Option type 29 | implicit def opt[T: Typeclass: Manifest]: Typeclass[Option[T]] = 30 | new Typeclass[Option[T]] { 31 | // this helps to avoid type erasure warning 32 | private val rc = implicitly[Manifest[T]].runtimeClass 33 | 34 | override def clean(a: Any): Option[T] = 35 | a match { 36 | case ox: Option[_] if ox.forall(x => rc.isAssignableFrom(x.getClass)) => 37 | ox.asInstanceOf[Option[T]] 38 | case null => None 39 | case x => Option(implicitly[Typeclass[T]].clean(x)) 40 | } 41 | } 42 | 43 | // Instance for Seq type 44 | implicit def seq[T: Typeclass: Manifest]: Typeclass[Seq[T]] = 45 | new Typeclass[Seq[T]] { 46 | // this helps to avoid type erasure warning 47 | private val rc = implicitly[Manifest[T]].runtimeClass 48 | 49 | override def clean(a: Any): Seq[T] = 50 | a match { 51 | case Nil => Nil 52 | case xs: Seq[_] if xs.forall(x => rc.isAssignableFrom(x.getClass)) => 53 | xs.asInstanceOf[Seq[T]] 54 | case x: Seq[_] => x.map(implicitly[Typeclass[T]].clean) 55 | } 56 | } 57 | 58 | // Instance generator for case classes 59 | def combine[T: ClassTag](ctx: CaseClass[CleanFromRow, T]): Typeclass[T] = 60 | new Typeclass[T] { 61 | override def clean(a: Any): T = 62 | a match { 63 | case a: T => a 64 | case r: Row => 65 | val values: Seq[Any] = 66 | r.toSeq 67 | .zip(ctx.parameters) 68 | .map { 69 | case (rowValue, param) => param.typeclass.clean(rowValue) 70 | } 71 | ctx.rawConstruct(values) 72 | } 73 | } 74 | 75 | implicit def gen[T]: CleanFromRow[T] = macro Magnolia.gen[T] 76 | 77 | } 78 | -------------------------------------------------------------------------------- /plumbus/src/main/scala/io/univalence/plumbus/test/SparkTestLike.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.plumbus.test 2 | 3 | import org.apache.spark.sql.SparkSession 4 | 5 | trait SparkTestLike { 6 | 7 | lazy val spark: SparkSession = 8 | SparkSession 9 | .builder() 10 | .master("local[1]") 11 | .config("spark.ui.enabled", value = false) 12 | .getOrCreate() 13 | 14 | } 15 | 16 | object SparkTestLike extends SparkTestLike 17 | -------------------------------------------------------------------------------- /plumbus/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | log4j.rootCategory=WARN, console 2 | log4j.appender.console=org.apache.log4j.ConsoleAppender 3 | log4j.appender.console.target=System.err 4 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 5 | log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n 6 | -------------------------------------------------------------------------------- /plumbus/src/test/scala/io/univalence/plumbus/Cc2mapTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.plumbus 2 | 3 | import org.scalatest.FunSuite 4 | 5 | import scala.util.Try 6 | 7 | case class PersonCC(name: String, age: Int, x: Option[Int]) 8 | case class PersonCC2(name: String, age: Int) 9 | 10 | class Cc2mapTest extends FunSuite { 11 | 12 | val p1 = PersonCC("a", 1, None) 13 | val m1: Map[String, Any] = Map("name" -> "a", "age" -> 1) 14 | val p2 = PersonCC("a", 1, Some(3)) 15 | val m2: Map[String, Any] = Map("name" -> "a", "age" -> 1, "x" -> 3) 16 | 17 | val p1_2 = PersonCC2("a", 1) 18 | 19 | test("testFromMap") { 20 | assert(cc2map.fromMap[PersonCC](m1) == Try(p1)) 21 | assert(cc2map.fromMap[PersonCC](m2) == Try(p2)) 22 | } 23 | 24 | test("testToMap") { 25 | assert(cc2map.toMap(p1) == m1) 26 | assert(cc2map.toMap(p2) == m2) 27 | 28 | } 29 | 30 | test("CC to CC") { 31 | assert(cc2map.fromMap[PersonCC2](cc2map.toMap(p1)) == Try(p1_2)) 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /plumbus/src/test/scala/io/univalence/plumbus/CogroupTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.plumbus 2 | import io.univalence.plumbus.test.SparkTestLike 3 | import org.apache.spark.sql.Dataset 4 | import org.scalatest.{ FunSuiteLike, Matchers } 5 | import com.github.mrpowers.spark.fast.tests.DatasetComparer 6 | 7 | class CogroupTest extends FunSuiteLike with SparkTestLike with Matchers with DatasetComparer { 8 | import spark.implicits._ 9 | import io.univalence.plumbus.cogroup._ 10 | 11 | val person1 = PersonWithId("1", "John", 32) 12 | val person2 = PersonWithId("2", "Mary", 32) 13 | 14 | val address1 = Address("1", "address1") 15 | val address2 = Address("2", "address2") 16 | val address3 = Address("1", "address3") 17 | 18 | val persons: Dataset[PersonWithId] = Seq(person1, person2).toDS() 19 | val addresses: Dataset[Address] = Seq(address1, address2, address3).toDS() 20 | 21 | test("apply test") { 22 | val applyDS = apply(persons, addresses)(_.id, _.idPerson) 23 | val expectedDS = Seq( 24 | ("1", Seq(person1), Seq(address1, address3)), 25 | ("2", Seq(person2), Seq(address2)) 26 | ).toDS() 27 | assertSmallDatasetEquality(applyDS, expectedDS, orderedComparison = false) 28 | } 29 | } 30 | 31 | case class Address(idPerson: String, name: String) 32 | -------------------------------------------------------------------------------- /plumbus/src/test/scala/io/univalence/plumbus/CompressDumpTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.plumbus 2 | 3 | import io.univalence.plumbus.compress.CompressDump 4 | import org.apache.spark.sql.{ DataFrame, Dataset, SparkSession } 5 | import org.scalatest.FunSuite 6 | 7 | class CompressDumpTest extends FunSuite { 8 | 9 | val ss: SparkSession = 10 | SparkSession 11 | .builder() 12 | .master("local[*]") 13 | .appName("test") 14 | .config("spark.default.parallelism", "1") 15 | .getOrCreate() 16 | 17 | import ss.implicits._ 18 | 19 | test("compressUsingDF2") { 20 | val stringToRs: Map[String, Seq[R]] = 21 | Map( 22 | "dump1" -> Seq( 23 | R(1, "a", 1), 24 | R(2, "b", 22) 25 | ), 26 | "dump2" -> Seq( 27 | R(1, "a", 3), 28 | R(2, "b", 22) 29 | ) 30 | ) 31 | 32 | val df1: Dataset[(Int, Seq[RCompressed])] = 33 | CompressDump 34 | .compressUsingDF2(dfs = stringToRs.mapValues(s => ss.createDataset(s).toDF()), groupExpr = "id") 35 | .as[(Int, Seq[RCompressed])] 36 | 37 | /*val df2: DataFrame = 38 | CompressDump 39 | .compressUsingDF(dfs = stringToRs.mapValues(s ⇒ ss.createDataset(s).toDF()), groupCols = Seq("id")) 40 | 41 | println(df1.queryExecution.sparkPlan.treeString) 42 | println(df2.queryExecution.sparkPlan.treeString)*/ 43 | 44 | val map: Map[Int, Seq[RCompressed]] = df1.collect().toMap 45 | 46 | assert(map(1).size == 2) 47 | } 48 | 49 | } 50 | 51 | case class R(id: Int, a: String, b: Int) 52 | 53 | case class RCompressed(id: Int, a: String, b: Int, compressDumpDts: Seq[String]) 54 | -------------------------------------------------------------------------------- /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version = 1.2.8 2 | -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | addSbtPlugin("com.geirsson" % "sbt-scalafmt" % "1.5.1") 2 | addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.9.1") 3 | addSbtPlugin("com.jsuereth" % "sbt-pgp" % "2.0.0-M2") 4 | addSbtPlugin("org.xerial.sbt" % "sbt-sonatype" % "2.3") 5 | addSbtPlugin("com.eed3si9n" % "sbt-buildinfo" % "0.9.0") 6 | addSbtPlugin("org.scoverage" % "sbt-scoverage" % "1.5.1") 7 | addSbtPlugin("ch.epfl.scala" % "sbt-release-early" % "2.1.1") 8 | addSbtPlugin("com.47deg" % "sbt-microsites" % "0.9.0") 9 | addSbtPlugin("ch.epfl.scala" % "sbt-scalafix" % "0.9.4") 10 | addSbtPlugin("pl.project13.scala" % "sbt-jmh" % "0.3.4") 11 | -------------------------------------------------------------------------------- /scalafix/build.sbt: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | lazy val V = _root_.scalafix.sbt.BuildInfo 5 | inThisBuild( 6 | List( 7 | organization := "io.univalence", 8 | homepage := Some(url("https://github.com/com/example")), 9 | licenses := List("Apache-2.0" -> url("http://www.apache.org/licenses/LICENSE-2.0")), 10 | developers := List( 11 | Developer( 12 | "ahoy-jon", 13 | "Jonathan WINANDY", 14 | "jonathan@univalence.io", 15 | url("https://univalence.io") 16 | ) 17 | ), 18 | scalaVersion := V.scala211, 19 | addCompilerPlugin(scalafixSemanticdb), 20 | scalacOptions ++= List( 21 | "-Yrangepos" 22 | ) 23 | ) 24 | ) 25 | 26 | skip in publish := true 27 | 28 | lazy val rules = project.settings( 29 | moduleName := "scalafix", 30 | libraryDependencies += "ch.epfl.scala" %% "scalafix-core" % V.scalafixVersion 31 | ) 32 | 33 | lazy val input = project.settings( 34 | skip in publish := true, 35 | libraryDependencies += "io.univalence" %% "fenek" % "0.2" 36 | ) 37 | 38 | 39 | lazy val output = project.settings( 40 | skip in publish := true, 41 | libraryDependencies += "io.univalence" %% "fenek" % "0.2+156-76b5475c" 42 | ) 43 | 44 | 45 | lazy val tests = project 46 | .settings( 47 | skip in publish := true, 48 | libraryDependencies += "ch.epfl.scala" % "scalafix-testkit" % V.scalafixVersion % Test cross CrossVersion.full, 49 | compile.in(Compile) := 50 | compile.in(Compile).dependsOn(compile.in(input, Compile)).value, 51 | scalafixTestkitOutputSourceDirectories := 52 | sourceDirectories.in(output, Compile).value, 53 | scalafixTestkitInputSourceDirectories := 54 | sourceDirectories.in(input, Compile).value, 55 | scalafixTestkitInputClasspath := 56 | fullClasspath.in(input, Compile).value, 57 | ) 58 | .dependsOn(rules) 59 | .enablePlugins(ScalafixTestkitPlugin) 60 | -------------------------------------------------------------------------------- /scalafix/input/src/main/scala/fix/Fenek2to3All.scala: -------------------------------------------------------------------------------- 1 | /* 2 | rules = [ 3 | "Fenek2to3Import" 4 | "Fenek2to3Path" 5 | "Fenek2to3Rest" 6 | ] 7 | */ 8 | package fix 9 | 10 | import io.univalence.fenek._ 11 | import io.univalence.fenek.Fnk._ 12 | import TypedExpr._ 13 | 14 | object Fenek2to3All { 15 | 16 | //Types 17 | type T1 = Expr 18 | type T2 = TypedExpr[String] 19 | 20 | //The struct syntax 21 | val expr: Expr = ??? 22 | val somestruct: Struct = struct(abc = expr) 23 | 24 | 25 | //The >.syntax 26 | val expr2: TypedExpr[String] = >.toto.>.tata.>.titi.as[String] |> (_ + "!") 27 | 28 | 29 | //The caseWhen syntax 30 | val expr3: TypedExpr[Int] = expr caseWhen (1 -> 2 | 2 -> 3) 31 | 32 | 33 | val dm = >.dm 34 | 35 | val code = dm.>.pack.>.code 36 | 37 | val combo: Struct = struct(abc = >.toto) 38 | 39 | type T3 = Fnk.Expr 40 | } 41 | -------------------------------------------------------------------------------- /scalafix/output/src/main/scala/fix/Fenek2to3All.scala: -------------------------------------------------------------------------------- 1 | package fix 2 | 3 | import io.univalence.fenek._ 4 | import io.univalence.fenek.Expr 5 | import io.univalence.fenek.Expr._ 6 | import io.univalence.typedpath._ 7 | 8 | object Fenek2to3All { 9 | 10 | //Types 11 | type T1 = Expr[Any] 12 | type T2 = Expr[String] 13 | 14 | //The struct syntax 15 | val expr: Expr[Any] = ??? 16 | val somestruct: Struct = struct("abc" <<- expr) 17 | 18 | 19 | //The >.syntax 20 | val expr2: Expr[String] = key"toto.tata.titi".as[String] |> (_ + "!") 21 | 22 | 23 | //The caseWhen syntax 24 | val expr3: Expr[Int] = expr caseWhen (1 -> 2 , 2 -> 3) 25 | 26 | 27 | val dm = key"dm" 28 | 29 | val code = key"$dm.pack.code" 30 | 31 | val combo: Struct = struct("abc" <<- key"toto") 32 | 33 | type T3 = Expr[Any] 34 | } 35 | -------------------------------------------------------------------------------- /scalafix/project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version=1.2.8 2 | -------------------------------------------------------------------------------- /scalafix/project/plugins.sbt: -------------------------------------------------------------------------------- 1 | resolvers += Resolver.sonatypeRepo("releases") 2 | addSbtPlugin("ch.epfl.scala" % "sbt-scalafix" % "0.9.4") 3 | -------------------------------------------------------------------------------- /scalafix/readme.md: -------------------------------------------------------------------------------- 1 | # Scalafix rules for Fenek 2 | 3 | To develop rules: 4 | ``` 5 | sbt ~tests/test 6 | # edit rules/src/main/scala/fix/Fenek2to3.scala 7 | ``` 8 | -------------------------------------------------------------------------------- /scalafix/rules/src/main/resources/META-INF/services/scalafix.v1.Rule: -------------------------------------------------------------------------------- 1 | fix.Fenek2to3Path 2 | fix.Fenek2to3Rest 3 | fix.Fenek2to3Import -------------------------------------------------------------------------------- /scalafix/rules/src/main/scala/fix/Fenek2to3.scala: -------------------------------------------------------------------------------- 1 | package fix 2 | 3 | import scalafix.v1._ 4 | 5 | import scala.collection.immutable 6 | import scala.meta._ 7 | 8 | object SelectField { 9 | 10 | def unapply(tree: Tree): Option[Vector[String]] = 11 | tree match { 12 | case Term.Select(Term.Name(">"), Term.Name(arg)) => Some(Vector(arg)) 13 | case Term.Select(Term.Select(SelectField(args), Term.Name(">")), Term.Name(arg)) => Some(args :+ arg) 14 | case _ => None 15 | } 16 | } 17 | 18 | object PathFrom { 19 | 20 | def unapply(tree: Tree): Option[(String, Vector[String])] = 21 | tree match { 22 | case Term.Select(Term.Select(Term.Name(pos), Term.Name(">")), Term.Name(arg)) => 23 | Some((pos, Vector(arg))) 24 | case Term.Select(Term.Select(PathFrom(pos, args), Term.Name(">")), Term.Name(arg)) => 25 | Some((pos, args :+ arg)) 26 | case _ => None 27 | } 28 | } 29 | 30 | class Fenek2to3Path extends SemanticRule("Fenek2to3Path") { 31 | override def fix(implicit doc: SemanticDocument): Patch = { 32 | 33 | val pathRewrite: Seq[Patch] = doc.tree 34 | .collect({ 35 | case s @ SelectField(args) => 36 | (s.pos, Patch.replaceTree(s, args.mkString("path\"", ".", "\""))) 37 | 38 | case p @ PathFrom(root, args) => 39 | (p.pos, Patch.replaceTree(p, args.mkString("path\"$" + root + ".", ".", "\""))) 40 | }) 41 | .groupBy(_._1.start) 42 | .map(_._2.maxBy(_._1.text.length)._2) 43 | .toSeq 44 | 45 | pathRewrite.fold(Patch.empty)(_ + _) 46 | } 47 | 48 | } 49 | 50 | class Fenek2to3Import extends SemanticRule("Fenek2to3Import") { 51 | override def fix(implicit doc: SemanticDocument): Patch = { 52 | val patches: Seq[Seq[Patch]] = doc.tree.collect({ 53 | case i @ Importer(ref, xs) if ref.toString.contains("Fnk") || ref.toString.contains("TypedExpr") => 54 | xs.map(Patch.removeImportee) 55 | }) 56 | 57 | if (patches.nonEmpty) { 58 | patches.flatten.reduce(_ + _) + 59 | Patch.addGlobalImport(importer"io.univalence.fenek.Expr._") + 60 | Patch.addGlobalImport(importer"io.univalence.typedpath._") + 61 | Patch.addGlobalImport(importer"io.univalence.fenek.Expr") 62 | } else Patch.empty 63 | } 64 | } 65 | 66 | class Fenek2to3Rest extends SemanticRule("Fenek2to3Rest") { 67 | 68 | override def fix(implicit doc: SemanticDocument): Patch = { 69 | //println("Tree.structureLabeled: " + doc.tree.structureLabeled) 70 | 71 | val patches: Seq[Seq[Patch]] = doc.tree.collect({ 72 | 73 | case t @ Type.Name("Expr") => 74 | Seq(Patch.replaceTree(t, "Expr[Any]")) 75 | 76 | case t @ Type.Apply(Type.Name("TypedExpr"), x :: Nil) => 77 | Seq(Patch.replaceTree(t, s"Expr[$x]")) 78 | 79 | 80 | case t @ Type.Select(Term.Name("Fnk"), _) => 81 | Seq(Patch.removeTokens(t.tokens.take(2))) 82 | 83 | case s @ Term.Apply(Term.Name("struct"), args) if args.forall(x => x.isInstanceOf[Term.Assign]) => 84 | val assigns = args.asInstanceOf[Seq[Term.Assign]] 85 | assigns.map({ 86 | case t @ Term.Assign(n @ Term.Name(_), rhs) => 87 | val equalToken = t.tokens.tokens.drop(n.tokens.end).find(_.text == "=").get 88 | Patch.addLeft(n, "\"") + 89 | Patch.addRight(n, "\"") + 90 | Patch.removeToken(equalToken) + 91 | Patch.addLeft(rhs, "<<- ") 92 | }) 93 | 94 | case Term.ApplyInfix(_, p @ Term.Name("|"), _, _) => 95 | Seq(Patch.replaceTree(p, ",")) 96 | }) 97 | 98 | //println("Tree.syntax: " + doc.tree.syntax) 99 | //println("Tree.structure: " + doc.tree.structure) 100 | 101 | 102 | patches.flatten.fold(Patch.empty)(_ + _) 103 | } 104 | 105 | } 106 | -------------------------------------------------------------------------------- /scalafix/tests/src/test/scala/fix/RuleSuite.scala: -------------------------------------------------------------------------------- 1 | package fix 2 | 3 | import scalafix.testkit.SemanticRuleSuite 4 | 5 | class RuleSuite extends SemanticRuleSuite() { 6 | runAllTests() 7 | } 8 | -------------------------------------------------------------------------------- /schema/src/main/scala/io/univalence/schema/Schema.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.schema 2 | 3 | import io.univalence.typedpath.{FieldKey, KeyOrRoot, Root} 4 | import org.apache.spark.sql.types.DataType 5 | import org.apache.spark.sql.{DataFrame, Dataset} 6 | 7 | import scala.util.Try 8 | 9 | sealed trait StrExp { 10 | def exp: String 11 | } 12 | final case class SingleExp(exp: String) extends StrExp 13 | 14 | final case class StructExp(fieldExps: Seq[(StrExp, String)]) extends StrExp { 15 | override def exp: String = fieldExps.map(x => x._1.exp + " as " + x._2).mkString("struct(", ", ", ")") 16 | def asProjection: String = fieldExps.map(x => x._1.exp + " as " + x._2).mkString(", ") 17 | } 18 | 19 | 20 | object Schema { 21 | 22 | type Tx = Dataset[_] => Try[DataFrame] 23 | 24 | def move(from: KeyOrRoot, to: KeyOrRoot):Tx = { 25 | (from, to) match { 26 | case (FieldKey(a,Root), FieldKey(b,Root)) => 27 | df => Try(df.withColumnRenamed(a,b)) 28 | } 29 | } 30 | 31 | case class Point(dt: DataType, ref:String) 32 | 33 | def transformAtPath(target: KeyOrRoot, tx: Point => StrExp): Tx = ??? 34 | 35 | 36 | 37 | } 38 | 39 | 40 | 41 | -------------------------------------------------------------------------------- /schema/src/test/scala/io/univalence/schema/SchemaTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.schema 2 | 3 | import io.univalence.sparktest.SparkTest 4 | import org.scalatest.FunSuite 5 | import io.univalence.typedpath._ 6 | 7 | class SchemaTest extends FunSuite with SparkTest { 8 | 9 | test("testMove") { 10 | 11 | val m = Schema.move(key"a", key"b").andThen(_.get) 12 | 13 | def check(in: String, out: String): Unit = 14 | m(dataframe(in)).assertEquals(dataframe(out)) 15 | 16 | check("{a:1}", "{b:1}") 17 | check("{a:1, c:0}", "{b:1, c:0}") 18 | } 19 | 20 | } 21 | -------------------------------------------------------------------------------- /site/README.md: -------------------------------------------------------------------------------- 1 | 2 | # Local setup 3 | 4 | You will need to install Jekyll, which is a ruby framework that you can 5 | get by using `gem` (the Ruby dependency manager). 6 | To do so, you first need to install Ruby. 7 | On OSX platform, the default Ruby with download on incompatible version 8 | of Jekyll. So use Homebrew to get a better version. 9 | 10 | $ brew install ruby 11 | 12 | then use `gem` to get Jekyll. 13 | 14 | $ sudo gem install jekyll 15 | 16 | # Build 17 | 18 | To build the Web site: 19 | 20 | $ sbt makeMicrosite 21 | 22 | # Local visualization 23 | 24 | Once the Web site is built, go to `site/target/site` and launch Jekyll. 25 | 26 | $ jekyll serve 27 | 28 | The Web site will open on http://127.0.0.1:4000/spark-tools/. 29 | 30 | # Publish 31 | 32 | $ sbt publishMicrosite 33 | -------------------------------------------------------------------------------- /site/project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version=1.2.8 2 | -------------------------------------------------------------------------------- /site/src/main/resources/microsite/css/custom.css: -------------------------------------------------------------------------------- 1 | .jumbotron { 2 | background: #000000 url("../img/jumbotron_pattern.png") no-repeat center bottom; 3 | background-size:cover; 4 | padding-top: 176px; 5 | padding-bottom: 90px; 6 | } -------------------------------------------------------------------------------- /site/src/main/resources/microsite/data/menu.yml: -------------------------------------------------------------------------------- 1 | options: 2 | - title: Spark-Test 3 | url: spark-test/ 4 | menu_section: spark-test 5 | menu_type: spark-test 6 | 7 | - title: Parka 8 | url: parka 9 | menu_type: parka 10 | menu_section: parka-parka 11 | 12 | - title: Plumbus 13 | url: plumbus/ 14 | menu_section: plumbus 15 | menu_type: plumbus 16 | 17 | - title: Fenek 18 | url: fenek/ 19 | menu_section: fenek 20 | menu_type: fenek 21 | 22 | - title: Spark-ZIO 23 | url: spark-zio/ 24 | menu_section: spark-zio 25 | menu_type: spark-zio 26 | 27 | ########################### 28 | # Spark Zio Menu Options # 29 | ########################### 30 | - 31 | - title: Getting Started 32 | url: spark-zio/getting_started 33 | menu_type: spark-zio 34 | menu_section: spark-zio-getting-started 35 | 36 | ########################### 37 | # Spark Test Menu Options # 38 | ########################### 39 | 40 | - title: Getting Started 41 | url: spark-test/getting_started 42 | menu_type: spark-test 43 | menu_section: spark-test-getting-started 44 | 45 | - title: Overview 46 | url: spark-test/overview 47 | menu_type: spark-test 48 | menu_section: spark-test-overview 49 | 50 | - title: Features 51 | url: spark-test/features 52 | menu_type: spark-test 53 | menu_section: spark-test-features 54 | 55 | ########################### 56 | # Parka Menu Options # 57 | ########################### 58 | 59 | - title: Structures 60 | url: parka/parka_structures 61 | menu_type: parka 62 | menu_section: parka-structures 63 | nested_options: 64 | - title: Inner 65 | url: parka/parka_structures_inner 66 | - title: Outer 67 | url: parka/parka_structures_outer 68 | - title: Describe 69 | url: parka/parka_structures_describe 70 | - title: Delta 71 | url: parka/parka_structures_delta 72 | 73 | - title: Interpretation 74 | url: parka/parka_printer 75 | menu_type: parka 76 | menu_section: parka-interpretation 77 | nested_options: 78 | - title: Printer 79 | url: parka/parka_printer 80 | - title: Serde 81 | url: parka/parka_serde 82 | 83 | - title: Q&A 84 | url: parka/parka_question 85 | menu_type: parka 86 | menu_section: parka-question -------------------------------------------------------------------------------- /site/src/main/resources/microsite/img/favicon.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/univalence/spark-tools/12a158ea185f77156624bd5797316141461b514b/site/src/main/resources/microsite/img/favicon.png -------------------------------------------------------------------------------- /site/src/main/resources/microsite/img/favicon16x16.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/univalence/spark-tools/12a158ea185f77156624bd5797316141461b514b/site/src/main/resources/microsite/img/favicon16x16.png -------------------------------------------------------------------------------- /site/src/main/resources/microsite/img/favicon32x32.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/univalence/spark-tools/12a158ea185f77156624bd5797316141461b514b/site/src/main/resources/microsite/img/favicon32x32.png -------------------------------------------------------------------------------- /site/src/main/resources/microsite/img/favicon64x64.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/univalence/spark-tools/12a158ea185f77156624bd5797316141461b514b/site/src/main/resources/microsite/img/favicon64x64.png -------------------------------------------------------------------------------- /site/src/main/resources/microsite/img/jumbotron_pattern.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/univalence/spark-tools/12a158ea185f77156624bd5797316141461b514b/site/src/main/resources/microsite/img/jumbotron_pattern.png -------------------------------------------------------------------------------- /site/src/main/resources/microsite/img/navbar_brand.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/univalence/spark-tools/12a158ea185f77156624bd5797316141461b514b/site/src/main/resources/microsite/img/navbar_brand.png -------------------------------------------------------------------------------- /site/src/main/resources/microsite/img/navbar_brand2x.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/univalence/spark-tools/12a158ea185f77156624bd5797316141461b514b/site/src/main/resources/microsite/img/navbar_brand2x.png -------------------------------------------------------------------------------- /site/src/main/resources/microsite/img/sidebar_brand.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/univalence/spark-tools/12a158ea185f77156624bd5797316141461b514b/site/src/main/resources/microsite/img/sidebar_brand.png -------------------------------------------------------------------------------- /site/src/main/resources/microsite/img/sidebar_brand2x.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/univalence/spark-tools/12a158ea185f77156624bd5797316141461b514b/site/src/main/resources/microsite/img/sidebar_brand2x.png -------------------------------------------------------------------------------- /site/src/main/tut/fenek/index.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | position: 5 4 | section: fenek 5 | title: "Fenek" 6 | --- 7 | 8 | # {{page.title}} 9 | -------------------------------------------------------------------------------- /site/src/main/tut/index.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: home 3 | position: 1 4 | section: home 5 | title: "Home" 6 | --- 7 | 8 | # {{page.title}} 9 | 10 | Spark-Tools is a set of tools dedicated to providing more clarity for data engineers when working on Spark jobs. 11 | 12 | Our tools were created based on the usage of meta-data, which allows for greater visibility on code structures. This in turn provides more context when something breaks down and needs fixing. 13 | 14 | Imagine you're stumbling through your room in the dark and walk straight onto a piece of Lego; you curse, turn on the light and see that piece of Lego. Spark-Tools is that light: it won't stop the pain, but at least you know what hurt you. 15 | 16 | ## Getting started with Spark-Test 17 | 18 | Spark-Test is the tool that will improve your test reports. 19 | 20 | We recommend you to start using Spark-Tools with Spark-Test, because it's the most accessible tool. Here's an example of how to use Spark-Test from A to Z. 21 | 22 | Include [Spark-Test](./spark-test) in your project by implementing the following lines inside your build sbt: 23 | 24 | ```scala 25 | resolvers += "spark-test" at "http://dl.bintray.com/univalence/univalence-jvm" 26 | 27 | libraryDependencies += "io.univalence" %% "spark-test" % "0.2+245-09a064d9" % Test 28 | ``` 29 | 30 | Spark-Test provides an `assertEquals` function which compares two RDDs, Datasets or Dataframes. It returns an `SparkTestError` 31 | if they are different. 32 | 33 | ```scala 34 | import io.univalence.sparktest.SparkTest 35 | import org.scalatest.FunSuiteLike 36 | 37 | class MyTestClass extends FunSuiteLike with SparkTest { 38 | test("some test"){ 39 | case class A(a:Int) 40 | 41 | val df = dataframe("{a:1, b:true}", "{a:2, b:false}") 42 | val ds = dataset(A(1), A(3)) 43 | 44 | df.assertEquals(ds) 45 | } 46 | } 47 | ``` 48 | 49 | ```scala 50 | java.lang.AssertionError: The data set content is different : 51 | in field a, 2 is not equals to expected value 3 52 | dataframe("{ a: 2 , b: false}") 53 | dataframe("{ a: 3 }") 54 | ``` 55 | 56 | There are many other features! To learn more about Spark-Test, see the [Spark-Test documentation](./spark-test). 57 | 58 | ## The tools 59 | 60 | Each tools are open source and available on Github 61 | 62 | * [Spark-test](https://github.com/univalence/spark-tools/tree/master/spark-test), testing tools for Spark 63 | * [Parka](https://github.com/univalence/spark-tools/tree/master/parka), a tool that applies deltaQA for Datasets 64 | * [Plumbus](https://github.com/univalence/spark-tools/tree/master/plumbus), light misc things for Spark 65 | * [Fenek](https://github.com/univalence/spark-tools/tree/master/fenek), a DSL for *semi-typed* transformation in Scala for Spark 66 | * [Spark-ZIO](https://github.com/univalence/spark-tools/tree/master/spark-zio), Spark in ZIO environment 67 | -------------------------------------------------------------------------------- /site/src/main/tut/parka/index.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | position: 3 4 | section: parka 5 | title: "Parka" 6 | --- 7 | 8 | Parka 9 | ====================== 10 | 11 | Parka is a library about data quality of a Datasets in Scala. 12 | 13 | It implements DeltaQA for Datasets, comparing two Datasets to each other and notifying differences into Parka Analysis which is an object that contains the comparison’s data.. 14 | 15 | ## Table of content 16 | 17 | - [Installation](#installation) 18 | - [Usage](#usage) 19 | - [Support](#support) 20 | - [Authors](#authors) 21 | - [License](#license) 22 | - [Dependencies](#dependencies) 23 | - [Links](#links) 24 | 25 | ## Installation 26 | 27 | ### Stable version 28 | 29 | A stable version isn't available yet. 30 | 31 | ### Latest version 32 | 33 | If you want to get the very last version of this library you can still download it using bintray here : https://bintray.com/univalence/univalence-jvm/parka 34 | 35 | Here is an example using ```version 0.3+79-4936e981``` that work with ```scala 2.11.X```: 36 | 37 | ```scala 38 | resolvers += "parka" at "http://dl.bintray.com/univalence/univalence-jvm" 39 | libraryDependencies += "io.univalence" %% "parka" % "0.3+79-4936e981" 40 | ``` 41 | 42 | ## Usage 43 | 44 | The entry of Parka is the Parka Analysis object, this object contains a lot of information about the comparison between two Datasets which is very important for Data Quality. 45 | 46 | To get Parka Analysis, first import parka and then generate the analysis from two Datasets as below : 47 | 48 | ```scala 49 | import io.univalence.parka.Parka 50 | 51 | val pa: ParkaAnalysis = Parka(df1, d2f)("key") 52 | ``` 53 | 54 | First give the two Datasets to compare to and then column(s) that are keys. 55 | the console or export it in JSON. 56 | 57 | Here is an example : 58 | 59 | ```scala 60 | import io.univalence.parka.Printer 61 | 62 | println(Printer.printParkaResult(pa.result)) 63 | ``` 64 | ## Support 65 | 66 | If you have any problem/question don't hesitate to add a new issue. 67 | 68 | ## Authors 69 | 70 | Made with :heart: by Univalence's team. 71 | 72 | ## License 73 | 74 | Parka is licensed under the Apache License, Version 2.0 (the “License”); you may not use this software except in compliance with the License. 75 | 76 | 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. 77 | 78 | ## Dependencies 79 | 80 | * [algebird](https://github.com/twitter/algebird) - 0.13.4 81 | * [magnolia](https://github.com/propensive/magnolia) - 0.10.0 82 | * [jline](https://github.com/jline/jline3) - 3.12.1 83 | * [circe](https://github.com/circe/circe) - 0.11.1 84 | * [spark-test](https://github.com/univalence/spark-tools/tree/master/spark-test) - current 85 | 86 | ## Links 87 | 88 | * [Univalence Web site](https://www.univalence.io/) 89 | * [Microsite](https://univalence.github.io/spark-tools/parka/) 90 | * [Source code](https://github.com/univalence/spark-tools/tree/master/parka/src/main/scala/io/univalence/parka) 91 | * [Video](https://www.youtube.com/watch?v=t24sUF2zWLY) - DeltaQA introduction between [14:25](http://www.youtube.com/watch?v=t24sUF2zWLY&t=14m25s) and [28:10](http://www.youtube.com/watch?v=t24sUF2zWLY&t=28m10s) 92 | 93 | -------------------------------------------------------------------------------- /site/src/main/tut/parka/parka_analysis.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | section: parka 4 | title: "Parka Analysis" 5 | --- 6 | 7 | # Parka Analysis 8 | 9 | Everything in Parka start by generating a Parka Analysis case class. 10 | 11 | This class contains all the information about the data quality of our data between our two Datasets. 12 | 13 | To obtain our Parka Analysis we first need two Datasets to compare to each other. Both Datasets one or more keys 14 | 15 | Then just import Parka and use it as below : 16 | 17 | ```scala 18 | import io.univalence.parka.Parka 19 | 20 | val pa: ParkaAnalysis = Parka(df1, d2f)("key") 21 | ``` 22 | 23 | As we said before, a Parka Analysis contains a lot of information. 24 | 25 | ````scala 26 | case class ParkaAnalysis(datasetInfo: Both[DatasetInfo], result: ParkaResult) 27 | ```` 28 | 29 | ## Dataset Info 30 | 31 | DatasetInfo isn't implemented yet. 32 | 33 | ## Parka Result 34 | 35 | ParkaResult contains two type of analysis, an inner and an outer analysis. 36 | 37 | The difference between them is simple, if the Datasets on the right and the Datasets on the left have a row with the same key(s) then this row will be part of the inner analysis. 38 | If there is a row with a particular key only on the right or on the left Datasets then this row will be part of the outer analysis. -------------------------------------------------------------------------------- /site/src/main/tut/parka/parka_analysis_datasetinfo.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | section: parka 4 | title: "Dataset Information" 5 | --- 6 | 7 | # Dataset Information 8 | 9 | DatasetInfo isn't implemented yet. -------------------------------------------------------------------------------- /site/src/main/tut/parka/parka_analysis_inner.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | section: parka 4 | title: "Inner Analysis" 5 | --- 6 | 7 | # Inner Analysis 8 | 9 | Inner contains meaningful information about rows with keys on both Datasets. 10 | 11 | It contains the number of similar rows and different rows. Also for each different keys the number of difference and a big part of Inner is about comparison between similar rows for each columns. 12 | 13 | This comparison contains the number of similar rows and different rows for a particular key and the repartition of both Datasets for this key. -------------------------------------------------------------------------------- /site/src/main/tut/parka/parka_analysis_outer.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | section: parka 4 | title: "Outer Analysis" 5 | --- 6 | 7 | # Outer Analysis 8 | 9 | Outer contains meaningful information about added or removed key from the right to the left Datasets. 10 | 11 | It contains the number of unique key's rows for each Datasets and additional information about these rows for each column. 12 | 13 | Basically we provide a map where keys are column names (excepted keys) and values are a Describe object containing repartition information which is slightly different according to the DataType of the column. 14 | -------------------------------------------------------------------------------- /site/src/main/tut/parka/parka_printer.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | section: parka 4 | title: "Parka Printer" 5 | --- 6 | 7 | # Parka Printer 8 | 9 | When you obtain your Parka Analysis, you can analyse it directly on the console using the Parka Printer. 10 | 11 | ```scala 12 | import io.univalence.parka.{Parka, ParkaPrinter} 13 | 14 | val pa = Parka(df1, d2f)("key") 15 | println(ParkaPrinter.printParkaResult(pa.result)) 16 | ``` 17 | 18 | On a simple Parka Analysis, you can obtain the following result : 19 | 20 | ```text 21 | Parka Result: 22 | Inner: 23 | Number of equal row: 0 24 | Number of different row: 10 25 | Differences by sequence of keys: Key (value) has 10 occurrences 26 | Delta by key: 27 | value: 28 | Number of similarities: 0 29 | Number of differences: 10 30 | Describes: 31 | Left: Right: 32 | Histogram: Histogram: 33 | 100,00 | oooooooooooooooooooooo 2 0,00 | ooooooooooo 1 34 | 100,20 | oooooooooooooooooooooo 2 18,20 | oooooooooooooooooooooo 2 35 | 100,40 | oooooooooooooooooooooo 2 36,40 | oooooooooooooooooooooo 2 36 | 100,60 | oooooooooooooooooooooo 2 54,60 | oooooooooooooooooooooo 2 37 | 100,80 | oooooooooooooooooooooo 2 72,80 | oooooooooooooooooooooo 2 38 | 101,00 | oooooooooooooooooooooo 2 91,00 | ooooooooooo 1 39 | value: 40 | 10,00 | ooooooooooo 1 41 | 28,20 | oooooooooooooooooooooo 2 42 | 46,40 | oooooooooooooooooooooo 2 43 | 64,60 | oooooooooooooooooooooo 2 44 | 82,80 | oooooooooooooooooooooo 2 45 | 101,00 | ooooooooooo 1 46 | Outer: 47 | Number of unique row on the left dataset: 0 48 | Number of unique row on the right dataset: 1 49 | Describe by key: 50 | value: 51 | Describes: 52 | Left: Right: 53 | Histogram: 54 | 100,00 | oooooooooooooooooooooo 1 55 | 100,20 | 0 56 | 100,40 | 0 57 | 100,60 | 0 58 | 100,80 | 0 59 | 101,00 | 0 60 | ``` -------------------------------------------------------------------------------- /site/src/main/tut/parka/parka_question.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | section: parka 4 | title: "Parka Question" 5 | --- 6 | 7 | # Questions & Answers 8 | -------------------------------------------------------------------------------- /site/src/main/tut/parka/parka_serde.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | section: parka 4 | title: "Parka Serde" 5 | --- 6 | 7 | # Parka Serde 8 | 9 | We saw that we can create Parka Analysis from two Datasets, but you can also import a Parka Analysis from a JSON or import a Parka Analysis to JSON. 10 | 11 | ## Export to JSON 12 | 13 | To generate JSON from Parka Analysis you need to follow the example below: 14 | 15 | ```scala 16 | import io.univalence.parka.{Parka, ParkaAnalysisSerde} 17 | 18 | val pa = Parka(df1, d2f)("key") 19 | val paJson = ParkaAnalysisSerde.toJson(pa) 20 | ``` 21 | 22 | ## Import from JSON 23 | 24 | To generate Parka Analysis from JSON you need to follow the example below: 25 | 26 | ```scala 27 | import io.univalence.parka.{Parka, ParkaAnalysisSerde} 28 | 29 | val paJson = """{"datasetInfo":{"left":{"source":[],"nStage":0},"right":{"source":[],"nStage":0}},"result":{"inner":{"countRowEqual":1,"countRowNotEqual":1,"countDiffByRow":[{"key":["n"],"value":1},{"key":[],"value":1}],"byColumn":{"n":{"DeltaLong":{"nEqual":1,"nNotEqual":1,"describe":{"left":{"value":{"neg":null,"countZero":0,"pos":{"_1":0,"_2":2,"_3":2,"_4":{"_1":0,"_2":1,"_3":1,"_4":null,"_5":{"_1":1,"_2":0,"_3":1,"_4":null,"_5":null}},"_5":{"_1":1,"_2":1,"_3":1,"_4":{"_1":2,"_2":0,"_3":1,"_4":null,"_5":null},"_5":null}}}},"right":{"value":{"neg":null,"countZero":0,"pos":{"_1":0,"_2":2,"_3":2,"_4":{"_1":0,"_2":1,"_3":1,"_4":null,"_5":{"_1":1,"_2":0,"_3":1,"_4":null,"_5":null}},"_5":{"_1":1,"_2":1,"_3":1,"_4":null,"_5":{"_1":3,"_2":0,"_3":1,"_4":null,"_5":null}}}}}},"error":{"neg":{"_1":1,"_2":0,"_3":1,"_4":null,"_5":null},"countZero":1,"pos":null}}}}},"outer":{"countRow":{"left":0,"right":0},"byColumn":{}}}}""" 30 | val pa = ParkaAnalysisSerde.fromJson(paJson).right.get 31 | ``` -------------------------------------------------------------------------------- /site/src/main/tut/parka/parka_structures.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | section: parka 4 | title: "Parka Structures" 5 | --- 6 | 7 | # Parka Structures 8 | 9 | This section refers to the structures of a Parka Analysis. As we said before, a Parka Analysis contains a lot of information, this Structure Part will help you to understand each kind of structures present on a Parka Analysis. 10 | 11 | Everything start with a Parka Analysis as below : 12 | 13 | ````scala 14 | case class ParkaAnalysis(datasetInfo: Both[DatasetInfo], result: ParkaResult) 15 | ```` 16 | 17 | The left part, datasetInfo, isn't implemented yet. 18 | 19 | The right part, result, contains two type of analysis, an inner and an outer analysis. 20 | 21 | The difference between them is simple: 22 | - if the Datasets on the right and the Datasets on the left have a row with the same key(s) then this row will be part of the inner analysis. 23 | - if there is a row with a particular key only on the right or on the left Datasets then this row will be part of the outer analysis. 24 | 25 | ## Global data structure 26 | 27 | Here is the global structure of a Parka Analysis: 28 | 29 | - datasetInfo: Both[DatasetInfo] 30 | - Not Implemented 31 | - resul: ParkaResult 32 | - inner: Inner 33 | - countRowEqual: Long 34 | - countRowNotEqual: Long 35 | - countDeltaByRow: Map[Set[String], DeltaByRow] 36 | - equalRows: DescribeByRow 37 | - byColumn: Map[String, Delta] 38 | - outer: Outer 39 | - both: Both[DescribeByRow] 40 | 41 | ## Related link(s) 42 | 43 | https://github.com/univalence/spark-tools/blob/master/parka/src/main/scala/io/univalence/parka/Structure.scala 44 | -------------------------------------------------------------------------------- /site/src/main/tut/parka/parka_structures_delta.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | section: parka 4 | title: "Parka Structures" 5 | --- 6 | 7 | # Delta 8 | 9 | Delta is a structure which describe a sequence of value from both Datasets accompaninied by an error description. 10 | 11 | ## Structure 12 | 13 | ```scala 14 | case class Delta(nEqual: Long, nNotEqual: Long, describe: Both[Describe], error: Describe) 15 | ``` 16 | 17 | ### nEqual & nNotEqual 18 | 19 | **type**: Long 20 | 21 | They are two counter which tell you how many rows have exactly the same values by incrementing "nEqual" and how many have at least one difference by incrementing "nNotEqual". 22 | 23 | That different from "countRowEqual" and "countRowNotEqual" in Inner since difference or not is usually corresponding to one or a set of columns not every column. 24 | 25 | ### describe 26 | 27 | **type**: Both[Describe] 28 | 29 | Two Describes for values in each Datasets since both are different. 30 | 31 | ### error 32 | 33 | **type**: Describe 34 | 35 | A Describe for the difference between the left and the right Datasets. Generally "right_value - left_value", for special case such as string or array, we use levenshtein to calculate the distance for the moment. 36 | 37 | ## DeltaByRow 38 | 39 | ```scala 40 | case class DeltaByRow(count: Long, byColumn: Map[String, Delta]) 41 | ``` 42 | 43 | When there is a difference between two rows, we use a DeltaByRow instead of a [DescribeByRow](/spark-tools/parka/parka_structures_describe). It allows us to get a Describe for both Datasets since they are different but also a describe for our error. 44 | 45 | DeltaByRow are only available inside Inner analysis and that is totally logic. In outer you can't compare rows since they don't have any equivalent in the other side. 46 | 47 | -------------------------------------------------------------------------------- /site/src/main/tut/parka/parka_structures_describe.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | section: parka 4 | title: "Parka Structures" 5 | --- 6 | 7 | # Describe 8 | 9 | Describe is a structure which describe a sequence of value in different ways. 10 | 11 | ## Structure 12 | 13 | ```scala 14 | case class Describe(count: Long, histograms: Map[String, Histogram], counts: Map[String, Long]) 15 | ``` 16 | 17 | ### count 18 | 19 | **type**: Long 20 | 21 | This is one counter, incrementing each time a value is added to the Describe 22 | 23 | ### histograms 24 | 25 | **type**: Map[String, Histogram] 26 | 27 | Repartition of values can be describe using histograms, so we provide a map containing them. 28 | 29 | Normally, a Describe will contain one histogram containing each value. However in some cases, we can use multiple histograms to describe a same sequence of values. Imagine if we have a sequence of date, then we can have histograms according to days, months and so on for example. 30 | 31 | Another things important about histogram if your amount of data is too large then the histogram will be converted into a [QTree](https://twitter.github.io/algebird/datatypes/approx/q_tree.html) which is an approximate histogram giving us scalability. 32 | 33 | ### counts 34 | 35 | **type**: Map[String, Long] 36 | 37 | This is a map of counter. 38 | 39 | For the moment, there are three kind of counter: nNull, nTrue, nFalse counting the number of null value, the number of true and the number of false added to the Describe. 40 | 41 | ## DescribeByRow 42 | 43 | ```scala 44 | case class DescribeByRow(count: Long, byColumn: Map[String, Describe]) 45 | ``` 46 | 47 | A Describe is always a part of a little more complex structure such as DescribeByRow. 48 | 49 | We usually have a Describe for each column represented here by the "byColumn" argument so we have created a DescribeByRow case class. We have attached a counter that is, for example, usefull to know how much rows are unique inside both Datasets for our Outer analysis. -------------------------------------------------------------------------------- /site/src/main/tut/parka/parka_structures_inner.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | section: parka 4 | title: "Parka Structures" 5 | --- 6 | 7 | # Inner 8 | 9 | Inner contains meaningful information about rows with keys on both Datasets. 10 | It means that in both Datasets, there is at least one row with exactly same set of keys. 11 | 12 | ## Structure 13 | 14 | ```scala 15 | case class Inner(countRowEqual: Long, 16 | countRowNotEqual: Long, 17 | countDeltaByRow: Map[Set[String], DeltaByRow], 18 | equalRows: DescribeByRow) { 19 | @transient lazy val byColumn: Map[String, Delta] = { 20 | val m = implicitly[Monoid[Map[String, Delta]]] 21 | m.combineAll(countDeltaByRow.map(_._2.byColumn).toSeq :+ equalRows.byColumn.mapValues(d => { 22 | Delta(d.count, 0, Both(d, d), Describe.empty) 23 | })) 24 | } 25 | } 26 | ``` 27 | 28 | ### countRowEqual & countRowNotEqual 29 | 30 | **type**: Long 31 | 32 | They are two counter which tell you how many rows have exactly the same values by incrementing "countRowEqual" and how many have at least one difference by incrementing "countRowNotEqual". 33 | 34 | ### countDeltaByRow 35 | 36 | **type**: Map[Set[String], DeltaByRow] 37 | 38 | Rows with at least one difference are are attached to the "countDeltaByRow". It is a map which is composed by a set of String as a key. Each key is composed by columns where differences happens. 39 | 40 | Nothing is better than an example, imagine the following Datasets : 41 | 42 | - Left Dataset : 43 | 44 | | Key | Col1 | Col2 | 45 | | ------------- |---------------| ------| 46 | | key1 | a | 1 | 47 | | key2 | b | 2 | 48 | | key3 | c | 3 | 49 | | key4 | d | 4 | 50 | 51 | - Right Dataset : 52 | 53 | | Key | Col1 | Col2 | 54 | | ------------- |---------------| ------| 55 | | key1 | a | 1 | 56 | | key2 | x | 2 | 57 | | key3 | y | 4 | 58 | | key4 | z | 5 | 59 | 60 | 61 | In this case, you will obtain two elements in that map, one with the key [Col1] containing information about the line n°2 and one with the key [Col1, Col2] containing information about the line n°3 and n°4. Each of them has a [DeltaByRow](/spark-tools/parka/parka_structures_delta) as a value. 62 | 63 | ### equalRows 64 | 65 | **type**: DescribeByRow 66 | 67 | equalRows contains composition of Rows which are exactly equal don't need a DeltaByRow since we will not compare them to each other, we directly can add them to a [DescribeByRow](/spark-tools/parka/parka_structures_describe). 68 | 69 | ### byColumn 70 | 71 | **type**: Map[String, Delta] 72 | 73 | The value "byColumn" is an additional analysis which gives you information in a column oriented manner instead of a row oriented manner. Each column has its own [Delta](/spark-tools/parka/parka_structures_delta) describing how data evolve for that particular row no matter if there is or no a difference. 74 | -------------------------------------------------------------------------------- /site/src/main/tut/parka/parka_structures_outer.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | section: parka 4 | title: "Parka Structures" 5 | --- 6 | 7 | # Outer 8 | 9 | Outer contains meaningful information about added or removed key from the right to the left Datasets.It means that in the set of key appear only in one of both Datasets. 10 | 11 | ## Structure 12 | 13 | ```scala 14 | case class Outer(both: Both[DescribeByRow]) 15 | ``` 16 | 17 | ### both 18 | 19 | **type**: Both[DescribeByRow] 20 | 21 | Both[T] is just a structure that contains T two times accessible using both.left or both.right. 22 | 23 | In this case, there are two [DescribeByRow](/spark-tools/parka/parka_structures_describe), one for the left Datasets and one for the right Datasets. We don't need any [DeltaByRow](/spark-tools/parka/parka_structures_delta) in that case because the key appear only in one Dataset so there isn't any comparison. 24 | -------------------------------------------------------------------------------- /site/src/main/tut/plumbus/index.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | position: 4 4 | section: plumbus 5 | title: "Plumbus" 6 | --- 7 | 8 | # {{page.title}} 9 | -------------------------------------------------------------------------------- /site/src/main/tut/spark-test/features.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | section: spark-test 4 | title: "Features" 5 | --- 6 | 7 | # {{page.title}} 8 | 9 | \[IN PROGRESS] -------------------------------------------------------------------------------- /site/src/main/tut/spark-test/getting_started.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | section: spark-test 4 | title: "Getting started" 5 | --- 6 | 7 | # {{page.title}} 8 | 9 | \[IN PROGRESS] 10 | 11 | Import the library: 12 | ```scala 13 | resolvers += "Spark-tools" at "http://dl.bintray.com/univalence/univalence-jvm" 14 | 15 | // replace version with a suitable one 16 | libraryDependencies += "io.univalence" %% "spark-test" % "0.2+245-09a064d9" % Test 17 | ``` 18 | 19 | ## Create a DataFrame with a JSON string 20 | 21 | We start by importing and extending SparkTest. 22 | ```scala 23 | import io.univalence.sparktest.SparkTest 24 | import org.scalatest.FunSuiteLike 25 | 26 | class MyTestClass extends FunSuiteLike with SparkTest {} 27 | ``` 28 | 29 | Then, we can create our test: 30 | ```scala 31 | class MyTestClass extends FunSuiteLike with SparkTest { 32 | test("create df with json string") { 33 | // create df with json string 34 | val df = dfFromJsonString("{a:1}", "{a:2}") 35 | } 36 | } 37 | ``` 38 | 39 | ## Comparing DataFrames 40 | To compare DataFrames, you can simply call the assertEquals method. It throws an SparkTestError if they are not equal. 41 | 42 | For instance, this: 43 | ```scala 44 | val dfUT = Seq(1, 2, 3).toDF("id") 45 | val dfExpected = Seq(2, 1, 4).toDF("id") 46 | 47 | dfUT.assertEquals(dfExpected) 48 | ``` 49 | ... throws the following exception: 50 | ``` 51 | io.univalence.sparktest.SparkTest$ValueError: The data set content is different : 52 | 53 | in value at id, 2 was not equal to 1 54 | dataframe({id: 1}) 55 | dataframe({id: 2}) 56 | 57 | in value at id, 1 was not equal to 2 58 | dataframe({id: 2}) 59 | dataframe({id: 1}) 60 | 61 | in value at id, 4 was not equal to 3 62 | dataframe({id: 3}) 63 | dataframe({id: 4}) 64 | ``` 65 | 66 | ## Testing with predicates 67 | 68 | One of our test functionality is shouldForAll. 69 | 70 | It throws an AssertionException if there are rows that don't match the predicate. 71 | 72 | This example: 73 | ```scala 74 | val rdd = sc.parallelize(Seq(Person("John", 19), Person("Paul", 17), Person("Emilie", 25), Person("Mary", 5))) 75 | rdd.shouldForAll(p => p.age > 18) // Paul and Mary are too young 76 | ``` 77 | 78 | ... will throw this exception: 79 | ```scala 80 | java.lang.AssertionError: No rows from the dataset match the predicate. Rows not matching the predicate : 81 | Person(Paul,17) 82 | Person(Mary,5) 83 | ``` 84 | 85 | Whereas this example: 86 | ```scala 87 | val rdd = sc.parallelize(Seq(Person("John", 19), Person("Paul", 52), Person("Emilie", 25), Person("Mary", 83))) 88 | rdd.shouldForAll(p => p.age > 18) // Everyone pass the predicate 89 | ``` 90 | ... will pass! -------------------------------------------------------------------------------- /site/src/main/tut/spark-test/index.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | position: 2 4 | section: spark-test 5 | title: "Spark-Test" 6 | --- 7 | 8 | # {{page.title}} 9 | 10 | Spark-Test is a library to help developers test their Spark applications. 11 | 12 | Our goal is to provide you with tools that can help you test your code fast and in an easy way, precisely pointing out why something bad occurred with your Spark collections. 13 | 14 | You just have to extend SparkTest to have all those tools! 15 | -------------------------------------------------------------------------------- /site/src/main/tut/spark-test/overview.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | section: spark-test 4 | title: "Overview" 5 | --- 6 | 7 | # {{page.title}} 8 | 9 | \[IN PROGRESS] -------------------------------------------------------------------------------- /site/src/main/tut/spark-zio/getting_started.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | section: spark-zio 4 | title: "Getting started" 5 | --- 6 | 7 | # {{page.title}} 8 | 9 | ## What you will need 10 | 11 | If you want to use Spark-ZIO, you will need these: 12 | 13 | ```scala 14 | val runtime: DefaultRuntime = new DefaultRuntime {} 15 | val ss: SparkSession = SparkSession.builder.master("local[*]").getOrCreate() 16 | val sparkEnv: SparkZIO = new SparkZIO(ss) 17 | ``` 18 | 19 | You can obviously change the configuration for your runtime and your SparkSession. 20 | 21 | Now that you wield the power of Spark and ZIO together, let's make a simple local Spark job: 22 | 23 | ```scala 24 | import io.univalence.sparkzio.SparkEnv.TaskS 25 | val program: TaskS[Unit] = for { 26 | df <- sparkEnv.read.parquet(pathToParquet) 27 | _ <- Task(df.show()) 28 | } yield () 29 | runtime.unsafeRun(program) 30 | ``` 31 | -------------------------------------------------------------------------------- /site/src/main/tut/spark-zio/index.md: -------------------------------------------------------------------------------- 1 | --- 2 | layout: docs 3 | position: 6 4 | section: spark-zio 5 | title: "Spark-ZIO" 6 | --- 7 | 8 | # {{page.title}} 9 | 10 | Spark-ZIO lets us access common mutable (relatively) Spark operations with ZIO's first type parameter R when working with ZIO IO monads. 11 | -------------------------------------------------------------------------------- /spark-test/README.md: -------------------------------------------------------------------------------- 1 | Spark-Test 2 | ====================== 3 | 4 | [ ![Download](https://api.bintray.com/packages/univalence/univalence-jvm/spark-test/images/download.svg) ](https://bintray.com/univalence/univalence-jvm/spark-test/_latestVersion) 5 | 6 | Spark-Test is a library to help developers test their Spark applications. 7 | 8 | Our goal is to provide you with tools that can help you test your code fast and in an easy way, precisely pointing out why something bad occurred with your Spark collections. 9 | 10 | You just have to extend SparkTest to have all those tools! 11 | 12 | ## Table of content 13 | 14 | - [Installation](#installation) 15 | - [Usage](#usage) 16 | - [Support](#support) 17 | - [Authors](#authors) 18 | - [License](#license) 19 | - [Dependencies](#dependencies) 20 | - [Links](#links) 21 | 22 | ## Installation 23 | 24 | ### Stable version 25 | 26 | Version for scala 2.11.X : 27 | 28 | ```scala 29 | libraryDependencies += "io.univalence" % "spark-test_2.11" % "0.3" % Test 30 | ``` 31 | 32 | Version for scala 2.12.X : 33 | 34 | ```scala 35 | libraryDependencies += "io.univalence" % "spark-test_2.12" % "0.3" % Test 36 | ``` 37 | 38 | ### Latest version 39 | 40 | If you want to get the very last version of this library you can still download it using bintray here : https://bintray.com/univalence/univalence-jvm/spark-test 41 | 42 | Here is an example using ```version 0.3+79-4936e981``` that work with ```scala 2.11.X```: 43 | 44 | ```scala 45 | resolvers += "spark-test" at "http://dl.bintray.com/univalence/univalence-jvm" 46 | libraryDependencies += "io.univalence" %% "spark-test" % "0.3+79-4936e981" % Test 47 | ``` 48 | 49 | ## Usage 50 | 51 | ### Implicit Spark Session 52 | 53 | Spark-Test provides an implicit Spark Session, it means that you don't have to define a Spark Session for your tests. You can immediately 54 | 55 | ### Create a DataFrame with a JSON string 56 | 57 | We start by importing and extending SparkTest. 58 | ```scala 59 | import io.univalence.sparktest.SparkTest 60 | import org.scalatest.FunSuiteLike 61 | 62 | class MyTestClass extends FunSuiteLike with SparkTest {} 63 | ``` 64 | 65 | Then, we can create our test: 66 | ```scala 67 | class MyTestClass extends FunSuiteLike with SparkTest { 68 | test("create df with json string") { 69 | // create df with json string 70 | val df = dfFromJsonString("{a:1}", "{a:2}") 71 | } 72 | } 73 | ``` 74 | 75 | ### Comparing DataFrames 76 | To compare DataFrames, you can simply call the assertEquals method. It throws an SparkTestError if they are not equal. 77 | 78 | For instance, this: 79 | ```scala 80 | val dfUT = Seq(1, 2, 3).toDF("id") 81 | val dfExpected = Seq(2, 1, 4).toDF("id") 82 | 83 | dfUT.assertEquals(dfExpected) 84 | ``` 85 | ... throws the following exception: 86 | ``` 87 | io.univalence.sparktest.SparkTest$ValueError: The data set content is different : 88 | 89 | in value at id, 2 was not equal to 1 90 | dataframe({id: 1}) 91 | dataframe({id: 2}) 92 | 93 | in value at id, 1 was not equal to 2 94 | dataframe({id: 2}) 95 | dataframe({id: 1}) 96 | 97 | in value at id, 4 was not equal to 3 98 | dataframe({id: 3}) 99 | dataframe({id: 4}) 100 | ``` 101 | 102 | ## Support 103 | 104 | If you have any problem/question don't hesitate to add a new issue. 105 | 106 | ## Authors 107 | 108 | Made with :heart: by Univalence's team. 109 | 110 | ## License 111 | 112 | Spark-Test is licensed under the Apache License, Version 2.0 (the “License”); you may not use this software except in compliance with the License. 113 | 114 | 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. 115 | 116 | ## Dependencies 117 | 118 | * [typedpath](https://github.com/univalence/spark-tools/tree/master/typedpath) - current 119 | 120 | ## Links 121 | 122 | * [Univalence Web site](https://www.univalence.io/) 123 | * [Documentation](https://www.javadoc.io/doc/io.univalence/spark-test_2.11) 124 | * [Microsite](https://univalence.github.io/spark-tools/spark-test/) 125 | * [Source code](https://github.com/univalence/spark-tools/tree/master/spark-test) 126 | * [Article EN](https://blog.univalence.io/tests-with-spark-how-to-keep-our-heads-above-water/) | [FR](https://blog.univalence.io/les-tests-avec-spark-sortir-la-tete-de-leau/) 127 | 128 | :star: Star us on GitHub — it helps! -------------------------------------------------------------------------------- /spark-test/src/main/scala/io/univalence/sparktest/Schema2CaseClass.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.sparktest 2 | 3 | import org.apache.spark.sql.types._ 4 | 5 | class Schema2CaseClass { 6 | 7 | type TypeConverter = DataType => String 8 | 9 | def schemaToCaseClass(schema: StructType, className: String)(implicit tc: TypeConverter): String = { 10 | def genField(s: StructField): String = { 11 | val f = tc(s.dataType) 12 | s match { 13 | case x if x.nullable => s" ${s.name}:Option[$f]" 14 | case _ => s" ${s.name}:$f" 15 | } 16 | } 17 | 18 | val fieldsStr = schema.map(genField).mkString(",\n ") 19 | s"""|case class $className ( 20 | | $fieldsStr 21 | |)""".stripMargin 22 | } 23 | 24 | object implicits { 25 | 26 | implicit val defaultTypeConverter: TypeConverter = { 27 | case _: ByteType => "Byte" 28 | case _: ShortType => "Short" 29 | case _: IntegerType => "Int" 30 | case _: LongType => "Long" 31 | case _: FloatType => "Float" 32 | case _: DoubleType => "Double" 33 | case _: DecimalType => "java.math.BigDecimal" 34 | case _: StringType => "String" 35 | case _: BinaryType => "Array[Byte]" 36 | case _: BooleanType => "Boolean" 37 | case _: TimestampType => "java.sql.Timestamp" 38 | case _: DateType => "java.sql.Date" 39 | case _: ArrayType => "scala.collection.Seq" 40 | case _: MapType => "scala.collection.Map" 41 | case _: StructType => "org.apache.spark.sql.Row" 42 | case _ => "String" 43 | } 44 | 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /spark-test/src/main/scala/io/univalence/sparktest/SparkTestSession.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.sparktest 2 | 3 | import org.apache.spark.sql.SparkSession 4 | 5 | trait SparkTestSession { 6 | 7 | def spark: SparkSession = 8 | SparkSession 9 | .builder() 10 | .master("local[1]") 11 | .config("spark.sql.shuffle.partitions", 1) 12 | .config("spark.ui.enabled", value = false) 13 | .getOrCreate() 14 | 15 | } 16 | 17 | object SparkTestSession extends SparkTestSession 18 | -------------------------------------------------------------------------------- /spark-test/src/main/scala/io/univalence/sparktest/internal/DatasetUtils.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.sparktest.internal 2 | 3 | import org.apache.spark.sql.{ Dataset, SparkTestExt } 4 | 5 | object DatasetUtils { 6 | 7 | def cacheIfNotCached(ds: Dataset[_]): Unit = 8 | if (!isCached(ds)) ds.cache() 9 | 10 | def isCached(ds: Dataset[_]): Boolean = 11 | SparkTestExt.isCached(ds) 12 | } 13 | -------------------------------------------------------------------------------- /spark-test/src/main/scala/org/apache/spark/sql/SparkTestExt.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql 2 | 3 | object SparkTestExt { 4 | 5 | /** 6 | * Check if a dataset is already cached or not 7 | * @param ds The Dataset that may be in cache 8 | * @return 9 | */ 10 | def isCached(ds: Dataset[_]): Boolean = { 11 | val ss = ds.sparkSession 12 | val cacheManager = ss.sharedState.cacheManager 13 | cacheManager.lookupCachedData(ds).nonEmpty 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /spark-test/src/test/resources/jsonTest.json: -------------------------------------------------------------------------------- 1 | {"name":"Michael"} 2 | {"name":"Andy", "age":30} 3 | {"name":"Justin", "age":19} -------------------------------------------------------------------------------- /spark-test/src/test/resources/jsonTest2.json: -------------------------------------------------------------------------------- 1 | {"name":"Chandler"} 2 | {"name":"Phoebe", "age":30} 3 | {"name":"Monica", "age":30} 4 | {"name":"Ross", "age":30} 5 | {"name":"Rachel", "age":30} 6 | {"name":"Joey", "age":30} -------------------------------------------------------------------------------- /spark-test/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | log4j.rootCategory=WARN, console 2 | log4j.appender.console=org.apache.log4j.ConsoleAppender 3 | log4j.appender.console.target=System.err 4 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 5 | log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n 6 | -------------------------------------------------------------------------------- /spark-test/src/test/scala/io/univalence/sparktest/DataFrameComparisonTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.sparktest 2 | 3 | import io.univalence.schema.SchemaComparator.SchemaError 4 | import org.apache.spark.SparkContext 5 | import org.apache.spark.sql.{ Row, SparkSession } 6 | import org.apache.spark.sql.types.{ IntegerType, StructField, StructType } 7 | import org.scalatest.FunSuite 8 | 9 | class DataFrameComparisonTest extends FunSuite with SparkTest { 10 | 11 | val sharedSparkSession: SparkSession = ss 12 | val sc: SparkContext = ss.sparkContext 13 | 14 | // TODO : unordered 15 | ignore("should assertEquals unordered between equal DF") { 16 | val dfUT = Seq(1, 2, 3).toDF("id") 17 | val dfExpected = Seq(3, 2, 1).toDF("id") 18 | 19 | dfUT.assertEquals(dfExpected) 20 | } 21 | 22 | // TODO : unordered 23 | ignore("should not assertEquals unordered between DF with different contents") { 24 | val dfUT = Seq(1, 2, 3).toDF("id") 25 | val dfExpected = Seq(2, 1, 4).toDF("id") 26 | 27 | assertThrows[SparkTestError] { 28 | dfUT.assertEquals(dfExpected) 29 | } 30 | } 31 | 32 | test("should assertEquals ordered between equal DF") { 33 | val dfUT = Seq(1, 2, 3).toDF("id") 34 | val dfExpected = Seq(1, 2, 3).toDF("id") 35 | 36 | dfUT.assertEquals(dfExpected) 37 | } 38 | 39 | test("should not assertEquals ordered between DF with different contents") { 40 | val dfUT = Seq(1, 2, 3).toDF("id") 41 | val dfExpected = Seq(1, 3, 4).toDF("id") 42 | 43 | assertThrows[SparkTestError] { 44 | dfUT.assertEquals(dfExpected) 45 | } 46 | } 47 | 48 | test("should not assertEquals between DF with different schema") { 49 | val dfUT = Seq(1, 2, 3).toDF("id") 50 | val dfExpected = Seq(1, 2, 3).toDF("di") 51 | 52 | assertThrows[SchemaError] { 53 | dfUT.assertEquals(dfExpected) 54 | } 55 | } 56 | 57 | test("assertEquals (DF & Seq) : a DF and a Seq with the same content are equal") { 58 | val seq = Seq(1, 2, 3) 59 | val df = ss.createDataFrame( 60 | sc.parallelize(seq.map(Row(_))), 61 | StructType(List(StructField("number", IntegerType, nullable = true))) 62 | ) 63 | 64 | df.assertEquals(seq) 65 | } 66 | 67 | test("assertEquals (DF & Seq) : a DF and a Seq with different content are not equal") { 68 | val df = Seq(1, 3, 3).toDF("number") 69 | val seqEx = Seq(1, 2, 3) 70 | 71 | assertThrows[SparkTestError] { 72 | df.assertEquals(seqEx) 73 | } 74 | } 75 | 76 | test("should assertEquals ordered between equal DF with columns containing special character") { 77 | val dfUT = Seq(1, 2, 3).toDF("id.a") 78 | val dfExpected = Seq(2, 1, 4).toDF("id.a") 79 | 80 | assertThrows[SparkTestError] { 81 | dfUT.assertEquals(dfExpected) 82 | } 83 | } 84 | 85 | /*test("assertEquals (DF & Map) : a DF and a Map with the same content are equal") { 86 | 87 | }*/ 88 | 89 | /*test("shouldExists (DF) : at least one row should match the predicate") { 90 | val l = List(1, 2, 3) 91 | val df = ss.createDataFrame( 92 | sc.parallelize(l.map(Row(_))), 93 | StructType(List(StructField("number", IntegerType, nullable = true))) 94 | ) 95 | //df.shouldExists((n : Int) => n > 2) // ca fonctionne pas non plus :( 96 | //df.shouldExists(_ > 2) 97 | } 98 | 99 | test("shouldExists : should throw an error if all the rows don't match the predicate") { 100 | val df = Seq(1, 2, 3).toDF() 101 | 102 | assertThrows[AssertionError] { 103 | df.shouldExists((n: Int) => n > 3) 104 | } 105 | } 106 | 107 | test("shouldForAll : all the rows should match the predicate") { 108 | val df = Seq(1, 2, 3).toDF() 109 | 110 | df.shouldForAll((n: Int) => n >= 1) 111 | } 112 | 113 | test("shouldForAll : should throw an error if one of the row does not match the predicate") { 114 | val df = Seq(1, 2, 3).toDF() 115 | 116 | assertThrows[AssertionError] { 117 | df.shouldForAll((n: Int) => n >= 2) 118 | } 119 | }*/ 120 | 121 | } 122 | -------------------------------------------------------------------------------- /spark-test/src/test/scala/io/univalence/sparktest/GettingStartedTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.sparktest 2 | 3 | import org.scalatest.FunSuite 4 | 5 | class GettingStartedTest extends FunSuite with SparkTest { 6 | 7 | test("some test") { 8 | val df1 = dataframe("{a:1, b:true}", "{a:2, b:false}") 9 | val df2 = dataframe("{a:1}", "{a:3}") 10 | 11 | assertThrows[SparkTestError] { 12 | /* 13 | The data set content is different : 14 | 15 | in value at a, 3 was diff to 2 16 | dataframe({a: 2, b: false}) 17 | dataframe({a: 3}) 18 | */ 19 | df1.assertEquals(df2) 20 | } 21 | } 22 | 23 | test("some test with custom configuration") { 24 | val df1 = dataframe("{a:1, b:true}") 25 | val df2 = dataframe("{a:1, c:false}") 26 | 27 | withConfiguration(failOnMissingExpectedCol = false, failOnMissingOriginalCol = false)({ df1.assertEquals(df2) }) 28 | } 29 | 30 | test("test view") { 31 | val df1 = 32 | dataframe("{a:1, b:true}", "{a:1, b:true}", "{a:1, b:true}", "{a:1, b:true}", "{a:1, b:true}", "{a:1, b:true}") 33 | val df2 = dataframe("{a:1, b:false}", 34 | "{a:1, b:false}", 35 | "{a:1, b:false}", 36 | "{a:1, b:false}", 37 | "{a:1, b:false}", 38 | "{a:1, b:false}") 39 | 40 | assertThrows[SparkTestError] { 41 | withConfiguration(maxRowError = 2)({ 42 | df1.assertEquals(df2) 43 | }) 44 | } 45 | } 46 | } 47 | 48 | case class A(a: Int) 49 | -------------------------------------------------------------------------------- /spark-test/src/test/scala/io/univalence/sparktest/RDDComparisonTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.sparktest 2 | 3 | import org.apache.spark.SparkContext 4 | import org.apache.spark.sql.SparkSession 5 | import org.scalatest.FunSuite 6 | 7 | class RDDComparisonTest extends FunSuite with SparkTest { 8 | val sharedSparkSession: SparkSession = ss 9 | val sc: SparkContext = ss.sparkContext 10 | 11 | // TODO : Add tests when not equal for comparisons with Seq, List 12 | test("assertEquals (RDD & Seq) : an RDD and a Seq with the same content are equal") { 13 | val seq = Seq(1, 2, 3) 14 | val rdd = sc.parallelize(seq) 15 | 16 | rdd.assertEquals(seq) 17 | } 18 | 19 | test("assertEquals (RDD & List) : an RDD and a List with the same content are equal") { 20 | val l = List(1, 2, 3) 21 | val rdd = sc.parallelize(l) 22 | 23 | rdd.assertEquals(l) 24 | } 25 | 26 | test("shouldExists (RDD) : at least one row should match the predicate") { 27 | val rdd = sc.parallelize(Seq(1, 2, 3)) 28 | 29 | rdd.shouldExists(_ > 2) 30 | } 31 | 32 | test("shouldExists (RDD) : should throw an error if all the rows don't match the predicate") { 33 | val rdd = sc.parallelize(Seq(1, 2, 3)) 34 | 35 | assertThrows[AssertionError] { 36 | rdd.shouldExists(_ > 3) 37 | } 38 | } 39 | 40 | test("shouldForAll (RDD) : all the rows should match the predicate") { 41 | val rdd = sc.parallelize(Seq(1, 2, 3)) 42 | 43 | rdd.shouldForAll(_ >= 1) 44 | } 45 | 46 | test("shouldForAll (RDD) : should throw an error if one of the row does not match the predicate") { 47 | val rdd = sc.parallelize(Seq(1, 2, 3)) 48 | 49 | assertThrows[AssertionError] { 50 | rdd.shouldForAll(_ >= 2) 51 | } 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /spark-test/src/test/scala/io/univalence/sparktest/ReadFromJsonTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.sparktest 2 | 3 | import org.apache.spark.SparkContext 4 | import org.apache.spark.sql.SparkSession 5 | import org.scalactic.Prettifier 6 | import org.scalatest.FunSuite 7 | 8 | class ReadFromJsonTest extends FunSuite with SparkTest { 9 | val sharedSparkSession: SparkSession = ss 10 | val sc: SparkContext = ss.sparkContext 11 | 12 | implicit val default: Prettifier = Prettifier.default 13 | 14 | test("load Json from String") { 15 | //manage json option for jackson 16 | val df = dataframe("{a:1}", "{a:2}") 17 | 18 | //df.as[Long].assertEquals(Seq(1L, 2L)) 19 | 20 | df.as[Long].assertContains(1, 2) 21 | df.as[Long].assertContains(1) 22 | df.as[Long].assertContains(2) 23 | 24 | df.assertEquals(df) 25 | } 26 | 27 | ignore("load Json from String2") { 28 | //#Hard we need to use magnolia to solve this one and generate an expression encoder 29 | /* 30 | val df = dfFromJsonString("[{a:1},{a:2}]") 31 | 32 | case class A(a:Int) 33 | 34 | df.as[A].assertContains(A(1),A(2)) 35 | */ 36 | } 37 | 38 | /* 39 | TODO : manage spark logs, only warnings, no logs in info 40 | TODO : optimize spark for small load 41 | */ 42 | 43 | test("load Json from file") { 44 | val path = "spark-test/src/test/resources/jsonTest.json" 45 | val df = dfFromJsonFile(path) 46 | 47 | assert(df.count == 3) 48 | } 49 | 50 | test("load Json from multiple files") { 51 | val path = "spark-test/src/test/resources/jsonTest.json" 52 | val path2 = "spark-test/src/test/resources/jsonTest2.json" 53 | val df = loadJson(path, path2) 54 | 55 | assert(df.count == 9) 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /spark-test/src/test/scala/io/univalence/sparktest/ValueComparisonTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.sparktest 2 | 3 | import ValueComparison._ 4 | import org.apache.spark.SparkContext 5 | import org.apache.spark.sql.types.{ IntegerType, StringType, StructField, StructType } 6 | import org.apache.spark.sql.{ Row, SparkSession } 7 | import org.scalactic.Prettifier 8 | import org.scalatest.FunSuiteLike 9 | 10 | import scala.collection.mutable 11 | import scala.collection.mutable.ArrayBuffer 12 | 13 | class ValueComparisonTest extends FunSuiteLike with SparkTest { 14 | import io.univalence.typedpath._ 15 | 16 | val sharedSparkSession: SparkSession = ss 17 | val sc: SparkContext = ss.sparkContext 18 | 19 | implicit val default: Prettifier = Prettifier.default 20 | 21 | test("A row should have no modification with itself") { 22 | val df = Seq( 23 | ("1", "2") 24 | ).toDF("set", "id") 25 | 26 | val value = fromRow(df.first()) 27 | assert(compareValue(value, value).isEmpty) 28 | } 29 | 30 | test("Add an ArrayType in an ArrayType") { 31 | 32 | val df1 = Seq( 33 | (Array(Array("2", "1")), 2) 34 | ).toDF("set", "id") 35 | 36 | val df2 = Seq( 37 | (Array(Array("2", "1"), Array("2", "5")), 2) 38 | ).toDF("set", "id") 39 | 40 | /** 41 | * in field set at index 1, WrappedArray(2, 5) was added 42 | * 43 | * in field set at index 0, WrappedArray(2, 4) was not equal to WrappedArray(2, 3) 44 | * in field set at index 1, WrappedArray(2, 5) was not equal to WrappedArray(2, 2) 45 | */ 46 | assert( 47 | compareValue(fromRow(df1.first), fromRow(df2.first)) == 48 | ArrayBuffer( 49 | ObjectModification(index".set[1]", AddValue(AtomicValue(mutable.WrappedArray.make(Array("2", "5"))))) 50 | ) 51 | ) 52 | } 53 | 54 | test("Remove a field") { 55 | val df1 = Seq( 56 | (Array("2", "2"), 2, 2) 57 | ).toDF("set", "id", "id2") 58 | 59 | val df2 = Seq( 60 | (Array("2", "2"), 2) 61 | ).toDF("set", "id") 62 | 63 | assert( 64 | compareValue(fromRow(df1.first), fromRow(df2.first)) == 65 | ArrayBuffer(ObjectModification(index".id2", RemoveValue(AtomicValue(2)))) 66 | ) 67 | } 68 | 69 | test("Add a field") { 70 | val df1 = Seq( 71 | (Array("2", "2"), 2) 72 | ).toDF("set", "id") 73 | 74 | val df2 = Seq( 75 | (Array("2", "2"), 2, 4) 76 | ).toDF("set", "id", "id2") 77 | 78 | assert( 79 | compareValue(fromRow(df1.first), fromRow(df2.first)) == 80 | ArrayBuffer(ObjectModification(index".id2", AddValue(AtomicValue(4)))) 81 | ) 82 | } 83 | 84 | test("Change a field type") { 85 | val data1 = Seq( 86 | Row("1") 87 | ) 88 | 89 | val data2 = Seq( 90 | Row(1) 91 | ) 92 | 93 | val df1 = ss.createDataFrame( 94 | sc.parallelize(data1), 95 | StructType(List(StructField("number", StringType, nullable = false))) 96 | ) 97 | 98 | val df2 = ss.createDataFrame( 99 | sc.parallelize(data2), 100 | StructType(List(StructField("number", IntegerType, nullable = false))) 101 | ) 102 | 103 | assert( 104 | compareValue(fromRow(df1.first), fromRow(df2.first)) == 105 | ArrayBuffer(ObjectModification(index".number", ChangeValue(AtomicValue("1"), AtomicValue(1)))) 106 | ) 107 | } 108 | 109 | test("From null to a value should be an AddValue modification") { 110 | val df1 = dataframe("{a: null, b: false}") 111 | val df2 = dataframe("{a: 2}") 112 | assert( 113 | compareValue(fromRow(df1.first), fromRow(df2.first)) == 114 | ArrayBuffer(ObjectModification(index".b", RemoveValue(AtomicValue(false))), 115 | ObjectModification(index".a", AddValue(AtomicValue(2)))) 116 | ) 117 | } 118 | 119 | test("From null to null should not return any error") { 120 | val df1 = dataframe("{a: null}") 121 | val df2 = dataframe("{a: null}") 122 | assert(compareValue(fromRow(df1.first), fromRow(df2.first)) == Seq()) 123 | 124 | } 125 | } 126 | -------------------------------------------------------------------------------- /spark-test/src/test/scala/io/univalence/sparktest/bug/DataFrameEqualityBugSparkTraining.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.sparktest.bug 2 | 3 | import io.univalence.sparktest.SparkTest 4 | import org.scalatest.FunSuite 5 | 6 | class DataFrameEqualityBugSparkTraining extends FunSuite with SparkTest { 7 | 8 | test("select name and age inlined") { 9 | dataframe("""{name:"John",age:13, num:1}""").createTempView("df") 10 | ss.sql("select name,age from df").assertEquals(dataframe("""{name:"John",age:13}""")) 11 | } 12 | } 13 | -------------------------------------------------------------------------------- /spark-test/src/test/scala/io/univalence/sparktest/bug/DataFrameNoCommonColumnsButIsEqual.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.sparktest.bug 2 | 3 | import io.univalence.schema.SchemaComparator.NoCommonFieldError 4 | import io.univalence.sparktest.SparkTest 5 | import org.scalatest.FunSuite 6 | 7 | class DataFrameNoCommonColumnsButIsEqual extends FunSuite with SparkTest { 8 | test("with failOnMissingExpectedCol config") { 9 | val actualDf = dataframe("{c:0}") 10 | val expectedDf = dataframe("{a:0, b:false}") 11 | 12 | assertThrows[NoCommonFieldError.type] { 13 | withConfiguration(failOnMissingExpectedCol = false)( 14 | actualDf.assertEquals(expectedDf) 15 | ) 16 | } 17 | } 18 | 19 | test("with failOnMissingExpectedCol config and empty dataframes") { 20 | val actualDf = dataframe("{}") 21 | val expectedDf = dataframe("{}") 22 | 23 | withConfiguration(failOnMissingExpectedCol = false)( 24 | actualDf.assertEquals(expectedDf) 25 | ) 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /spark-test/src/test/scala/io/univalence/sparktest/featurematching/SparkFastTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.sparktest.featurematching 2 | 3 | import io.univalence.schema.SchemaComparator.SchemaError 4 | import io.univalence.sparktest.SparkTest 5 | import org.apache.spark.SparkContext 6 | import org.apache.spark.sql.{ Row, SparkSession } 7 | import org.apache.spark.sql.types.{ DoubleType, IntegerType, StructField, StructType } 8 | import org.scalatest.FunSuite 9 | 10 | //https://github.com/MrPowers/spark-fast-tests 11 | class SparkFastTest extends FunSuite with SparkTest { 12 | 13 | val sharedSparkSession: SparkSession = ss 14 | val sc: SparkContext = ss.sparkContext 15 | 16 | test("test DataFrame equality") { 17 | val sourceDF = Seq( 18 | "jose", 19 | "li", 20 | "luisa" 21 | ).toDF("name") 22 | 23 | val expectedDF = Seq( 24 | "jose", 25 | "li", 26 | "luisa" 27 | ).toDF("name") 28 | 29 | //assertSmallDatasetEquality(sourceDF, expectedDF) // equal 30 | sourceDF.assertEquals(expectedDF) 31 | } 32 | 33 | test("column equality") { 34 | val df = Seq( 35 | ("Pierre", "Pierre"), 36 | ("Louis", "Louis"), 37 | (null, null), 38 | ("Jean", "Jean") 39 | ).toDF("name", "expected_name") 40 | 41 | //assertColumnEquality(df, "name", "expected_name") 42 | df.assertColumnEquality("name", "expected_name") 43 | } 44 | 45 | // TODO unordered 46 | ignore("unordered equality") { 47 | val sourceDF = Seq( 48 | "1", 49 | "5" 50 | ).toDF("number") 51 | 52 | val expectedDF = Seq( 53 | "5", 54 | "1" 55 | ).toDF("number") 56 | 57 | //assertSmallDataFrameEquality(sourceDF, expectedDF, orderedComparison = false) // equal 58 | //by default SparkTest doesn't check for ordering 59 | sourceDF.assertEquals(expectedDF) 60 | } 61 | 62 | test("ignore nullable flag equality") { 63 | val data = Seq( 64 | Row(1), 65 | Row(5) 66 | ) 67 | 68 | val sourceDF = ss.createDataFrame( 69 | sc.parallelize(data), 70 | StructType(List(StructField("number", IntegerType, nullable = false))) 71 | ) 72 | 73 | val expectedDF = ss.createDataFrame( 74 | sc.parallelize(data), 75 | StructType(List(StructField("number", IntegerType, nullable = true))) 76 | ) 77 | 78 | sourceDF.assertEquals(expectedDF) 79 | 80 | assertThrows[SchemaError] { 81 | withConfiguration(failOnNullable = true)(sourceDF.assertEquals(expectedDF)) 82 | } 83 | } 84 | 85 | test("approximate dataframe equality") { 86 | val sourceData = Seq( 87 | Row(1.11), 88 | Row(5.22), 89 | Row(null) 90 | ) 91 | 92 | val sourceDF = ss.createDataFrame( 93 | sc.parallelize(sourceData), 94 | StructType(List(StructField("number", DoubleType, nullable = true))) 95 | ) 96 | 97 | val expectedData = Seq( 98 | Row(1.1), 99 | Row(5.2), 100 | Row(null) 101 | ) 102 | 103 | val expectedDF = ss.createDataFrame( 104 | sc.parallelize(expectedData), 105 | StructType(List(StructField("number", DoubleType, nullable = true))) 106 | ) 107 | 108 | //assertApproximateDataFrameEquality(sourceDF, expectedDF, 0.1) 109 | sourceDF.assertApproxEquals(expectedDF, 0.1) 110 | } 111 | 112 | } 113 | -------------------------------------------------------------------------------- /spark-test/src/test/scala/io/univalence/sparktest/featurematching/SparkTestingBase.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.sparktest.featurematching 2 | 3 | import io.univalence.sparktest.SparkTest 4 | import org.apache.spark.SparkContext 5 | import org.apache.spark.sql.SparkSession 6 | import org.scalatest.FunSuite 7 | 8 | //from https://github.com/holdenk/spark-testing-base/wiki 9 | class SparkTestingBase extends FunSuite with SparkTest { 10 | 11 | //SharedSparkContext 12 | val sharedSparkSession: SparkSession = ss 13 | val sc: SparkContext = ss.sparkContext 14 | 15 | //https://github.com/holdenk/spark-testing-base/wiki/RDDComparisons 16 | test("test RDDComparisons") { 17 | val expectedRDD = sc.parallelize(Seq(1, 2, 3)) 18 | val resultRDD = sc.parallelize(Seq(3, 2, 1)) 19 | 20 | assert(None === expectedRDD.compareRDD(resultRDD)) 21 | expectedRDD.assertEquals(resultRDD) 22 | assert(Some((Some(1), Some(3))) === expectedRDD.compareRDDWithOrder(resultRDD)) 23 | intercept[AssertionError] { 24 | expectedRDD.assertEqualsWithOrder(resultRDD) 25 | } 26 | } 27 | 28 | //https://github.com/holdenk/spark-testing-base/wiki/DataFrameSuiteBase 29 | test("dataframe should be equal to itself") { 30 | val input1 = sc.parallelize(List(1, 2, 4)).toDF 31 | 32 | input1 assertEquals input1 // equal 33 | } 34 | 35 | test("dataframe should not be equal to a different dataframe") { 36 | val input1 = sc.parallelize(List(1, 2, 4)).toDF 37 | val input2 = sc.parallelize(List(2, 4, 1)).toDF 38 | 39 | intercept[SparkTestError] { 40 | input1.assertEquals(input2) // not equal 41 | } 42 | } 43 | 44 | test("test DataFrame Comparison with precision") { 45 | val input1 = sc.parallelize(List[(Int, Double)]((1, 1.1), (2, 2.2), (3, 3.3))).toDF 46 | val input2 = sc.parallelize(List[(Int, Double)]((1, 1.2), (2, 2.3), (3, 3.4))).toDF 47 | //assertDataFrameApproximateEquals(input1, input2, 0.11) // equal 48 | input1.assertApproxEquals(input2, 0.11) // equal 49 | intercept[SparkTestError] { 50 | input1.assertApproxEquals(input2, 0.05) // not equal 51 | } 52 | } 53 | 54 | //https://github.com/holdenk/spark-testing-base/wiki/DatasetSuiteBase 55 | test("dataset should be equal to itself") { 56 | val input1 = sc.parallelize(List(1, 2, 3)).toDS 57 | 58 | input1 assertEquals input1 // equal 59 | } 60 | 61 | test("dataset should not be equal to a different dataset") { 62 | val input1 = sc.parallelize(List(1, 2, 3)).toDS 63 | val input2 = sc.parallelize(List(4, 5, 6)).toDS 64 | 65 | intercept[SparkTestError] { 66 | input1 assertEquals input2 // not equal 67 | } 68 | } 69 | 70 | test("dataset should be equal to itself with precision") { 71 | val input1 = sc.parallelize(List[(Int, Double)]((1, 1.1), (2, 2.2), (3, 3.3))).toDS 72 | val input2 = sc.parallelize(List[(Int, Double)]((1, 1.2), (2, 2.3), (3, 3.4))).toDS 73 | 74 | // assertDatasetApproximateEquals(input1, input2, 0.11) // equal 75 | input1.assertApproxEquals(input2, 0.11) 76 | } 77 | 78 | test("dataset should not be equal to a different even with precision") { 79 | val input1 = sc.parallelize(List[(Int, Double)]((1, 1.1), (2, 2.2), (3, 3.3))).toDS 80 | val input2 = sc.parallelize(List[(Int, Double)]((1, 1.2), (2, 2.3), (3, 3.4))).toDS 81 | 82 | intercept[SparkTestError] { 83 | input1.assertApproxEquals(input2, 0.05) 84 | } 85 | 86 | // assertDatasetApproximateEquals(input1, input2, 0.05) // not equal 87 | } 88 | 89 | //https://github.com/holdenk/spark-testing-base/wiki/RDDGenerator 90 | //https://github.com/holdenk/spark-testing-base/wiki/DataFrameGenerator 91 | //https://github.com/holdenk/spark-testing-base/wiki/Dataset-Generator 92 | //Not supported, IMO (Jon) doesn't really make sense for real use cases to generate data like that. 93 | 94 | } 95 | -------------------------------------------------------------------------------- /spark-zio/README.md: -------------------------------------------------------------------------------- 1 | Spark-ZIO 2 | ====================== 3 | 4 | [ ![Download](https://api.bintray.com/packages/univalence/univalence-jvm/spark-zio/images/download.svg) ](https://bintray.com/univalence/univalence-jvm/spark-zio/_latestVersion) 5 | 6 | Spark-ZIO allows access to Spark using ZIO's environment. 7 | 8 | ## Installation 9 | 10 | ### Stable version 11 | 12 | Version for scala 2.11.X : 13 | 14 | ```scala 15 | libraryDependencies += "io.univalence" % "spark-zio_2.11" % "46e659ce" 16 | ``` 17 | 18 | ### Latest version 19 | 20 | If you want to get the very last version of this library you can still download it using bintray here : https://bintray.com/univalence/univalence-jvm/spark-zio 21 | 22 | Here is an example using ```version 0.3+79-4936e981``` that work with ```scala 2.11.X```: 23 | 24 | ```scala 25 | resolvers += "spark-zio" at "http://dl.bintray.com/univalence/univalence-jvm" 26 | libraryDependencies += "io.univalence" %% "spark-zio" % "0.3+79-4936e981" 27 | ``` 28 | 29 | ## License 30 | 31 | Copyright 2019 Univalence.io 32 | 33 | Licensed under the Apache License, Version 2.0: 34 | http://www.apache.org/licenses/LICENSE-2.0 35 | -------------------------------------------------------------------------------- /spark-zio/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | log4j.rootCategory=WARN, console 2 | log4j.appender.console=org.apache.log4j.ConsoleAppender 3 | log4j.appender.console.target=System.err 4 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 5 | log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n 6 | -------------------------------------------------------------------------------- /spark-zio/src/test/resources/toto/toto.txt: -------------------------------------------------------------------------------- 1 | bonjour -------------------------------------------------------------------------------- /spark-zio/src/test/scala/io/univalence/sparkzio/CircuitTapTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.sparkzio 2 | 3 | import zio.test._ 4 | import zio.test.Assertion._ 5 | 6 | object CircuitTapTest 7 | extends DefaultRunnableSpec( 8 | suite("tap")( 9 | testM("smoking")({ 10 | import zio.syntax._ 11 | import syntax._ 12 | 13 | for { 14 | percent <- Ratio(0.05).toTask 15 | tap <- CircuitTap.make[String, String](percent, _ => true, "rejected", 1000) 16 | _ <- tap("first".fail).ignore 17 | _ <- tap("second".fail).ignore 18 | a <- tap("third".fail).either 19 | s <- tap.getState 20 | } yield { 21 | assert(a.isLeft, isTrue) && 22 | assert(s.failed, equalTo(1)) && 23 | assert(s.rejected, equalTo(2)) && 24 | assert(s.decayingErrorRatio.ratio.value, isGreaterThan(Ratio.zero.value)) 25 | } 26 | }) 27 | ) 28 | ) 29 | -------------------------------------------------------------------------------- /spark-zio/src/test/scala/io/univalence/sparkzio/IteratorTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.sparkzio 2 | 3 | import zio.{ DefaultRuntime, IO, Ref, Task, UIO, ZIO, ZManaged } 4 | import zio.clock.Clock 5 | import zio.stream.{ Stream, ZSink, ZStream } 6 | import zio.test.DefaultRunnableSpec 7 | import zio.test._ 8 | import zio.test.Assertion._ 9 | 10 | object StreamTest { 11 | 12 | def assertForAll[R, E, A](zstream: ZStream[R, E, A])(f: A => TestResult): ZIO[R, E, TestResult] = 13 | zstream.fold(assert(Unit, Assertion.anything))((as, a) => as && f(a)) 14 | 15 | def isSorted[A: Ordering]: Assertion[Iterable[A]] = 16 | Assertion.assertion("sorted")()(x => { 17 | val y = x.toList 18 | y.sorted == y 19 | }) 20 | } 21 | 22 | object IteratorTest 23 | extends DefaultRunnableSpec( 24 | suite("iterator")( 25 | testM("to iterator should be lazy")({ 26 | case class Element(n: Int, time: Long) 27 | 28 | (for { 29 | clock <- ZIO.environment[Clock] 30 | n <- Ref.make(0) 31 | incCounter <- n.update(_ + 1).forever.fork 32 | 33 | } yield { 34 | def element: UIO[Element] = n.get.zipWith(clock.clock.nanoTime)(Element) 35 | 36 | val in = Stream.repeatEffect(element) 37 | 38 | val iterator = Iterator.unwrapManaged(Iterator.fromStream(in)) 39 | 40 | val out: ZStream[Any, Nothing, Element] = 41 | ZStream.fromIterator(iterator).mapConcatM(e => element.map(List(e, _))) 42 | 43 | implicit val ordering: Ordering[Element] = Ordering.by(x => x.n -> x.time) 44 | 45 | out.take(2000).runCollect.map(e => assert(e, StreamTest.isSorted)) 46 | }).flatten 47 | }), 48 | testM("<=>")({ 49 | val in: List[Int] = (1 to 100).toList 50 | 51 | (for { 52 | _ <- UIO.unit.toManaged_ 53 | stream1 = ZStream.fromIterator(UIO(in.toIterator)) 54 | iterator <- Iterator.fromStream(stream1) 55 | stream2 = ZStream.fromIterator(UIO(iterator)) 56 | out <- stream2.runCollect.toManaged_ 57 | } yield { 58 | assert(in, equalTo(out)) 59 | }).use(x => ZIO.effect(x)) 60 | 61 | }), 62 | testM("on exit")( 63 | (for { 64 | isOpen <- Ref.make(false).toManaged_ 65 | stream = ZStream.managed(ZManaged.make(isOpen.update(_ => true))(_ => isOpen.set(false))) 66 | iterator <- Iterator.fromStream(stream) 67 | } yield { 68 | assert(iterator.toList, equalTo(List(true))) 69 | }).use(x => IO.effect(x)) 70 | ) 71 | ) 72 | ) 73 | -------------------------------------------------------------------------------- /spark-zio/src/test/scala/io/univalence/sparkzio/ProtoMapWithEffet.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.sparkzio 2 | 3 | import java.util.concurrent.{ BlockingQueue, SynchronousQueue, TimeUnit } 4 | 5 | import io.univalence.sparktest.SparkTest 6 | import org.apache.spark.rdd.RDD 7 | import org.apache.spark.sql.{ Dataset, SparkSession } 8 | import org.scalatest.FunSuite 9 | import zio._ 10 | import zio.clock.Clock 11 | import zio.duration.Duration 12 | import zio.stream._ 13 | import zio.syntax._ 14 | 15 | import scala.util.{ Failure, Success, Try } 16 | 17 | object syntax { 18 | 19 | implicit class ToTask[A](t: Try[A]) { 20 | def toTask: Task[A] = Task.fromTry(t) 21 | } 22 | } 23 | 24 | object ProtoMapWithEffetTest { 25 | 26 | def putStrLn(line: String): UIO[Unit] = zio.console.putStrLn(line).provide(console.Console.Live) 27 | 28 | def tap[E1, E2 >: E1, A]( 29 | rddIO: RDD[IO[E1, A]] 30 | )(onRejected: E2, 31 | maxErrorRatio: Ratio = Ratio(0.05).get, 32 | keepOrdering: Boolean = false, 33 | decayScale: Int = 1000, 34 | localConcurrentTasks: Int = 4): RDD[Either[E2, A]] = 35 | rddIO.mapPartitions(it => { 36 | 37 | val in: stream.Stream[Nothing, IO[E1, A]] = zio.stream.Stream.fromIterator(it.succeed) 38 | 39 | val circuitBreaked: ZIO[Any, Nothing, ZStream[Any, Nothing, Either[E2, A]]] = for { 40 | tap <- CircuitTap.make[E2, E2](maxErrorRatio, _ => true, onRejected, decayScale) 41 | } yield { 42 | if (keepOrdering) 43 | in.mapMPar(localConcurrentTasks)(x => tap(x).either) 44 | else 45 | in.mapMParUnordered(localConcurrentTasks)(x => tap(x).either) 46 | 47 | } 48 | 49 | val iterator: ZIO[Any, Nothing, Iterator[Nothing, Either[E2, A]]] = 50 | Iterator.unwrapManaged(circuitBreaked.toManaged_ >>= Iterator.fromStream) 51 | 52 | new DefaultRuntime {}.unsafeRun(iterator) 53 | 54 | }) 55 | 56 | } 57 | 58 | class ProtoMapWithEffetTest extends FunSuite with SparkTest { 59 | 60 | import ProtoMapWithEffetTest._ 61 | 62 | test("1") { 63 | 64 | val someThing: RDD[Task[Int]] = ss.sparkContext.parallelize(1 to 100).map(x => Task(x)) 65 | 66 | val executed: RDD[Either[Throwable, Int]] = tap(someThing)(new Exception("rejected")) 67 | 68 | assert(executed.count() == 100) 69 | 70 | } 71 | 72 | def time[R](block: => R): (Duration, R) = { 73 | val t0 = System.nanoTime() 74 | val result = block // call-by-name 75 | val t1 = System.nanoTime() 76 | (Duration(t1 - t0, TimeUnit.NANOSECONDS), result) 77 | } 78 | 79 | test("2") { 80 | 81 | val n = 500 82 | val ds: Dataset[Int] = ss.createDataset(1 to n) 83 | 84 | def duration(i: Int) = Duration(if (i % 20 == 0 && i < 200) 800 else 10, TimeUnit.MILLISECONDS) 85 | 86 | def io(i: Int): IO[String, Int] = IO.fail(s"e$i").delay(duration(i)).provide(Clock.Live) 87 | 88 | val value: RDD[IO[String, Int]] = ds.rdd.map(io) 89 | 90 | val unit: RDD[Either[String, Int]] = 91 | tap(value)( 92 | onRejected = "rejected", 93 | maxErrorRatio = Ratio(0.10).get, 94 | keepOrdering = false, 95 | localConcurrentTasks = 8 96 | ) 97 | 98 | val (d, _) = time(assert(unit.count() == n)) 99 | 100 | val computeTime: Long = (1 to n).map(duration).reduce(_ + _).toMillis 101 | 102 | val speedUp = computeTime.toDouble / d.toMillis 103 | 104 | println(s"speedUp of $speedUp") 105 | } 106 | 107 | test("asyncZIO") { 108 | 109 | val n = 50 110 | val s: Stream[Nothing, Int] = stream.Stream.fromIterator(UIO((1 to n).toIterator)) 111 | 112 | def effect(i: Int): ZIO[Any, String, String] = if (i % 4 == 0) s"f$i".fail else s"s$i".succeed 113 | 114 | val g = s.map(effect) 115 | 116 | val h: UIO[Stream[Nothing, Either[String, String]]] = for { 117 | tap <- CircuitTap.make[String, String](Ratio.full, _ => true, "rejected", 1000) 118 | } yield { 119 | g.mapMParUnordered(4)(i => tap(i).either) 120 | } 121 | 122 | val prg: UIO[Iterator[Nothing, Either[String, String]]] = 123 | Iterator.unwrapManaged(h.toManaged_ >>= Iterator.fromStream) 124 | val xs: Seq[Either[String, String]] = new DefaultRuntime {}.unsafeRun(prg).toSeq 125 | 126 | assert(xs.length == n) 127 | 128 | } 129 | 130 | } 131 | -------------------------------------------------------------------------------- /spark-zio/src/test/scala/io/univalence/sparkzio/SparkEnvImplicitClassTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.sparkzio 2 | 3 | import io.univalence.sparkzio.SparkEnv.TaskS 4 | import org.apache.spark.sql.{ DataFrame, SparkSession } 5 | import org.scalatest.FunSuite 6 | import zio.{ DefaultRuntime, IO, Task, ZIO } 7 | import SparkEnv.implicits._ 8 | 9 | class SparkEnvImplicitClassTest extends FunSuite { 10 | val runtime: DefaultRuntime = new DefaultRuntime {} 11 | val sparkZIO: Task[SparkZIO] = Task(SparkSession.builder.master("local[*]").getOrCreate()).map(x => new SparkZIO(x)) 12 | val pathToto: String = "spark-zio/src/test/resources/toto" 13 | 14 | test("sparkEnv read and SparkEnv sql example") { 15 | 16 | val prg: TaskS[(DataFrame, DataFrame)] = for { 17 | df <- SparkEnv.read.textFile(pathToto) 18 | _ <- Task(df.createTempView("totoview")) 19 | df2 <- SparkEnv.sql(s"""SELECT * FROM totoview""") 20 | //_ <- df.zwrite.text("totoWriteZIO") 21 | } yield (df, df2) 22 | 23 | //Providing SparkEnv to ZIO 24 | val liveProgram: IO[Throwable, (DataFrame, DataFrame)] = sparkZIO.flatMap(prg.provide) 25 | 26 | //Unsafe run 27 | val resRun: (DataFrame, DataFrame) = runtime.unsafeRun(liveProgram) 28 | println("toto") 29 | 30 | assert(resRun._1.collect() sameElements resRun._2.collect()) 31 | } 32 | 33 | test("ZIO catchAll example") { 34 | val tigrou: String = "tigrou will only appear in success" 35 | 36 | val prgSuccess: Task[Int] = for { 37 | _ <- Task(println("Program with no Exception: ")) 38 | code <- Task { 39 | for { 40 | successCode <- Task(1) 41 | _ <- Task(println(tigrou)) 42 | } yield successCode 43 | }.flatten.catchAll { 44 | case e: Exception => 45 | for { 46 | _ <- Task { 47 | print("Error: ") 48 | println(e.getMessage) 49 | } 50 | errorCode <- Task(-1) 51 | } yield errorCode 52 | } 53 | } yield code 54 | 55 | val prgFail: Task[Int] = for { 56 | _ <- Task(println("Program with Exception: ")) 57 | code <- Task { 58 | for { 59 | successCode <- Task(1 / 0) 60 | _ <- Task(println(tigrou)) 61 | } yield successCode 62 | }.flatten.catchAll { 63 | case e: Exception => 64 | for { 65 | _ <- Task { 66 | print("Exception: ") 67 | println(e.getMessage) 68 | } 69 | errorCode <- Task(-1) 70 | } yield errorCode 71 | } 72 | } yield code 73 | 74 | assert(runtime.unsafeRun(prgSuccess) === 1) 75 | assert(runtime.unsafeRun(prgFail) === -1) 76 | } 77 | 78 | /*test("sometimes you will use collectAll") { 79 | val primeSeq: Seq[Int] = Seq(2, 3, 5, 7, 11) 80 | 81 | for { 82 | ss <- SparkEnv.sparkSession 83 | seqDF <- { 84 | import ss.implicits._ 85 | Task( 86 | primeSeq.map { primeNumber => 87 | for { 88 | df <- sparkEnv.read.textFile(pathToto) 89 | } yield df.withColumn("prime", 'primeNumber) 90 | } 91 | ) 92 | } 93 | } yield seqDF 94 | 95 | }*/ 96 | 97 | } 98 | -------------------------------------------------------------------------------- /spark-zio/src/test/scala/io/univalence/sparkzio/ZDynamicConcurrency.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.sparkzio 2 | 3 | import zio.{ Fiber, Promise, Queue, Ref, Semaphore, UIO, ZIO } 4 | import zio.stream.ZStream 5 | import zio.stream.ZStream.Pull 6 | 7 | object ZDynamicConcurrency { 8 | 9 | private def releaseCapacity(semaphore: Semaphore, toRelease: UIO[Int]): UIO[Fiber[Nothing, Nothing]] = 10 | (for { 11 | concurrency <- Ref.make(0) 12 | } yield { 13 | def update(next: Int): UIO[Unit] = 14 | (for { 15 | current <- concurrency.get 16 | _ <- if (next >= current) semaphore.releaseN(next - current) else semaphore.acquireN(current - next) 17 | _ <- concurrency.set(next) 18 | } yield {}).uninterruptible 19 | 20 | val nextConcurrencyBounded: UIO[Int] = toRelease.map({ 21 | case x if x < 0 => 0 22 | case x => x 23 | }) 24 | 25 | (nextConcurrencyBounded >>= update).forever.ensuring(update(0)).fork 26 | }).flatten 27 | 28 | implicit class Ops[-R, +E, +A](stream: ZStream[R, E, A]) { 29 | def mapMDynamicPar[R1 <: R, E1 >: E, B](concurrencyLevel: UIO[Int], 30 | maxConcurrency: Int = 16)(f: A => ZIO[R1, E1, B]): ZStream[R1, E1, B] = 31 | ZStream[R1, E1, B] { 32 | for { 33 | out <- Queue.bounded[Pull[R1, E1, B]](maxConcurrency).toManaged(_.shutdown) 34 | permits <- Semaphore.make(permits = 0).toManaged_ 35 | updateConcurrencyFiber <- releaseCapacity(permits, concurrencyLevel).toManaged_ 36 | interruptWorkers <- Promise.make[Nothing, Unit].toManaged_ 37 | _ <- stream.foreachManaged { a => 38 | for { 39 | latch <- Promise.make[Nothing, Unit] 40 | p <- Promise.make[E1, B] 41 | _ <- out.offer(Pull.fromPromise(p)) 42 | _ <- (permits.withPermit(latch.succeed(()) *> f(a).to(p)) race interruptWorkers.await).fork 43 | _ <- latch.await 44 | } yield () 45 | }.foldCauseM( 46 | c => (interruptWorkers.succeed(()) *> out.offer(Pull.halt(c))).unit.toManaged_, 47 | _ => out.offer(Pull.end).unit.toManaged_ 48 | ) 49 | .ensuringFirst(interruptWorkers.succeed(()) *> updateConcurrencyFiber.interrupt) 50 | .fork 51 | } yield out.take.flatten 52 | } 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /typedpath/README.md: -------------------------------------------------------------------------------- 1 | Utils 2 | ====================== 3 | 4 | [ ![Download](https://api.bintray.com/packages/univalence/univalence-jvm/typedpath/images/download.svg) ](https://bintray.com/univalence/univalence-jvm/typedpath/_latestVersion) 5 | 6 | Utils contains mini librairies, it's like a mini monorepository inside our monorepository. 7 | 8 | ## Installation 9 | 10 | TODO 11 | 12 | ## Usage 13 | 14 | ### Schema 15 | 16 | Schema is a helper that compare precisely differences between two schemas. 17 | 18 | ```scala 19 | import io.univalence.schema.SchemaComparator 20 | 21 | val schema_1: StructType = ??? 22 | val schema_2: StructType = ??? 23 | 24 | // If modifications is empty then there is no differences 25 | val modifications: Seq[SchemaModification] = compareSchema(schema_1, schema_2) 26 | 27 | // If there are modifications, throw a SchemaError with details for each differences 28 | assert(schema_1, schema_2) 29 | ``` 30 | 31 | 32 | ### TypedPath 33 | Typedpath is a set of case classes (Algebraic Data Types) and a StringContext macro to help represented "simple" key in datastructures. 34 | 35 | We have the following types : `Key = ArrayKey | FieldKey` and `KeyOrRoot = Key | Root`. 36 | 37 | If we have the following data : 38 | ```clojure 39 | {:person {:name "John", :age 12}, 40 | :status "active"} 41 | ``` 42 | The value `"John"` is at key `person.name`. In Scala we would do the following. 43 | 44 | ```scala 45 | import io.univalence.typedpath._ 46 | import scala.util.Try 47 | 48 | 49 | val p0:Try[KeyOrRoot] = Key.create("person.name") 50 | //or 51 | val p1:Try[FieldKey] = for { 52 | p <- FieldKey.createName("person") 53 | n <- FieldKey.createName("name") 54 | } yield FieldKey(n,FieldKey(p,Root)) 55 | //or 56 | val p2:Try[FieldKey] = for { 57 | p <- FieldKey("person",Root) 58 | n <- FieldKey("name", p) 59 | } yield n 60 | //or using the macro 61 | val p3:FieldKey = key"person.name" 62 | ``` 63 | 64 | -------------------------------------------------------------------------------- /typedpath/src/main/scala/io/univalence/typedpath/strings.scala: -------------------------------------------------------------------------------- 1 | package io.univalence 2 | 3 | package object typedpath { 4 | implicit class KeyHelper(val sc: StringContext) extends AnyVal { 5 | def key(args: Key*): Key = macro KeyMacro.keyMacro 6 | 7 | //TODO : replace with macro 8 | def name(args: Nothing*): String with FieldName = FieldKey.createName(sc.raw()).get 9 | 10 | def index(args: Nothing*): Index = Index.create(sc.raw()).get 11 | } 12 | 13 | } 14 | -------------------------------------------------------------------------------- /typedpath/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | log4j.rootCategory=WARN, console 2 | log4j.appender.console=org.apache.log4j.ConsoleAppender 3 | log4j.appender.console.target=System.err 4 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 5 | log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n 6 | -------------------------------------------------------------------------------- /typedpath/src/test/scala/io/univalence/typedpath/FenekSyntaxTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.typedpath 2 | 3 | object FenekSyntaxTest { 4 | 5 | trait StrucField 6 | 7 | implicit class StringOps(val name: String) { 8 | def <<-(path: Key): StrucField = ??? 9 | } 10 | implicit class PathOps(val path: Key) { 11 | def as(name: String): StrucField = ??? 12 | } 13 | 14 | case class struct(field: StrucField*) { 15 | def addField(field: StrucField*): struct = ??? 16 | 17 | def |+(field: StrucField): struct = ??? 18 | } 19 | 20 | val x: struct = struct( 21 | key"source.source.source.source" as "target1", 22 | "target0" <<- key"source.source.source.source", 23 | "target42" <<- key"source.source.source.source" 24 | ) 25 | 26 | val y: struct = x addField ( 27 | "target2" <<- key"source", 28 | "target43" <<- key"source" 29 | ) 30 | 31 | y addField "target" <<- key"source" 32 | 33 | } 34 | -------------------------------------------------------------------------------- /typedpath/src/test/scala/io/univalence/typedpath/IndexTest.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.typedpath 2 | 3 | import io.univalence.typedpath.Index.{ ArrayIndex, FieldIndex } 4 | import org.scalatest.FunSuite 5 | 6 | import scala.util.Try 7 | 8 | class IndexTest extends FunSuite { 9 | 10 | test("testCreate index") { 11 | assert(Index.create("abc[1].defg").get == (Index(name"abc") at 1 at name"defg")) 12 | 13 | assert(Index.create("abc[1][2][3].defg").get == (Index(name"abc") at 1 at 2 at 3 at name"defg")) 14 | 15 | assert(Index.create("abc[-1].defg").get == (Index(name"abc") at -1 at name"defg")) 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /typedpath/src/test/scala/io/univalence/typedpath/KeySpec.scala: -------------------------------------------------------------------------------- 1 | package io.univalence.typedpath 2 | 3 | import org.scalatest.FunSuite 4 | 5 | import scala.util.{ Failure, Try } 6 | 7 | class KeySpec extends FunSuite { 8 | 9 | import Key._ 10 | 11 | test("tokenize") { 12 | 13 | assert(Token.tokenize("abc[].def") == Seq(NamePart("abc"), Brackets, Dot, NamePart("def"))) 14 | 15 | assert( 16 | Token.tokenize("$abc[].def") == 17 | Seq(ErrorToken("$"), NamePart("abc"), Brackets, Dot, NamePart("def")) 18 | ) 19 | 20 | } 21 | 22 | test("create error") { 23 | val f = Key.create("$abc[].(edf").asInstanceOf[Failure[_]] 24 | assert(f.exception.getMessage.contains("[$]")) 25 | assert(f.exception.getMessage.contains("[(]")) 26 | 27 | } 28 | 29 | test("interpolation") { 30 | import Key._ 31 | 32 | val prefix = "abc" 33 | 34 | //TODO : check illtyped 35 | //val u = Path.create(prefix).map(prefix => path"$prefix.abc") 36 | 37 | //Check IllTyped 38 | //val x: Root.type = path"" 39 | 40 | val r = Root 41 | 42 | //check IllTyped 43 | //val r1 = path"$r" 44 | //check IllTyped 45 | //val r0 = path"1abc" 46 | //check Illtyped 47 | //val r1 = path"$r/" 48 | 49 | //check Illtyped 50 | //val r2: ArrayPath = path"$r.def/" 51 | 52 | val abc: FieldKey = key"abc" 53 | 54 | assert(abc.name == "abc") 55 | assert(abc.parent == Root) 56 | 57 | val ghi: FieldKey = key"$abc.ghi" 58 | 59 | assert(ghi.name == "ghi") 60 | assert(ghi.parent == abc) 61 | 62 | val lol: FieldKey = key"lol" // 63 | val comp: FieldKey = key"$abc[].$lol" 64 | 65 | assert(comp.name == "lol") 66 | assert(comp.parent == ArrayKey(abc)) 67 | 68 | val comp2: ArrayKey = key"$comp[]" 69 | 70 | assert(comp2.parent == comp) 71 | 72 | val comp3: ArrayKey = key"$comp2" 73 | 74 | assert(comp3 == comp2) 75 | 76 | } 77 | 78 | test("create Key") { 79 | assert( 80 | Key.create("abcd.edfg[][].hijk") == 81 | FieldKey("hijk", ArrayKey(ArrayKey(FieldKey("edfg", FieldKey("abcd", Root).get).get))) 82 | ) 83 | 84 | assert(Key.create("abc[][][]") == Try(ArrayKey(ArrayKey(ArrayKey(FieldKey("abc", Root).get))))) 85 | } 86 | 87 | test("error") { 88 | assert(Key.create("123").isFailure) 89 | } 90 | 91 | test("create special Key") { 92 | assert(FieldKey.createName("12.*+.-3").isFailure) 93 | assert(FieldKey.createName("\"12.*+.-3\"").isSuccess) 94 | } 95 | 96 | test("follow up") { 97 | 98 | assert(Key.create("").get == Root) 99 | assert(Key.create("abc") == FieldKey("abc", Root)) 100 | assert(Key.create("abc.def[]").get == ArrayKey(FieldKey("def", FieldKey("abc", Root).get).get)) 101 | 102 | /* 103 | {:abc {:def 1}} abc.def 104 | {:abc {:def [{:ghi 1} {:ghi 2}]}} abc.def/ghi 105 | */ 106 | 107 | } 108 | 109 | } 110 | --------------------------------------------------------------------------------