├── .gitignore ├── LICENSE ├── NOTICE ├── README.md ├── benchmarks ├── GraphGenerator.scala ├── KMeansDataFrame.scala ├── KMeansSQL.scala ├── NestedBuild.scala ├── NestedFlink.scala ├── NestedFlink2.scala ├── NestedSQL.scala ├── NestedScalding.scala ├── NestedScalding2.scala ├── NestedSpark.scala ├── NestedSpark2.scala ├── PageRankDataFrame.scala ├── PageRankSQL.scala ├── PointGenerator.java ├── buildFlink ├── buildScalding ├── buildSpark ├── comet.build ├── comet.build2 ├── comet.build3 ├── comet.run ├── comet.run2 ├── comet.run3 ├── comet.runFlink ├── comet.runFlink2 ├── comet.runScalding ├── comet.runScalding2 ├── comet.runSpark ├── comet.runSpark2 ├── diablo │ ├── README.md │ ├── add.scala │ ├── build │ ├── casper │ │ ├── ConditionalSumCasper.java │ │ ├── EqualCasper.java │ │ ├── StringMatchCasper.java │ │ └── WordCountCasper.java │ ├── comet-build │ ├── comet-test.run │ ├── comet1.run │ ├── comet2.run │ ├── comet3.run │ ├── compile │ ├── conditionalSum.scala │ ├── equal.scala │ ├── factorization.scala │ ├── groupBy.scala │ ├── histogram.scala │ ├── kmeans.scala │ ├── linearRegression.scala │ ├── multiply.scala │ ├── pagerank.scala │ ├── parallel │ │ ├── build-parallel │ │ ├── build-sequential │ │ ├── comet-run │ │ ├── diablo-par.jar │ │ ├── diablo-seq.jar │ │ ├── parallel.scala │ │ ├── run-parallel │ │ ├── run-sequential │ │ ├── run.log │ │ └── sequential.scala │ ├── run │ ├── run-1.log │ ├── run-2.log │ ├── run-3.log │ ├── run-4.log │ ├── stringMatch.scala │ └── wordCount.scala ├── kmeans.scala ├── old-diablo │ ├── FactorizationGenerator.java │ ├── GraphGenerator.scala │ ├── MatrixGenerator.java │ ├── PointGenerator.java │ ├── add-spark.scala │ ├── add.scala │ ├── build │ ├── comet.build │ ├── comet.run │ ├── factorization-spark.scala │ ├── factorization.scala │ ├── factorization2.scala │ ├── kmeans-spark.scala │ ├── kmeans.scala │ ├── multiply-spark.scala │ ├── multiply.scala │ ├── pagerank-spark.scala │ ├── pagerank.scala │ ├── parallel │ │ ├── build │ │ ├── build-seq │ │ ├── comet.run │ │ ├── factorization-seq.scala │ │ ├── factorization.scala │ │ ├── kmeans-seq.scala │ │ ├── kmeans.scala │ │ ├── kmeans2-seq.scala │ │ ├── kmeans2.scala │ │ ├── multiply-seq.scala │ │ ├── multiply.scala │ │ ├── pagerank-seq.scala │ │ └── pagerank.scala │ └── run ├── pagerank.mrql ├── pagerank.scala ├── pagerank2.scala ├── runFlink ├── runScalding ├── runSpark ├── runSparkDistr └── sqlgen │ ├── .ipynb_checkpoints │ └── SQLgen Benchmark-checkpoint.ipynb │ ├── GroupBy.scala │ ├── GroupByJoin.scala │ ├── Histogram.scala │ ├── LinearRegression.scala │ ├── MatrixAddition.scala │ ├── MatrixFactorization.scala │ ├── MatrixMultiplication.scala │ ├── PCA.scala │ ├── PageRank.scala │ ├── SQLgen Benchmark.ipynb │ ├── StringMatch.scala │ ├── Sum.scala │ ├── WordCount.scala │ ├── group-by-join.eps │ ├── group-by-join.log │ ├── group-by-join.run │ ├── group-by.eps │ ├── group-by.log │ ├── group-by.png │ ├── group-by.run │ ├── groupby.eps │ ├── histogram.eps │ ├── histogram.log │ ├── histogram.run │ ├── linear-regression.eps │ ├── linear-regression.log │ ├── linear-regression.run │ ├── matrix-addition.eps │ ├── matrix-addition.log │ ├── matrix-addition.run │ ├── matrix-addition2.log │ ├── matrix-factorization.eps │ ├── matrix-factorization.log │ ├── matrix-factorization.run │ ├── matrix-multiplication.eps │ ├── matrix-multiplication.png │ ├── matrix-multiplication.run │ ├── matrix-multiplication1.log │ ├── matrix-multiplication2.log │ ├── matrix-multiplication3.log │ ├── page-rank.eps │ ├── page-rank.log │ ├── page-rank.run │ ├── page-rank2.eps │ ├── pca.eps │ ├── pca.log │ ├── pca.run │ ├── pca2.log │ ├── string-match.eps │ ├── string-match.log │ ├── string-match.run │ ├── sum.eps │ ├── sum.log │ ├── sum.png │ ├── sum.run │ ├── word-count.eps │ ├── word-count.log │ ├── word-count.png │ └── word-count.run ├── build.sbt ├── lib └── diql-spark.jar ├── pom-flink.xml ├── pom-parallel.xml ├── pom-scalding.xml ├── pom-sequential.xml ├── pom-spark.xml ├── pom.xml ├── project └── build.properties ├── src ├── diablo │ └── scala │ │ └── edu │ │ └── uta │ │ └── diablo │ │ ├── AST.scala │ │ ├── ComprehensionTranslator.scala │ │ ├── DefaultTranslator.scala │ │ ├── Diablo.scala │ │ ├── Normalizer.scala │ │ ├── Optimizer.scala │ │ ├── Parser.scala │ │ ├── Translator.scala │ │ └── Typechecker.scala ├── flink │ └── scala │ │ └── edu │ │ └── uta │ │ └── diql │ │ ├── FlinkCodeGenerator.scala │ │ └── QueryCodeGenerator.scala ├── main │ └── scala │ │ └── edu │ │ └── uta │ │ └── diql │ │ ├── AST.scala │ │ ├── CodeGeneration.scala │ │ ├── Core.scala │ │ ├── DIQL.scala │ │ ├── Debugger.scala │ │ ├── DistributedCodeGenerator.scala │ │ ├── InMemoryBag.scala │ │ ├── Lineage.scala │ │ ├── Normalizer.scala │ │ ├── Optimizer.scala │ │ ├── Parser.scala │ │ ├── PrettyPrint.scala │ │ ├── Streaming.scala │ │ └── Translator.scala ├── parallel │ └── scala │ │ └── edu │ │ └── uta │ │ └── diql │ │ ├── ParallelCodeGererator.scala │ │ └── QueryCodeGenerator.scala ├── scalding │ └── scala │ │ └── edu │ │ └── uta │ │ └── diql │ │ ├── QueryCodeGenerator.scala │ │ └── ScaldingCodeGenerator.scala ├── sequential │ └── scala │ │ └── edu │ │ └── uta │ │ └── diql │ │ ├── QueryCodeGenerator.scala │ │ └── SequentialCodeGererator.scala ├── spark │ └── scala │ │ └── edu │ │ └── uta │ │ └── diql │ │ ├── QueryCodeGenerator.scala │ │ └── SparkCodeGererator.scala └── sqlgen │ └── scala │ └── edu │ └── uta │ └── sql │ ├── CodeGenerator.scala │ ├── SqlAST.scala │ ├── SqlGen.scala │ └── SqlTranslator.scala └── tests ├── diablo ├── flink │ ├── build │ ├── multiply.scala │ └── run ├── parallel │ ├── add.scala │ ├── avg.scala │ ├── build │ ├── build-seq │ ├── c │ ├── f.txt │ ├── factorization.scala │ ├── graph.txt │ ├── kmeans.scala │ ├── kmeans2.scala │ ├── m.txt │ ├── multiply.scala │ ├── n.txt │ ├── p │ ├── pagerank.scala │ ├── pagerank2.scala │ ├── run │ ├── run-seq │ ├── t.txt │ └── wordCount.scala └── spark │ ├── Average.scala │ ├── Average2.scala │ ├── ConditionalCount.scala │ ├── ConditionalSum.scala │ ├── Count.scala │ ├── Equal.scala │ ├── EqualFrequency.scala │ ├── GraphGenerator.scala │ ├── ImageHistogram.scala │ ├── ImageHistogramSpark.scala │ ├── KMeansSpark.scala │ ├── LinearRegression.scala │ ├── LinearRegressionSpark.scala │ ├── MatrixAddition.scala │ ├── MatrixFactorization.scala │ ├── MatrixFactorization2.scala │ ├── MatrixFactorizationSpark.scala │ ├── MatrixMultiplication.scala │ ├── MatrixMultiplicationSpark.scala │ ├── MatrixMultiplicationSumma.scala │ ├── PCA.scala │ ├── PCASpark.scala │ ├── PageRankSpark.scala │ ├── ReverseIndex.scala │ ├── ReverseIndexSpark.scala │ ├── StringMatch.scala │ ├── StringMatchSpark.scala │ ├── build │ ├── c │ ├── f.txt │ ├── graph.txt │ ├── groupBy.scala │ ├── groupBy2.scala │ ├── ih.txt │ ├── k.txt │ ├── kmeans.scala │ ├── kmeans2.scala │ ├── lr.txt │ ├── m.txt │ ├── n.txt │ ├── p │ ├── pagerank.scala │ ├── pagerank2.scala │ ├── pca.txt │ ├── q │ ├── reverseIndex │ ├── 1.html │ └── 2.html │ ├── run │ ├── small-graph.txt │ ├── sum.scala │ ├── t.txt │ ├── w │ ├── w.txt │ ├── wordCount.scala │ └── wordCountSpark.scala ├── flink ├── build ├── g.txt ├── graph.txt ├── kmeans.scala ├── pagerank.scala ├── points.txt ├── run └── test.scala ├── parallel ├── build ├── graph.txt ├── join.scala ├── kmeans.scala ├── map.scala ├── pagerank.scala ├── points.txt ├── run └── test.scala ├── scalding ├── build ├── graph.txt ├── kmeans.scala ├── pagerank.scala ├── run └── test.scala ├── spark ├── build ├── factorization.scala ├── g.txt ├── graph.txt ├── kmeans.scala ├── m.txt ├── multiply.scala ├── n.txt ├── pagerank.scala ├── points.txt ├── run └── test.scala └── sqlgen └── spark ├── GroupBy.scala ├── GroupBy2.scala ├── GroupByJoin.scala ├── GroupByJoin2.scala ├── Histogram.scala ├── Histogram2.scala ├── MatrixAddition.scala ├── MatrixAddition2.scala ├── MatrixFactorization.scala ├── MatrixMultiplication.scala ├── MatrixMultiplication2.scala ├── MatrixMultiplication3.scala ├── PCA.scala ├── PCA2.scala ├── PCA3.scala ├── PageRank.scala ├── StringMatch.scala ├── StringMatch2.scala ├── StringMatch3.scala ├── StringMatch4.scala ├── StringMatch5.scala ├── Sum.scala ├── Sum2.scala ├── WordCount.scala ├── WordCount2.scala ├── build ├── classes ├── Cclass$.class ├── Cclass.class ├── Test$$typecreator5$1.class ├── Test$.class └── Test.class ├── group-by.txt ├── histogram.txt ├── k.txt ├── key.txt ├── m-matrix-small.txt ├── matrix-fact.txt ├── n-matrix-small.txt ├── page-rank.txt ├── pca.txt ├── run ├── string.txt ├── sum.txt ├── test.jar ├── v.txt └── word-count.txt /.gitignore: -------------------------------------------------------------------------------- 1 | *~ 2 | tmp/* 3 | lib/* 4 | target 5 | project/target 6 | .cache-main 7 | .classpath 8 | .project 9 | .settings 10 | .idea 11 | tests/*/classes 12 | tests/*/test.jar 13 | /target/ 14 | /bin/ 15 | /bin1/ 16 | /tests/sqlgen/spark/classes/ 17 | /benchmarks/sqlgen/.ipynb_checkpoints/ 18 | -------------------------------------------------------------------------------- /NOTICE: -------------------------------------------------------------------------------- 1 | Copyright 2017, University of Texas at Arlington 2 | This project includes software developed by the 3 | University of Texas at Arlington. 4 | 5 | Licensed under the Apache License, Version 2.0 (the 6 | "License"); you may not use this file except in compliance 7 | with the License. You may obtain a copy of the License at: 8 | 9 | http://www.apache.org/licenses/LICENSE-2.0 10 | 11 | Unless required by applicable law or agreed to in writing, 12 | software distributed under the License is distributed on 13 | an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | KIND, either express or implied. See the License for the 15 | specific language governing permissions and limitations 16 | under the License. 17 | 18 | This product includes software developed at 19 | The Apache Software Foundation (http://www.apache.org/). 20 | -------------------------------------------------------------------------------- /benchmarks/NestedBuild.scala: -------------------------------------------------------------------------------- 1 | import org.apache.spark.SparkContext 2 | import org.apache.spark.SparkConf 3 | import scala.util._ 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ): Unit = { 8 | val conf = new SparkConf().setAppName("NestedData") 9 | val sc = new SparkContext(conf) 10 | val n = Integer.parseInt(args(0)) 11 | val CF = args(1) 12 | val OF = args(2) 13 | val rand = new Random() 14 | def randomValue ( n: Int ): Int = Math.round(rand.nextDouble()*n).toInt 15 | sc.parallelize(1 to n).map( _ => (randomValue(n),rand.nextDouble()*1000.0) ) 16 | .map{ case (a,b) => "\"XYZ\","+a+","+b }.repartition(1).saveAsTextFile(CF) 17 | sc.parallelize(1 to 10*n).map( _ => (randomValue(10*n),randomValue(n),rand.nextDouble()*100.0) ) 18 | .map{ case (a,b,c) => a+","+b+","+c }.repartition(1).saveAsTextFile(OF) 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /benchmarks/NestedFlink.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.flink.api.scala._ 3 | import org.apache.flink.core.fs._ 4 | 5 | object Test { 6 | 7 | case class Customer ( name: String, cid: Int, account: Float ) 8 | 9 | case class Order ( oid: Int, cid: Int, price: Float ) 10 | 11 | def main ( args: Array[String] ) { 12 | val CF = args(0) 13 | val OF = args(1) 14 | val output_file = args(2) 15 | val env = ExecutionEnvironment.getExecutionEnvironment 16 | 17 | //explain(true) 18 | 19 | val t: Long = System.currentTimeMillis() 20 | 21 | val customers = env.readTextFile(CF).map{ line => line.split(",") 22 | match { case Array(a,b,c) => Customer(a,b.toInt,c.toFloat) } } 23 | val orders = env.readTextFile(OF).map{ line => line.split(",") 24 | match { case Array(a,b,c) => Order(a.toInt,b.toInt,c.toFloat) } } 25 | 26 | q(""" 27 | select c.name 28 | from c <- customers 29 | where c.account < +/(select o.price from o <- orders where o.cid == c.cid) 30 | """).writeAsText(output_file,FileSystem.WriteMode.OVERWRITE) 31 | 32 | env.execute() 33 | 34 | println("**** DIQL Flink run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 35 | 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /benchmarks/NestedFlink2.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.flink.api.scala._ 3 | import org.apache.flink.core.fs._ 4 | 5 | object Test { 6 | 7 | case class Customer ( name: String, cid: Int, account: Float ) 8 | 9 | case class Order ( oid: Int, cid: Int, price: Float ) 10 | 11 | def main ( args: Array[String] ) { 12 | val CF = args(0) 13 | val OF = args(1) 14 | val output_file = args(2) 15 | val env = ExecutionEnvironment.getExecutionEnvironment 16 | 17 | //explain(true) 18 | 19 | val t: Long = System.currentTimeMillis() 20 | 21 | val customers = env.readTextFile(CF).map{ line => line.split(",") 22 | match { case Array(a,b,c) => Customer(a,b.toInt,c.toFloat) } } 23 | val orders = env.readTextFile(OF).map{ line => line.split(",") 24 | match { case Array(a,b,c) => Order(a.toInt,b.toInt,c.toFloat) } } 25 | 26 | q(""" 27 | select ( k, avg/c.account ) 28 | from c <- customers 29 | where c.account < +/(select o.price from o <- orders where o.cid == c.cid 30 | && count/(select d from d <- customers where o.cid == d.cid) > 1) 31 | group by k: c.account % 10 32 | """).writeAsText(output_file,FileSystem.WriteMode.OVERWRITE) 33 | 34 | env.execute() 35 | 36 | println("**** DIQL Flink run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 37 | 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /benchmarks/NestedSQL.scala: -------------------------------------------------------------------------------- 1 | import org.apache.spark.sql.functions._ 2 | import org.apache.spark.sql._ 3 | import org.apache.spark._ 4 | import org.apache.log4j._ 5 | 6 | object Test { 7 | 8 | case class X ( A: Int, D: Int ) 9 | 10 | case class Y ( B: Int, C: Int ) 11 | 12 | def main ( args: Array[String] ) { 13 | val xfile = args(0) 14 | val yfile = args(1) 15 | val output_file = args(2) 16 | val sparkConf = new SparkConf().setAppName("Spark SQL Nested") 17 | val sc = new SparkContext(sparkConf) 18 | val sqlContext = new SQLContext(sc) 19 | val spark = SparkSession.builder().config(sparkConf).getOrCreate() 20 | 21 | import spark.implicits._ 22 | 23 | sparkConf.set("spark.logConf","false") 24 | sparkConf.set("spark.eventLog.enabled","false") 25 | LogManager.getRootLogger().setLevel(Level.WARN) 26 | 27 | val t: Long = System.currentTimeMillis() 28 | 29 | val XC = spark.sparkContext.textFile(xfile) 30 | .map(_.split(",")).map(n => X(n(0).toInt,n(1).toInt)) 31 | .toDF() 32 | val YC = spark.sparkContext.textFile(yfile) 33 | .map(_.split(",")).map(n => Y(n(0).toInt,n(1).toInt)) 34 | .toDF() 35 | XC.createOrReplaceTempView("X") 36 | YC.createOrReplaceTempView("Y") 37 | 38 | var out = spark.sql(""" 39 | SELECT x.A 40 | FROM X x 41 | WHERE x.D IN (SELECT y.C FROM Y y WHERE x.A=y.B) 42 | """) 43 | 44 | out.write.csv(output_file) 45 | 46 | println("*** Spark SQL run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 47 | 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /benchmarks/NestedScalding.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import com.twitter.scalding._ 3 | 4 | object Test extends ExecutionApp { 5 | 6 | case class Customer ( name: String, cid: Int, account: Float ) 7 | 8 | case class Order ( oid: Int, cid: Int, price: Float ) 9 | 10 | //explain(true) 11 | 12 | def job: Execution[Unit] 13 | = Execution.getArgs.flatMap { 14 | case args 15 | => val CF = args("CF") 16 | val OF = args("OF") 17 | val output_file = args("out") 18 | val customers = TypedPipe.from(TextLine(CF)).map{ line => line.split(",") 19 | match { case Array(a,b,c) => Customer(a,b.toInt,c.toFloat) } } 20 | val orders = TypedPipe.from(TextLine(OF)).map{ line => line.split(",") 21 | match { case Array(a,b,c) => Order(a.toInt,b.toInt,c.toFloat) } } 22 | q(""" 23 | select c.name 24 | from c <- customers 25 | where c.account < +/(select o.price from o <- orders where o.cid == c.cid) 26 | """).writeExecution(TypedTsv("out")) 27 | } 28 | 29 | override def main ( args: Array[String] ) { 30 | val t: Long = System.currentTimeMillis() 31 | super.main(args) 32 | println("**** DIQL Scalding run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /benchmarks/NestedScalding2.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import com.twitter.scalding._ 3 | 4 | object Test extends ExecutionApp { 5 | 6 | case class Customer ( name: String, cid: Int, account: Float ) 7 | 8 | case class Order ( oid: Int, cid: Int, price: Float ) 9 | 10 | explain(true) 11 | 12 | def job: Execution[Unit] 13 | = Execution.getArgs.flatMap { 14 | case args 15 | => val CF = args("CF") 16 | val OF = args("OF") 17 | val output_file = args("out") 18 | val customers = TypedPipe.from(TextLine(CF)).map{ line => line.split(",") 19 | match { case Array(a,b,c) => Customer(a,b.toInt,c.toFloat) } } 20 | val orders = TypedPipe.from(TextLine(OF)).map{ line => line.split(",") 21 | match { case Array(a,b,c) => Order(a.toInt,b.toInt,c.toFloat) } } 22 | q(""" 23 | select ( k, avg/c.account ) 24 | from c <- customers 25 | where c.account < +/(select o.price from o <- orders where o.cid == c.cid 26 | && count/(select d from d <- customers where o.cid == d.cid) > 1) 27 | group by k: c.account % 10 28 | """).writeExecution(TypedTsv("out")) 29 | } 30 | 31 | override def main ( args: Array[String] ) { 32 | val t: Long = System.currentTimeMillis() 33 | super.main(args) 34 | println("**** DIQL Scalding run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /benchmarks/NestedSpark.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.log4j._ 5 | 6 | object Test { 7 | 8 | case class Customer ( name: String, cid: Int, account: Float ) 9 | 10 | case class Order ( oid: Int, cid: Int, price: Float ) 11 | 12 | def main ( args: Array[String] ) { 13 | val CF = args(0) 14 | val OF = args(1) 15 | val output_file = args(2) 16 | val conf = new SparkConf().setAppName("Nested") 17 | val sc = new SparkContext(conf) 18 | 19 | conf.set("spark.logConf","false") 20 | conf.set("spark.eventLog.enabled","false") 21 | LogManager.getRootLogger().setLevel(Level.WARN) 22 | 23 | //explain(true) 24 | val t: Long = System.currentTimeMillis() 25 | 26 | val customers = sc.textFile(CF).map{ line => line.split(",") 27 | match { case Array(a,b,c) => Customer(a,b.toInt,c.toFloat) } } 28 | val orders = sc.textFile(OF).map{ line => line.split(",") 29 | match { case Array(a,b,c) => Order(a.toInt,b.toInt,c.toFloat) } } 30 | 31 | q(""" 32 | select c.name 33 | from c <- customers 34 | where c.account < +/(select o.price from o <- orders where o.cid == c.cid) 35 | """).saveAsTextFile(output_file) 36 | 37 | sc.stop() 38 | 39 | println("**** DIQL Spark run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /benchmarks/NestedSpark2.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.log4j._ 5 | 6 | object Test { 7 | 8 | case class Customer ( name: String, cid: Int, account: Float ) 9 | 10 | case class Order ( oid: Int, cid: Int, price: Float ) 11 | 12 | def main ( args: Array[String] ) { 13 | val CF = args(0) 14 | val OF = args(1) 15 | val output_file = args(2) 16 | val conf = new SparkConf().setAppName("Nested") 17 | val sc = new SparkContext(conf) 18 | 19 | conf.set("spark.logConf","false") 20 | conf.set("spark.eventLog.enabled","false") 21 | LogManager.getRootLogger().setLevel(Level.WARN) 22 | 23 | //explain(true) 24 | val t: Long = System.currentTimeMillis() 25 | 26 | val customers = sc.textFile(CF).map{ line => line.split(",") 27 | match { case Array(a,b,c) => Customer(a,b.toInt,c.toFloat) } } 28 | val orders = sc.textFile(OF).map{ line => line.split(",") 29 | match { case Array(a,b,c) => Order(a.toInt,b.toInt,c.toFloat) } } 30 | 31 | q(""" 32 | select ( k, avg/c.account ) 33 | from c <- customers 34 | where c.account < +/(select o.price from o <- orders where o.cid == c.cid 35 | && count/(select d from d <- customers where o.cid == d.cid) > 1) 36 | group by k: c.account % 10 37 | """).saveAsTextFile(output_file) 38 | 39 | sc.stop() 40 | 41 | println("**** DIQL Spark run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /benchmarks/PointGenerator.java: -------------------------------------------------------------------------------- 1 | /* Points in a 100*100 grid used by KMeans 2 | */ 3 | 4 | import java.io.BufferedWriter; 5 | import java.io.FileWriter; 6 | import java.util.Random; 7 | 8 | public class PointGenerator { 9 | static Random rand = new Random(); 10 | 11 | static double getd () { 12 | double v = rand.nextDouble()*20.0D; 13 | return ((int)v % 2 == 0) ? getd() : v; 14 | } 15 | 16 | public static void main ( String[] args ) throws Exception { 17 | long points = Long.parseLong(args[0]); 18 | BufferedWriter centroids = new BufferedWriter(new FileWriter(args[1])); 19 | for ( int i = 0; i < 10; i++ ) 20 | for ( int j = 0; j < 10; j++ ) 21 | centroids.write((i*2+1.2)+","+(j*2+1.2)+"\n"); 22 | centroids.close(); 23 | BufferedWriter data = new BufferedWriter(new FileWriter(args[2])); 24 | for ( long i = 0; i < points; i++ ) { 25 | double x = getd(); 26 | double y = getd(); 27 | data.write(x+","+y+"\n"); 28 | } 29 | data.close(); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /benchmarks/buildFlink: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME=~/diql 4 | 5 | if [ -z ${FLINK_HOME} ]; then 6 | FLINK_HOME=~/flink-1.2.1 7 | fi 8 | 9 | JARS=. 10 | for I in ${FLINK_HOME}/lib/*.jar; do 11 | JARS=${JARS}:$I 12 | done 13 | 14 | mkdir -p classes 15 | env JAVA_OPTS="-Xmx1G" scalac -d classes -cp ${JARS}:${DIQL_HOME}/lib/diql-flink.jar $* 16 | -------------------------------------------------------------------------------- /benchmarks/buildScalding: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME=~/diql 4 | 5 | if [ -z ${SCALDING_HOME} ]; then 6 | SCALDING_HOME=${HOME}/scalding 7 | fi 8 | if [ -z ${HADOOP_HOME} ]; then 9 | HADOOP_HOME=${HOME}/hadoop-2.6.0 10 | fi 11 | 12 | HADOOP_JARS=`${HADOOP_HOME}/bin/hadoop classpath` 13 | JARS=.:${HADOOP_JARS} 14 | for I in ${SCALDING_HOME}/scalding-core/target/scala-2.11/scalding-core-assembly*.jar; do 15 | JARS=${JARS}:$I 16 | done 17 | 18 | mkdir -p classes 19 | scalac -d classes -cp ${JARS}:${DIQL_HOME}/lib/diql-scalding.jar $* 20 | -------------------------------------------------------------------------------- /benchmarks/buildSpark: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME=~/diql 4 | 5 | if [ -z ${SPARK_HOME} ]; then 6 | SPARK_HOME=~/spark-2.1.0-bin-hadoop2.6 7 | fi 8 | 9 | JARS=. 10 | for I in ${SPARK_HOME}/jars/*.jar; do 11 | JARS=${JARS}:$I 12 | done 13 | for I in ${SPARK_HOME}/lib/*.jar; do 14 | JARS=${JARS}:$I 15 | done 16 | 17 | mkdir -p classes 18 | scalac -d classes -cp ${JARS}:${DIQL_HOME}/lib/diql-spark.jar $* 19 | -------------------------------------------------------------------------------- /benchmarks/comet.build: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot143 3 | #SBATCH --job-name="build" 4 | #SBATCH --output="build.out" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=3 7 | #SBATCH --export=ALL 8 | #SBATCH --time=300 9 | 10 | nodes=$SLURM_NNODES 11 | echo "Number of nodes = " $nodes 12 | # executor-cores=5, executor-memory=24G, num-executors=((nodes-1)*24/5-1) 13 | executors=$(((nodes-1)*24/5-1)) 14 | echo "Number of executors = " $executors 15 | 16 | # location of spark, scala, and diql 17 | export SW=/oasis/projects/nsf/uot143/fegaras 18 | # directory on local disk to store the dataset 19 | DATA=/oasis/projects/nsf/uot143/$USER/data 20 | 21 | export HADOOP_CONF_DIR=$HOME/cometcluster 22 | module load hadoop/2.6.0 23 | 24 | export SCALA_HOME=$SW/scala-2.11.8 25 | export SPARK_HOME=$SW/spark-2.1.0-bin-hadoop2.6 26 | export DIQL_HOME=$SW/diql 27 | 28 | myhadoop-configure.sh 29 | source $HOME/cometcluster/spark/spark-env.sh 30 | export SPARK_MASTER_HOST=$SPARK_MASTER_IP 31 | start-dfs.sh 32 | $SPARK_HOME/sbin/start-all.sh -h $SPARK_MASTER_HOST 33 | 34 | JARS=. 35 | for I in $SPARK_HOME/jars/*.jar; do 36 | JARS=$JARS:$I 37 | done 38 | 39 | mkdir -p $HOME/classes 40 | $SCALA_HOME/bin/scalac -d $HOME/classes -cp $JARS $DIQL_HOME/benchmarks/GraphGenerator.scala 41 | jar cf $HOME/rmat.jar -C $HOME/classes . 42 | 43 | SPARK_OPTIONS="--driver-memory 8G --num-executors $executors --executor-cores 5 --executor-memory 24G --supervise --verbose" 44 | 45 | mkdir -p $DATA 46 | rm -rf $DATA/* 47 | hdfs dfs -mkdir -p /user/$USER 48 | for ((i=1; i<=8; i++)); do 49 | $SPARK_HOME/bin/spark-submit --class Test --master $MASTER $SPARK_OPTIONS $HOME/rmat.jar $((2000000*i)) $((20000000*i)) /user/$USER/graph.txt 84 50 | hdfs dfs -ls -h /user/$USER/graph.txt 51 | hdfs dfs -get /user/$USER/graph.txt $DATA/G$i 52 | hdfs dfs -rm -r /user/$USER/graph.txt 53 | done 54 | 55 | $SPARK_HOME/sbin/stop-all.sh 56 | stop-dfs.sh 57 | myhadoop-cleanup.sh 58 | -------------------------------------------------------------------------------- /benchmarks/comet.build2: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot143 3 | #SBATCH --job-name="build2" 4 | #SBATCH --output="build2.out" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=3 7 | #SBATCH --export=ALL 8 | #SBATCH --time=300 9 | 10 | nodes=$SLURM_NNODES 11 | echo "Number of nodes = " $nodes 12 | # executor-cores=5, executor-memory=24G, num-executors=((nodes-1)*24/5-1) 13 | executors=$(((nodes-1)*24/5-1)) 14 | echo "Number of executors = " $executors 15 | 16 | # location of spark, scala, and diql 17 | export SW=/oasis/projects/nsf/uot143/fegaras 18 | # directory on local disk to store the dataset 19 | DATA=/oasis/projects/nsf/uot143/$USER/data 20 | 21 | export HADOOP_CONF_DIR=$HOME/cometcluster 22 | module load hadoop/2.6.0 23 | 24 | export SCALA_HOME=$SW/scala-2.11.8 25 | export SPARK_HOME=$SW/spark-2.1.0-bin-hadoop2.6 26 | export DIQL_HOME=$SW/diql 27 | 28 | myhadoop-configure.sh 29 | source $HOME/cometcluster/spark/spark-env.sh 30 | export SPARK_MASTER_HOST=$SPARK_MASTER_IP 31 | start-dfs.sh 32 | $SPARK_HOME/sbin/start-all.sh -h $SPARK_MASTER_HOST 33 | 34 | JARS=. 35 | for I in $SPARK_HOME/jars/*.jar; do 36 | JARS=$JARS:$I 37 | done 38 | 39 | mkdir -p $HOME/classes 40 | $SCALA_HOME/bin/scalac -d $HOME/classes -cp $JARS $DIQL_HOME/benchmarks/NestedBuild.scala 41 | jar cf $HOME/nested.jar -C $HOME/classes . 42 | 43 | SPARK_OPTIONS="--driver-memory 8G --num-executors $executors --executor-cores 5 --executor-memory 24G --supervise --verbose" 44 | 45 | mkdir -p $DATA 46 | rm -rf $DATA/S* $DATA/O* 47 | hdfs dfs -mkdir -p /user/$USER 48 | for ((i=1; i<=8; i++)); do 49 | $SPARK_HOME/bin/spark-submit --class Test --master $MASTER $SPARK_OPTIONS $HOME/nested.jar $((i*4000000)) /user/$USER/S.txt /user/$USER/O.txt 50 | hdfs dfs -get /user/$USER/S.txt/part-00000 $DATA/S$i 51 | hdfs dfs -get /user/$USER/O.txt/part-00000 $DATA/O$i 52 | hdfs dfs -rm -r /user/$USER/S.txt /user/$USER/O.txt 53 | done 54 | 55 | $SPARK_HOME/sbin/stop-all.sh 56 | stop-dfs.sh 57 | myhadoop-cleanup.sh 58 | -------------------------------------------------------------------------------- /benchmarks/comet.build3: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot143 3 | #SBATCH --job-name="build3" 4 | #SBATCH --output="build3.out" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=1 7 | #SBATCH --export=ALL 8 | #SBATCH --time=100 9 | 10 | # location of spark, scala, and diql 11 | export SW=/oasis/projects/nsf/uot143/fegaras 12 | # directory on local disk to store the dataset 13 | DATA=/oasis/projects/nsf/uot143/$USER/data 14 | 15 | export DIQL_HOME=$SW/diql 16 | 17 | javac -d $HOME/classes $DIQL_HOME/benchmarks/PointGenerator.java 18 | 19 | for ((i=1; i<=8; i++)); do 20 | java -cp $HOME/classes PointGenerator $((i*20000000)) $DATA/C$i $DATA/P$i 21 | done 22 | -------------------------------------------------------------------------------- /benchmarks/comet.runFlink: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot143 3 | #SBATCH --job-name="diql" 4 | #SBATCH --output="runFlink.log" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=10 7 | #SBATCH --export=ALL 8 | #SBATCH --time=500 9 | 10 | nodes=$SLURM_NNODES 11 | echo "Number of nodes = " $nodes 12 | 13 | # location of spark, scala, and diql 14 | SW=/oasis/projects/nsf/uot143/fegaras 15 | # directory on local disk to get the datasets 16 | DATA=/oasis/projects/nsf/uot143/$USER/data 17 | 18 | export HADOOP_CONF_DIR=$HOME/cometcluster 19 | module load hadoop/2.6.0 20 | 21 | export SCALA_HOME=$SW/scala 22 | export FLINK_HOME=$SW/flink-1.2.1 23 | export DIQL_HOME=$SW/diql 24 | 25 | myhadoop-configure.sh 26 | 27 | # start HDFS and Yarn 28 | start-dfs.sh 29 | start-yarn.sh 30 | 31 | JARS=.:$DIQL_HOME/lib/diql-flink.jar 32 | for I in ${FLINK_HOME}/lib/*.jar; do 33 | JARS=${JARS}:$I 34 | done 35 | 36 | rm -rf $HOME/classes 37 | mkdir -p $HOME/classes 38 | pushd $HOME/classes 39 | jar xf $DIQL_HOME/lib/diql-flink.jar 40 | popd 41 | $SCALA_HOME/bin/scalac -d $HOME/classes -cp $JARS $DIQL_HOME/benchmarks/NestedFlink.scala 42 | jar cf $HOME/nested-flink.jar -C $HOME/classes . 43 | 44 | hdfs dfs -mkdir -p /user/$USER /tmp /user/$USER/tmp 45 | for ((i=1; i<=8; i++)); do # for each dataset 46 | hdfs dfs -rm -r -f /user/$USER/out /user/$USER/C.txt /user/$USER/O.txt 47 | hdfs dfs -put $DATA/C$i /user/$USER/C.txt 48 | hdfs dfs -put $DATA/O$i /user/$USER/O.txt 49 | for ((j=1; j<=4; j++)); do # repeat experiments 4 times 50 | echo "@@@ dataset: $i, $j" 51 | $FLINK_HOME/bin/flink run -q -m yarn-cluster -yn 18 -yjm 4096 -ytm 4096 -c Test $HOME/nested-flink.jar hdfs:///user/$USER/C.txt hdfs:///user/$USER/O.txt hdfs:///user/$USER/out 52 | hdfs dfs -rm -r /user/$USER/out 53 | done 54 | done 55 | 56 | stop-yarn.sh 57 | stop-dfs.sh 58 | myhadoop-cleanup.sh 59 | -------------------------------------------------------------------------------- /benchmarks/comet.runFlink2: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot143 3 | #SBATCH --job-name="diql" 4 | #SBATCH --output="runFlink2.log" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=4 7 | #SBATCH --ntasks-per-node=24 8 | #SBATCH --mem=120G 9 | #SBATCH --export=ALL 10 | #SBATCH --time=500 11 | 12 | nodes=$SLURM_NNODES 13 | echo "Number of nodes = " $nodes 14 | 15 | # location of spark, scala, and diql 16 | SW=/oasis/projects/nsf/uot143/fegaras 17 | # directory on local disk to get the datasets 18 | DATA=/oasis/projects/nsf/uot143/$USER/data 19 | 20 | export HADOOP_CONF_DIR=$HOME/cometcluster 21 | module load hadoop/2.6.0 22 | 23 | export SCALA_HOME=$SW/scala-2.11.8 24 | export FLINK_HOME=$SW/flink-1.2.1 25 | export DIQL_HOME=$SW/diql 26 | 27 | myhadoop-configure.sh 28 | 29 | # start HDFS and Yarn 30 | start-dfs.sh 31 | start-yarn.sh 32 | 33 | JARS=.:$DIQL_HOME/lib/diql-flink.jar 34 | for I in ${FLINK_HOME}/lib/*.jar; do 35 | JARS=${JARS}:$I 36 | done 37 | 38 | rm -rf $HOME/classes 39 | mkdir -p $HOME/classes 40 | pushd $HOME/classes 41 | jar xf $DIQL_HOME/lib/diql-flink.jar 42 | popd 43 | $SCALA_HOME/bin/scalac -d $HOME/classes -cp $JARS $DIQL_HOME/benchmarks/NestedFlink2.scala 44 | jar cf $HOME/nested-flink2.jar -C $HOME/classes . 45 | 46 | hdfs dfs -mkdir -p /user/$USER /tmp /user/$USER/tmp 47 | for ((i=1; i<=8; i++)); do # for each dataset 48 | hdfs dfs -rm -r -f /user/$USER/out /user/$USER/S.txt /user/$USER/O.txt 49 | hdfs dfs -put $DATA/S$i /user/$USER/S.txt 50 | hdfs dfs -put $DATA/O$i /user/$USER/O.txt 51 | for ((j=1; j<=4; j++)); do # repeat experiments 4 times 52 | echo "@@@ dataset: $i, $j" 53 | $FLINK_HOME/bin/flink run -q -m yarn-cluster -yn 4 -ys 4 -yjm 4096 -ytm 4096 -c Test $HOME/nested-flink2.jar hdfs:///user/$USER/S.txt hdfs:///user/$USER/O.txt hdfs:///user/$USER/out 54 | hdfs dfs -rm -r /user/$USER/out 55 | done 56 | done 57 | 58 | stop-yarn.sh 59 | stop-dfs.sh 60 | myhadoop-cleanup.sh 61 | -------------------------------------------------------------------------------- /benchmarks/comet.runScalding: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot143 3 | #SBATCH --job-name="diql" 4 | #SBATCH --output="runScalding.log" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=4 7 | #SBATCH --ntasks-per-node=24 8 | #SBATCH --mem=120G 9 | #SBATCH --export=ALL 10 | #SBATCH --time=1000 11 | 12 | nodes=$SLURM_NNODES 13 | echo "Number of nodes = " $nodes 14 | 15 | # location of spark, scala, and diql 16 | SW=/oasis/projects/nsf/uot143/fegaras 17 | # directory on local disk to get the datasets 18 | DATA=/oasis/projects/nsf/uot143/$USER/data 19 | 20 | export HADOOP_CONF_DIR=$HOME/cometcluster 21 | module load hadoop/2.6.0 22 | 23 | export SCALA_HOME=$SW/scala 24 | export SCALDING_HOME=$SW/scalding 25 | export DIQL_HOME=$SW/diql 26 | 27 | myhadoop-configure.sh 28 | 29 | # start HDFS and Yarn 30 | start-dfs.sh 31 | start-yarn.sh 32 | 33 | rm -rf $HOME/classes 34 | mkdir -p $HOME/classes 35 | HADOOP_JARS=`${HADOOP_HOME}/bin/hadoop classpath` 36 | pushd $HOME/classes 37 | jar xf ${DIQL_HOME}/lib/diql-scalding.jar 38 | JARS=.:${HADOOP_JARS} 39 | for I in ${SCALDING_HOME}/scalding-core/target/scala-2.11/scalding-core-assembly*.jar; do 40 | JARS=${JARS}:$I 41 | jar xf $I 42 | done 43 | popd 44 | 45 | $SCALA_HOME/bin/scalac -d $HOME/classes -cp ${JARS}:${DIQL_HOME}/lib/diql-scalding.jar $DIQL_HOME/benchmarks/NestedScalding.scala 46 | jar cf $HOME/nested-scalding.jar -C $HOME/classes . 47 | 48 | hdfs dfs -mkdir -p /user/$USER /tmp /user/$USER/tmp 49 | for ((i=1; i<=8; i++)); do # for each dataset 50 | hdfs dfs -rm -r -f /user/$USER/out /user/$USER/C.txt /user/$USER/O.txt 51 | hdfs dfs -put $DATA/C$i /user/$USER/C.txt 52 | hdfs dfs -put $DATA/O$i /user/$USER/O.txt 53 | for ((j=1; j<=4; j++)); do # repeat experiments 4 times 54 | echo "@@@ dataset: $i, $j" 55 | hadoop jar $HOME/nested-scalding.jar Test --hdfs --CF /user/$USER/C.txt --OF /user/$USER/O.txt --out /user/$USER/out 56 | sleep 200 57 | hdfs dfs -rm -r /user/$USER/out 58 | done 59 | done 60 | 61 | stop-yarn.sh 62 | stop-dfs.sh 63 | myhadoop-cleanup.sh 64 | -------------------------------------------------------------------------------- /benchmarks/comet.runScalding2: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot143 3 | #SBATCH --job-name="diql" 4 | #SBATCH --output="runScalding2.log" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=8 7 | #SBATCH --ntasks-per-node=24 8 | #SBATCH --mem=120G 9 | #SBATCH --export=ALL 10 | #SBATCH --time=1000 11 | 12 | nodes=$SLURM_NNODES 13 | echo "Number of nodes = " $nodes 14 | 15 | # location of spark, scala, and diql 16 | SW=/oasis/projects/nsf/uot143/fegaras 17 | # directory on local disk to get the datasets 18 | DATA=/oasis/projects/nsf/uot143/$USER/data 19 | 20 | export HADOOP_CONF_DIR=$HOME/cometcluster 21 | module load hadoop/2.6.0 22 | 23 | export SCALA_HOME=$SW/scala-2.11.8 24 | export SCALDING_HOME=$SW/scalding 25 | export DIQL_HOME=$SW/diql 26 | 27 | myhadoop-configure.sh 28 | 29 | # start HDFS and Yarn 30 | start-dfs.sh 31 | start-yarn.sh 32 | 33 | rm -rf $HOME/classes 34 | mkdir -p $HOME/classes 35 | HADOOP_JARS=`${HADOOP_HOME}/bin/hadoop classpath` 36 | pushd $HOME/classes 37 | jar xf ${DIQL_HOME}/lib/diql-scalding.jar 38 | JARS=.:${HADOOP_JARS} 39 | for I in ${SCALDING_HOME}/scalding-core/target/scala-2.11/scalding-core-assembly*.jar; do 40 | JARS=${JARS}:$I 41 | jar xf $I 42 | done 43 | popd 44 | 45 | $SCALA_HOME/bin/scalac -d $HOME/classes -cp ${JARS}:${DIQL_HOME}/lib/diql-scalding.jar $DIQL_HOME/benchmarks/NestedScalding2.scala 46 | jar cf $HOME/nested-scalding2.jar -C $HOME/classes . 47 | 48 | hdfs dfs -mkdir -p /user/$USER /tmp /user/$USER/tmp 49 | for ((i=1; i<=8; i++)); do # for each dataset 50 | hdfs dfs -rm -r -f /user/$USER/out /user/$USER/S.txt /user/$USER/O.txt 51 | hdfs dfs -put $DATA/S$i /user/$USER/S.txt 52 | hdfs dfs -put $DATA/O$i /user/$USER/O.txt 53 | for ((j=1; j<=4; j++)); do # repeat experiments 4 times 54 | echo "@@@ dataset: $i, $j" 55 | hadoop jar $HOME/nested-scalding2.jar Test --hdfs --CF /user/$USER/S.txt --OF /user/$USER/O.txt --out /user/$USER/out 56 | sleep 200 57 | hdfs dfs -rm -r /user/$USER/out 58 | done 59 | done 60 | 61 | stop-yarn.sh 62 | stop-dfs.sh 63 | myhadoop-cleanup.sh 64 | -------------------------------------------------------------------------------- /benchmarks/diablo/README.md: -------------------------------------------------------------------------------- 1 | # DIABLO: Benchmarks 2 | 3 | The DIABLO benchmarks were evaluated on [SDSC Comet](https://portal.xsede.org/sdsc-comet). 4 | The SBATCH shell scripts used to run the benchmarks are comet1.run, comet2.run, and comet3.run. 5 | The log files generated by the scripts that contain the run times are run-1.log, run-2.log, and run-3.log, respectively. 6 | 7 | The cluster should support Slurm Workload Manager, Hadoop 2.6, and myhadoop. 8 | 9 | The DIQL and benchmark binaries are already provided (lib/diql-spark.jar and benchmarks/diablo/test.jar). 10 | You may recompile DIQL using `mvn install` on the top directory and compile the test files using `./compile` on the benchmarks/diablo directory. 11 | 12 | Steps to run the scripts on Comet (or on any Slurm-managed cluster): 13 | 14 | 1. Install [Scala 2.11](https://downloads.lightbend.com/scala/2.11.8/scala-2.11.8.tgz). 15 | 2. Install [Spark 2.2 on Hadoop 2.6](https://archive.apache.org/dist/spark/spark-2.2.0/spark-2.2.0-bin-hadoop2.6.tgz). 16 | 3. Change SCALA_HOME and SPARK_HOME in the SBATCH scripts to point to your installations. 17 | 4. Execute the scripts using sbatch, eg, `sbatch comet1.run`. 18 | 19 | The benchmarks that compare DIABLO on parallel Scala vs Scala lists are in the directory: benchmarks/diablo/parallel. 20 | You compile with `mvn -f pom-parallel.xml install` and `mvn -f pom-sequential.xml install` on the top directory and 21 | `./build-parallel` and `./build-sequential` on this directory. 22 | Use `sbatch comet-run` to run these benchmarks on one Comet node. The log file from the 23 | evaluations is run.log. 24 | -------------------------------------------------------------------------------- /benchmarks/diablo/build: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../..; pwd -P)" 4 | 5 | if [ -z ${SPARK_HOME} ]; then 6 | SPARK_HOME=~/spark-2.2.0-bin-hadoop2.6 7 | fi 8 | 9 | JARS=. 10 | for I in ${SPARK_HOME}/jars/*.jar; do 11 | JARS=${JARS}:$I 12 | done 13 | for I in ${SPARK_HOME}/lib/*.jar; do 14 | JARS=${JARS}:$I 15 | done 16 | 17 | mkdir -p classes 18 | scalac -d classes -cp classes:${JARS}:${DIQL_HOME}/lib/diql-spark.jar $* 19 | -------------------------------------------------------------------------------- /benchmarks/diablo/casper/ConditionalSumCasper.java: -------------------------------------------------------------------------------- 1 | import org.apache.spark.SparkConf; 2 | import org.apache.spark.api.java.JavaSparkContext; 3 | import org.apache.spark.api.java.JavaPairRDD; 4 | import org.apache.spark.api.java.JavaRDD; 5 | import org.apache.spark.api.java.function.Function2; 6 | import org.apache.spark.api.java.function.PairFlatMapFunction; 7 | import scala.Tuple2; 8 | import java.util.ArrayList; 9 | import java.util.Map; 10 | import java.util.List; 11 | import java.util.Arrays; 12 | import java.lang.Integer; 13 | import java.util.Iterator; 14 | 15 | public class ConditionalSumCasper { 16 | 17 | public static double sumList(JavaRDD rdd_0_0) { 18 | double sum = 0; 19 | sum = 0; 20 | { 21 | 22 | JavaPairRDD mapEmits = rdd_0_0.flatMapToPair(new PairFlatMapFunction() { 23 | public Iterator> call(java.lang.Double data_i) throws Exception { 24 | List> emits = new ArrayList>(); 25 | 26 | if(data_i < 100) emits.add(new Tuple2(0,data_i)); 27 | 28 | 29 | return emits.iterator(); 30 | } 31 | }); 32 | 33 | JavaPairRDD reduceEmits = mapEmits.reduceByKey(new Function2(){ 34 | public Double call(Double val1, Double val2) throws Exception { 35 | return val2 + val1; 36 | } 37 | }); 38 | 39 | Map output_rdd_0_0 = reduceEmits.collectAsMap(); 40 | sum = output_rdd_0_0.get(0); 41 | } 42 | return sum; 43 | } 44 | 45 | public ConditionalSumCasper() { super(); } 46 | } 47 | -------------------------------------------------------------------------------- /benchmarks/diablo/casper/EqualCasper.java: -------------------------------------------------------------------------------- 1 | import org.apache.spark.SparkConf; 2 | import org.apache.spark.api.java.JavaSparkContext; 3 | import org.apache.spark.api.java.JavaPairRDD; 4 | import org.apache.spark.api.java.JavaRDD; 5 | import org.apache.spark.api.java.function.Function2; 6 | import org.apache.spark.api.java.function.PairFlatMapFunction; 7 | import scala.Tuple2; 8 | import java.util.ArrayList; 9 | import java.util.Map; 10 | import java.util.List; 11 | import java.util.Arrays; 12 | import java.lang.Integer; 13 | import java.util.Iterator; 14 | 15 | public class EqualCasper { 16 | 17 | public static boolean equal(JavaRDD rdd_0_0) { 18 | boolean equal = false; 19 | equal = true; 20 | String val_final = rdd_0_0.first(); 21 | { 22 | 23 | JavaPairRDD mapEmits = rdd_0_0.flatMapToPair(new PairFlatMapFunction() { 24 | public Iterator> call(java.lang.String data_i) throws Exception { 25 | List> emits = new ArrayList>(); 26 | 27 | if(val_final != data_i) emits.add(new Tuple2(0,false)); 28 | 29 | 30 | return emits.iterator(); 31 | } 32 | }); 33 | 34 | JavaPairRDD reduceEmits = mapEmits.reduceByKey(new Function2(){ 35 | public Boolean call(Boolean val1, Boolean val2) throws Exception { 36 | return val2 || val2; 37 | } 38 | }); 39 | 40 | Map output_rdd_0_0 = reduceEmits.collectAsMap(); 41 | equal = output_rdd_0_0.get(0); 42 | } 43 | return equal; 44 | } 45 | 46 | public EqualCasper() { super(); } 47 | } 48 | -------------------------------------------------------------------------------- /benchmarks/diablo/casper/StringMatchCasper.java: -------------------------------------------------------------------------------- 1 | package generated.phoenix; 2 | 3 | import org.apache.spark.api.java.JavaRDD; 4 | import scala.Tuple2; 5 | 6 | import java.util.ArrayList; 7 | import java.util.List; 8 | import java.util.Map; 9 | 10 | public class StringMatchCasper { 11 | public static boolean[] matchWords(JavaRDD rdd_0_0) { 12 | String key1 = "key1"; 13 | String key2 = "key2"; 14 | String key3 = "key3"; 15 | boolean foundKey1 = false; 16 | boolean foundKey2 = false; 17 | boolean foundKey3 = false; 18 | int i = 0; 19 | Map rdd_0_0_output = rdd_0_0.flatMapToPair(words_i -> { 20 | List> emits = new ArrayList>(); 21 | if (words_i.equals(key3)) emits.add(new Tuple2(2,true)); 22 | if (key1.equals(words_i)) emits.add(new Tuple2(3,true)); 23 | if (key2.equals(words_i)) emits.add(new Tuple2(1,true)); 24 | return emits.iterator(); 25 | }).reduceByKey((v1,v2) -> v2 || v1).collectAsMap(); 26 | if (rdd_0_0_output.containsKey(2)) foundKey3 = rdd_0_0_output.get(2); 27 | if (rdd_0_0_output.containsKey(3)) foundKey1 = rdd_0_0_output.get(3); 28 | if (rdd_0_0_output.containsKey(1)) foundKey2 = rdd_0_0_output.get(1); 29 | boolean[] res = { foundKey1, foundKey2, foundKey3 }; 30 | return (boolean[]) res; 31 | } 32 | 33 | public StringMatchCasper() { super(); } 34 | } 35 | -------------------------------------------------------------------------------- /benchmarks/diablo/comet-build: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot143 3 | #SBATCH --job-name="diablo" 4 | #SBATCH --output="comet-build.log" 5 | #SBATCH --partition=shared 6 | #SBATCH --nodes=1 7 | #SBATCH --ntasks-per-node=1 8 | #SBATCH --export=ALL 9 | #SBATCH --time=20 10 | 11 | # location of scala and spark 12 | SW=/oasis/projects/nsf/uot143/$USER 13 | 14 | DIQL_HOME=$SW/diql 15 | 16 | if [ -z ${SCALA_HOME} ]; then 17 | SCALA_HOME=$SW/scala-2.11.8 18 | fi 19 | 20 | if [ -z ${SPARK_HOME} ]; then 21 | SPARK_HOME=$SW/spark-2.2.0-bin-hadoop2.6 22 | fi 23 | 24 | JARS=. 25 | for I in ${SPARK_HOME}/jars/*.jar; do 26 | JARS=${JARS}:$I 27 | done 28 | 29 | rm -rf classes 30 | mkdir -p classes 31 | 32 | java_files=`ls casper/*.java` 33 | for f in $java_files; do 34 | echo compiling $f ... 35 | javac -d classes -cp ${JARS} $f 36 | done 37 | 38 | scala_files=`ls *.scala` 39 | for f in $scala_files; do 40 | echo compiling $f ... 41 | $SCALA_HOME/bin/scalac -d classes -cp classes:${JARS}:${DIQL_HOME}/lib/diql-spark.jar $f 42 | done 43 | 44 | jar cf test.jar -C classes . 45 | -------------------------------------------------------------------------------- /benchmarks/diablo/comet-test.run: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot143 3 | #SBATCH --job-name="diablo" 4 | #SBATCH --output="run-test.log" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=10 7 | #SBATCH --export=ALL 8 | #SBATCH --time=100 # time limit in minutes 9 | 10 | nodes=$SLURM_NNODES 11 | echo "Number of nodes = " $nodes 12 | # executor-cores=5, executor-memory=24G, num-executors=nodes*4 13 | executors=$((nodes*4)) 14 | echo "Number of executors = " $executors 15 | 16 | # location of scala and spark 17 | SW=/oasis/projects/nsf/uot143/$USER 18 | 19 | export HADOOP_CONF_DIR=$HOME/cometcluster 20 | module load hadoop/2.6.0 21 | 22 | DIQL_HOME=$SW/diql 23 | 24 | export JAVA_HOME=/lib/jvm/java 25 | 26 | if [ -z ${SCALA_HOME} ]; then 27 | export SCALA_HOME=$SW/scala-2.11.8 28 | fi 29 | 30 | if [ -z ${SPARK_HOME} ]; then 31 | export SPARK_HOME=$SW/spark-2.2.0-bin-hadoop2.6 32 | fi 33 | 34 | myhadoop-configure.sh 35 | source $HOME/cometcluster/spark/spark-env.sh 36 | export SPARK_MASTER_HOST=$SPARK_MASTER_IP 37 | # start HDFS 38 | start-dfs.sh 39 | # start Spark 40 | $SPARK_HOME/sbin/start-all.sh -h $SPARK_MASTER_HOST 41 | 42 | JARS=. 43 | for I in $SPARK_HOME/jars/*.jar; do 44 | JARS=$JARS:$I 45 | done 46 | 47 | SPARK_OPTIONS="--driver-memory 24G --num-executors $executors --executor-cores 5 --executor-memory 24G --supervise" 48 | 49 | hdfs dfs -mkdir -p /user/$USER /tmp /user/$USER/tmp 50 | 51 | $SPARK_HOME/bin/spark-submit --jars ${DIQL_HOME}/lib/diql-spark.jar --class GroupBy --master $MASTER $SPARK_OPTIONS test.jar 1 10000000000 52 | 53 | $SPARK_HOME/sbin/stop-all.sh 54 | stop-dfs.sh 55 | myhadoop-cleanup.sh 56 | -------------------------------------------------------------------------------- /benchmarks/diablo/compile: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../..; pwd -P)" 4 | 5 | if [ -z ${SCALA_HOME} ]; then 6 | SCALA_HOME=$HOME/system/scala-2.11.6 7 | fi 8 | 9 | if [ -z ${SPARK_HOME} ]; then 10 | SPARK_HOME=$HOME/spark-2.2.0-bin-hadoop2.6 11 | fi 12 | 13 | JARS=. 14 | for I in ${SPARK_HOME}/jars/*.jar; do 15 | JARS=${JARS}:$I 16 | done 17 | 18 | rm -rf classes 19 | mkdir -p classes 20 | 21 | java_files=`ls casper/*.java` 22 | for f in $java_files; do 23 | echo compiling $f ... 24 | javac -d classes -cp ${JARS} $f 25 | done 26 | 27 | scala_files=`ls *.scala` 28 | for f in $scala_files; do 29 | echo compiling $f ... 30 | $SCALA_HOME/bin/scalac -d classes -cp classes:${JARS}:${DIQL_HOME}/lib/diql-spark.jar $f 31 | done 32 | 33 | jar cf test.jar -C classes . 34 | -------------------------------------------------------------------------------- /benchmarks/diablo/conditionalSum.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.log4j._ 5 | import scala.util.Random 6 | 7 | 8 | object ConditionalSum { 9 | 10 | def main ( args: Array[String] ) { 11 | val repeats = args(0).toInt 12 | val length = args(1).toLong 13 | 14 | val conf = new SparkConf().setAppName("ConditionalSum") 15 | val sc = new SparkContext(conf) 16 | conf.set("spark.logConf","false") 17 | conf.set("spark.eventLog.enabled","false") 18 | LogManager.getRootLogger().setLevel(Level.WARN) 19 | 20 | val rand = new Random() 21 | 22 | val V = sc.parallelize(1L to length/100) 23 | .flatMap{ i => (1 to 100).map{ j => rand.nextDouble()*200 } } 24 | .cache() 25 | 26 | println("*** %d %.2f GB".format(length,length.toDouble*8/(1024.0*1024.0*1024.0))) 27 | 28 | def test () { 29 | var t: Long = System.currentTimeMillis() 30 | 31 | try { 32 | println(V.filter( _ < 100).reduce(_+_)) 33 | 34 | println("**** ConditionalSumSpark run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 35 | } catch { case x: Throwable => println(x) } 36 | 37 | t = System.currentTimeMillis() 38 | 39 | try { 40 | v(sc,""" 41 | var sum: Double = 0.0; 42 | 43 | for v in V do 44 | if (v < 100) 45 | sum += v; 46 | 47 | println(sum); 48 | """) 49 | 50 | println("**** ConditionalSumDiablo run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 51 | } catch { case x: Throwable => println(x); } 52 | 53 | t = System.currentTimeMillis() 54 | 55 | try { 56 | import org.apache.spark.api.java.JavaRDD 57 | println(ConditionalSumCasper.sumList(new JavaRDD(V.map(Double.box)))) 58 | 59 | println("**** ConditionalSumCasper run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 60 | } catch { case x: Throwable => println(x) } 61 | } 62 | 63 | for ( i <- 1 to repeats ) 64 | test() 65 | 66 | sc.stop() 67 | } 68 | } 69 | -------------------------------------------------------------------------------- /benchmarks/diablo/groupBy.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.log4j._ 5 | import scala.util.Random 6 | 7 | 8 | case class GB ( K: Long, A: Double ) 9 | 10 | object GroupBy { 11 | 12 | def main ( args: Array[String] ) { 13 | val repeats = args(0).toInt 14 | val length = args(1).toLong 15 | 16 | val conf = new SparkConf().setAppName("GroupBy") 17 | val sc = new SparkContext(conf) 18 | conf.set("spark.logConf","false") 19 | conf.set("spark.eventLog.enabled","false") 20 | LogManager.getRootLogger().setLevel(Level.WARN) 21 | 22 | val rand = new Random() 23 | 24 | val max: Long = length/10 // 10 duplicates on the average 25 | 26 | val GBsize = sizeof(GB(1L,1.0D)) 27 | println("*** %d %.2f GB".format(length,length.toDouble*GBsize/(1024.0*1024.0*1024.0))) 28 | 29 | val V = sc.parallelize(1L to length/100) 30 | .flatMap{ i => (1 to 100).map{ j => GB( Math.abs(rand.nextDouble()*max).toLong, 31 | rand.nextDouble() ) } } 32 | .cache() 33 | 34 | def test () { 35 | var t: Long = System.currentTimeMillis() 36 | 37 | try { 38 | val C = V.map{ case GB(k,v) => (k,v) }.reduceByKey(_+_) 39 | 40 | println(C.count()) 41 | 42 | println("**** GroupBySpark run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 43 | } catch { case x: Throwable => println(x) } 44 | 45 | t = System.currentTimeMillis() 46 | 47 | try { 48 | v(sc,""" 49 | 50 | var C: vector[Double] = vector(); 51 | 52 | for v in V do 53 | C[v.K] += v.A; 54 | 55 | println(C.count); 56 | 57 | """) 58 | 59 | println("**** GroupByDiablo run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 60 | } catch { case x: Throwable => println(x) } 61 | } 62 | 63 | for ( i <- 1 to repeats ) 64 | test() 65 | sc.stop() 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /benchmarks/diablo/parallel/build-parallel: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../../..; pwd -P)" 4 | 5 | rm -rf classes 6 | mkdir -p classes 7 | scalac -d classes -cp ${DIQL_HOME}/lib/diql-parallel.jar parallel.scala 8 | 9 | jar cf diablo-par.jar -C classes . 10 | -------------------------------------------------------------------------------- /benchmarks/diablo/parallel/build-sequential: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../../..; pwd -P)" 4 | 5 | rm -rf classes 6 | mkdir -p classes 7 | scalac -d classes -cp ${DIQL_HOME}/lib/diql-sequential.jar sequential.scala 8 | 9 | jar cf diablo-seq.jar -C classes . 10 | -------------------------------------------------------------------------------- /benchmarks/diablo/parallel/comet-run: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot143 3 | #SBATCH --job-name="diablo" 4 | #SBATCH --output="run.log" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=1 7 | #SBATCH --export=ALL 8 | #SBATCH --time=200 9 | 10 | # location of scala and diql 11 | SW=/oasis/projects/nsf/uot143/fegaras 12 | 13 | export SCALA_HOME=$SW/scala-2.11.8 14 | export DIQL_HOME=$SW/diql 15 | 16 | 17 | ${SCALA_HOME}/bin/scala -cp ${DIQL_HOME}/lib/diql-parallel.jar:diablo-par.jar -J-Xmx96G -J-Xms96G -J-Xss1G Parallel 4 50 18 | 19 | ${SCALA_HOME}/bin/scala -cp ${DIQL_HOME}/lib/diql-sequential.jar:diablo-seq.jar -J-Xmx96G -J-Xms96G -J-Xss1G Sequential 4 50 20 | -------------------------------------------------------------------------------- /benchmarks/diablo/parallel/diablo-par.jar: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/benchmarks/diablo/parallel/diablo-par.jar -------------------------------------------------------------------------------- /benchmarks/diablo/parallel/diablo-seq.jar: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/benchmarks/diablo/parallel/diablo-seq.jar -------------------------------------------------------------------------------- /benchmarks/diablo/parallel/run-parallel: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../../..; pwd -P)" 4 | 5 | scala -cp ${DIQL_HOME}/lib/diql-parallel.jar:diablo-par.jar -J-Xmx16G -J-Xms16G -J-Xss1G Parallel $1 $2 6 | -------------------------------------------------------------------------------- /benchmarks/diablo/parallel/run-sequential: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../../..; pwd -P)" 4 | 5 | scala -cp ${DIQL_HOME}/lib/diql-sequential.jar:diablo-seq.jar -J-Xmx16G -J-Xms16G -J-Xss1G Sequential $1 $2 6 | -------------------------------------------------------------------------------- /benchmarks/diablo/run: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../..; pwd -P)" 4 | 5 | if [ -z ${SPARK_HOME} ]; then 6 | SPARK_HOME=~/spark-2.2.0-bin-hadoop2.6 7 | fi 8 | 9 | export HADOOP_CONF_DIR= 10 | 11 | jar cf test.jar -C classes . 12 | 13 | ${SPARK_HOME}/bin/spark-submit --jars ${DIQL_HOME}/lib/diql-spark.jar --class $1 --master local[2] test.jar $2 $3 $4 $5 $6 14 | -------------------------------------------------------------------------------- /benchmarks/kmeans.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.log4j._ 5 | 6 | object Test { 7 | 8 | case class Point ( X: Double, Y: Double ) 9 | 10 | def distance ( x: Point, y: Point ): Double 11 | = Math.sqrt(Math.pow(x.X-y.X,2)+Math.pow(x.Y-y.Y,2)) 12 | 13 | def main ( args: Array[String] ) { 14 | val iterations = args(0).toInt 15 | val centroid_file = args(1) 16 | val point_file = args(2) 17 | val conf = new SparkConf().setAppName("KMeans") 18 | val sc = new SparkContext(conf) 19 | 20 | conf.set("spark.logConf","false") 21 | conf.set("spark.eventLog.enabled","false") 22 | LogManager.getRootLogger().setLevel(Level.WARN) 23 | 24 | explain(true) 25 | val t: Long = System.currentTimeMillis() 26 | 27 | val points = sc.textFile(point_file) 28 | .map( _.split(",") ) 29 | .map( p => Point(p(0).toDouble,p(1).toDouble) ) 30 | 31 | val initial_centroids = sc.textFile(centroid_file) 32 | .map( _.split(",") ) 33 | .map( p => Point(p(0).toDouble,p(1).toDouble) ) 34 | .collect() 35 | 36 | q(""" repeat centroids = initial_centroids 37 | step select Point( avg/x, avg/y ) 38 | from p@Point(x,y) <- points 39 | group by k: ( select c 40 | from c <- centroids 41 | order by distance(c,p) ).head 42 | limit iterations 43 | """).map(println) 44 | 45 | sc.stop() 46 | 47 | println("**** DIQL run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/FactorizationGenerator.java: -------------------------------------------------------------------------------- 1 | 2 | /* Create matrixes for matrix factorization 3 | */ 4 | 5 | import java.io.*; 6 | import java.util.*; 7 | 8 | public class FactorizationGenerator { 9 | static Random rand = new Random(); 10 | 11 | static void randomMatrix ( int N, int M, String file ) throws IOException { 12 | BufferedWriter m = new BufferedWriter(new FileWriter(file)); 13 | List l = new ArrayList(); 14 | for ( int i = 0; i < N-1; i++ ) 15 | l.add(i); 16 | Collections.shuffle(l); 17 | List r = new ArrayList(); 18 | for ( int i = 0; i < M-1; i++ ) 19 | r.add(i); 20 | Collections.shuffle(r); 21 | for ( Integer i: l ) 22 | for ( Integer j: r ) 23 | if (rand.nextDouble() > 0.1) 24 | m.write(i+","+j+","+((int)(Math.floor(rand.nextDouble()*5+1)))+"\n"); 25 | m.close(); 26 | } 27 | 28 | public static void main ( String[] args ) throws Exception { 29 | int n = Integer.parseInt(args[0]); 30 | randomMatrix(n,n,args[1]); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/MatrixGenerator.java: -------------------------------------------------------------------------------- 1 | /* Create matrixes for multiplication/addition 2 | */ 3 | 4 | import java.io.*; 5 | import java.util.*; 6 | 7 | public class MatrixGenerator { 8 | static Random rand = new Random(); 9 | 10 | static void randomMatrix ( int N, int M, int max, String file ) throws IOException { 11 | BufferedWriter m = new BufferedWriter(new FileWriter(file)); 12 | List l = new ArrayList(); 13 | for ( int i = 0; i < N-1; i++ ) 14 | l.add(i); 15 | Collections.shuffle(l); 16 | List r = new ArrayList(); 17 | for ( int i = 0; i < M-1; i++ ) 18 | r.add(i); 19 | Collections.shuffle(r); 20 | for ( Integer i: l ) 21 | for ( Integer j: r ) 22 | m.write(i+","+j+","+(rand.nextDouble()*max)+"\n"); 23 | m.close(); 24 | } 25 | 26 | public static void main ( String[] args ) throws Exception { 27 | int n = Integer.parseInt(args[0]); 28 | randomMatrix(n,n,10,args[1]); 29 | randomMatrix(n,n,10,args[2]); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/PointGenerator.java: -------------------------------------------------------------------------------- 1 | /* Points in a 100*100 grid used by KMeans 2 | */ 3 | 4 | import java.io.BufferedWriter; 5 | import java.io.FileWriter; 6 | import java.util.Random; 7 | 8 | public class PointGenerator { 9 | static Random rand = new Random(); 10 | 11 | static double getd () { 12 | double v = rand.nextDouble()*20.0D; 13 | return ((int)v % 2 == 0) ? getd() : v; 14 | } 15 | 16 | public static void main ( String[] args ) throws Exception { 17 | long points = Long.parseLong(args[0]); 18 | BufferedWriter centroids = new BufferedWriter(new FileWriter(args[1])); 19 | for ( int i = 0; i < 10; i++ ) 20 | for ( int j = 0; j < 10; j++ ) 21 | centroids.write((i*2+1.2)+","+(j*2+1.2)+"\n"); 22 | centroids.close(); 23 | BufferedWriter data = new BufferedWriter(new FileWriter(args[2])); 24 | for ( long i = 0; i < points; i++ ) { 25 | double x = getd(); 26 | double y = getd(); 27 | data.write(x+","+y+"\n"); 28 | } 29 | data.close(); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/add-spark.scala: -------------------------------------------------------------------------------- 1 | import org.apache.spark._ 2 | import org.apache.spark.rdd._ 3 | import org.apache.log4j._ 4 | 5 | object AddSpark { 6 | 7 | def main ( args: Array[String] ) { 8 | val conf = new SparkConf().setAppName("AddSpark") 9 | val sc = new SparkContext(conf) 10 | 11 | conf.set("spark.logConf","false") 12 | conf.set("spark.eventLog.enabled","false") 13 | LogManager.getRootLogger().setLevel(Level.WARN) 14 | 15 | var M = sc.textFile(args(0)) 16 | .map( line => { val a = line.split(",") 17 | ((a(0).toInt,a(1).toInt),a(2).toDouble) } ) 18 | var N = sc.textFile(args(1)) 19 | .map( line => { val a = line.split(",") 20 | ((a(0).toInt,a(1).toInt),a(2).toDouble) } ) 21 | 22 | val t: Long = System.currentTimeMillis() 23 | 24 | val R = M.cogroup(N) 25 | .map{ case (k,(ms,ns)) 26 | => ( k, if (ms.isEmpty) 27 | ns.head 28 | else if (ns.isEmpty) 29 | ms.head 30 | else ns.head + ms.head ) } 31 | 32 | println(R.count) 33 | 34 | println("**** AddSpark run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 35 | sc.stop() 36 | 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/add.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.log4j._ 5 | 6 | object Add { 7 | 8 | def main ( args: Array[String] ) { 9 | val conf = new SparkConf().setAppName("Add") 10 | val sc = new SparkContext(conf) 11 | 12 | conf.set("spark.logConf","false") 13 | conf.set("spark.eventLog.enabled","false") 14 | LogManager.getRootLogger().setLevel(Level.WARN) 15 | 16 | //explain(true) 17 | 18 | val n = args(2).toLong 19 | val m = n 20 | 21 | var M = sc.textFile(args(0)) 22 | .map( line => { val a = line.split(",") 23 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ) 24 | var N = sc.textFile(args(1)) 25 | .map( line => { val a = line.split(",") 26 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ) 27 | 28 | val t: Long = System.currentTimeMillis() 29 | 30 | v(sc,""" 31 | 32 | var R: matrix[Double] = matrix(); 33 | 34 | for i = 0, 100 do 35 | for j = 0, 100 do { 36 | if (!exists(M[i,j])) 37 | R[i,j] := N[i,j]; 38 | else if (!exists(N[i,j])) 39 | R[i,j] := M[i,j]; 40 | else R[i,j] := M[i,j]+N[i,j]; 41 | }; 42 | 43 | println(R.count()); 44 | 45 | """) 46 | 47 | println("**** Add run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 48 | sc.stop() 49 | 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/build: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | DIQL_HOME=$HOME/diql 4 | export SCALA_HOME=/home/fegaras/system/scala-2.11.6 5 | # directory on local disk to store the dataset 6 | DATA=data 7 | 8 | mkdir -p $HOME/classes 9 | javac -d $HOME/classes $DIQL_HOME/benchmarks/diablo/MatrixGenerator.java 10 | javac -d $HOME/classes $DIQL_HOME/benchmarks/diablo/FactorizationGenerator.java 11 | $SCALA_HOME/bin/scalac -d $HOME/classes $DIQL_HOME/benchmarks/diablo/GraphGenerator.scala 12 | javac -d $HOME/classes $DIQL_HOME/benchmarks/diablo/PointGenerator.java 13 | 14 | mkdir -p $DATA 15 | rm -rf $DATA/* 16 | 17 | for ((i=1; i<=1; i++)); do 18 | java -cp $HOME/classes MatrixGenerator $((100*i)) $DATA/M$i $DATA/N$i 19 | $SCALA_HOME/bin/scala -cp $HOME/classes GraphGenerator $((20000*i)) $((200000*i)) $DATA/G$i 20 | java -cp $HOME/classes FactorizationGenerator $((200*i)) $DATA/F$i 21 | java -cp $HOME/classes PointGenerator $((100000*i)) $DATA/C$i $DATA/P$i 22 | done 23 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/comet.build: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot143 3 | #SBATCH --job-name="build" 4 | #SBATCH --output="build.out" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=1 7 | #SBATCH --export=ALL 8 | #SBATCH --time=300 9 | 10 | # location of spark, scala, and diql 11 | export SW=/oasis/projects/nsf/uot143/fegaras 12 | DIQL_HOME=$SW/diql 13 | export SCALA_HOME=$SW/scala-2.11.8 14 | # directory on local disk to store the dataset 15 | DATA=/oasis/projects/nsf/uot143/$USER/data2 16 | 17 | mkdir -p $HOME/classes 18 | javac -d $HOME/classes $DIQL_HOME/benchmarks/diablo/MatrixGenerator.java 19 | javac -d $HOME/classes $DIQL_HOME/benchmarks/diablo/FactorizationGenerator.java 20 | $SCALA_HOME/bin/scalac -d $HOME/classes $DIQL_HOME/benchmarks/diablo/GraphGenerator.scala 21 | javac -d $HOME/classes $DIQL_HOME/benchmarks/diablo/PointGenerator.java 22 | 23 | mkdir -p $DATA 24 | rm -rf $DATA/* 25 | 26 | for ((i=1; i<=8; i++)); do 27 | j1=$(echo "scale=3;sqrt($i*1.0)*200" | bc) 28 | n1=${j1%.*} 29 | j2=$(echo "scale=3;sqrt($i*1.0)*150" | bc) 30 | n2=${j2%.*} 31 | java -cp $HOME/classes MatrixGenerator $n1 $DATA/M$i $DATA/N$i 32 | $SCALA_HOME/bin/scala -cp $HOME/classes GraphGenerator $((25000*i)) $((250000*i)) $DATA/G$i 33 | java -cp $HOME/classes PointGenerator $((125000*i)) $DATA/C$i $DATA/P$i 34 | java -cp $HOME/classes FactorizationGenerator $n2 $DATA/F$i 35 | done 36 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/kmeans-spark.scala: -------------------------------------------------------------------------------- 1 | import org.apache.spark._ 2 | import org.apache.spark.rdd._ 3 | import Math._ 4 | import org.apache.log4j._ 5 | 6 | object KMeansSpark { 7 | 8 | type Point = (Double,Double) 9 | 10 | var centroids: Array[Point] = Array[Point]() 11 | 12 | def distance ( x: Point, y: Point ): Double 13 | = Math.sqrt((x._1-y._1)*(x._1-y._1)+(x._2-y._2)*(x._2-y._2)) 14 | 15 | def closest ( p: Point ): Point 16 | = centroids.minBy(distance(p,_)) 17 | 18 | def avg ( s: Iterable[Double] ): Double 19 | = s.sum/s.size 20 | 21 | def main ( args: Array[String] ) { 22 | val conf = new SparkConf().setAppName("KMeansSpark") 23 | val sc = new SparkContext(conf) 24 | 25 | conf.set("spark.logConf","false") 26 | conf.set("spark.eventLog.enabled","false") 27 | LogManager.getRootLogger().setLevel(Level.WARN) 28 | 29 | val points = sc.textFile(args(0)).map { 30 | line => val a = line.split(",") 31 | (a(0).toDouble, a(1).toDouble) 32 | } 33 | 34 | centroids = sc.textFile(args(1)).map { 35 | line => val a = line.split(",") 36 | (a(0).toDouble, a(1).toDouble) 37 | }.collect() 38 | 39 | val t: Long = System.currentTimeMillis() 40 | 41 | for ( i <- 1 to 10 ) { 42 | val cs = sc.broadcast(centroids) 43 | centroids = points.map { p => (cs.value.minBy(distance(p,_)), p) } 44 | .groupByKey().map { 45 | case (c, vs) => (avg(vs.map(_._1)), avg(vs.map(_._2))) 46 | }.collect() 47 | } 48 | 49 | println(centroids.length) 50 | 51 | println("**** KMeansSpark run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 52 | sc.stop() 53 | 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/multiply-spark.scala: -------------------------------------------------------------------------------- 1 | import org.apache.spark._ 2 | import org.apache.spark.rdd._ 3 | import org.apache.log4j._ 4 | 5 | object MultiplySpark { 6 | 7 | def main ( args: Array[String] ) { 8 | val conf = new SparkConf().setAppName("MultiplySpark") 9 | val sc = new SparkContext(conf) 10 | 11 | conf.set("spark.logConf","false") 12 | conf.set("spark.eventLog.enabled","false") 13 | LogManager.getRootLogger().setLevel(Level.WARN) 14 | 15 | var M = sc.textFile(args(0)) 16 | .map( line => { val a = line.split(",") 17 | ((a(0).toInt,a(1).toInt),a(2).toDouble) } ) 18 | var N = sc.textFile(args(1)) 19 | .map( line => { val a = line.split(",") 20 | ((a(0).toInt,a(1).toInt),a(2).toDouble) } ) 21 | 22 | val t: Long = System.currentTimeMillis() 23 | 24 | val R = M.map{ case ((i,j),v) => (j,(i,v)) } 25 | .cogroup( N.map{ case ((i,j),v) => (i,(j,v)) } ) 26 | .flatMap{ case (k,(ms,ns)) => ms.flatMap{ case (i,m) => ns.map{ case (j,n) => ((i,j),m*n) } } } 27 | .reduceByKey(_+_) 28 | 29 | println(R.count) 30 | 31 | println("**** MultiplySpark run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 32 | sc.stop() 33 | 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/multiply.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.log4j._ 5 | 6 | object Multiply { 7 | 8 | def main ( args: Array[String] ) { 9 | val conf = new SparkConf().setAppName("Multiply") 10 | val sc = new SparkContext(conf) 11 | 12 | conf.set("spark.logConf","false") 13 | conf.set("spark.eventLog.enabled","false") 14 | LogManager.getRootLogger().setLevel(Level.WARN) 15 | 16 | //explain(true) 17 | 18 | val n = args(2).toLong 19 | val m = n 20 | 21 | var M = sc.textFile(args(0)) 22 | .map( line => { val a = line.split(",") 23 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ) 24 | var N = sc.textFile(args(1)) 25 | .map( line => { val a = line.split(",") 26 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ) 27 | 28 | val t: Long = System.currentTimeMillis() 29 | 30 | v(sc,""" 31 | 32 | var R: matrix[Double] = matrix(); 33 | 34 | for i = 0, n-1 do 35 | for j = 0, n-1 do { 36 | R[i,j] := 0.0; 37 | for k = 0, m-1 do 38 | R[i,j] += M[i,k]*N[k,j]; 39 | }; 40 | 41 | println(R.count); 42 | 43 | """) 44 | 45 | println("**** Multiply run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 46 | sc.stop() 47 | 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/pagerank-spark.scala: -------------------------------------------------------------------------------- 1 | import org.apache.spark._ 2 | import org.apache.spark.rdd._ 3 | import Math._ 4 | import org.apache.log4j._ 5 | 6 | object PagerankSpark { 7 | 8 | def main ( args: Array[String] ) { 9 | val conf = new SparkConf().setAppName("PagerankSpark") 10 | val sc = new SparkContext(conf) 11 | 12 | conf.set("spark.logConf","false") 13 | conf.set("spark.eventLog.enabled","false") 14 | LogManager.getRootLogger().setLevel(Level.WARN) 15 | 16 | val E = sc.textFile(args(0)) 17 | .map( line => { val a = line.split(",").toList 18 | (a(0).toLong,a(1).toLong) } ) 19 | 20 | val N = args(1).toInt 21 | 22 | val t: Long = System.currentTimeMillis() 23 | 24 | val links = E.groupByKey().cache() 25 | var ranks = links.mapValues(v => 1.0/N) 26 | 27 | for (i <- 1 to 10) { 28 | val contribs = links.join(ranks).values.flatMap{ case (urls, rank) => 29 | val size = urls.size 30 | urls.map(url => (url, rank / size)) 31 | } 32 | ranks = contribs.reduceByKey(_ + _).mapValues(0.15/N + 0.85 * _) 33 | } 34 | println(ranks.count) 35 | 36 | println("**** PagerankSpark run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 37 | sc.stop() 38 | 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/pagerank.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import Math._ 5 | import org.apache.log4j._ 6 | 7 | object Pagerank { 8 | 9 | def main ( args: Array[String] ) { 10 | val conf = new SparkConf().setAppName("Pagerank") 11 | val sc = new SparkContext(conf) 12 | 13 | conf.set("spark.logConf","false") 14 | conf.set("spark.eventLog.enabled","false") 15 | LogManager.getRootLogger().setLevel(Level.WARN) 16 | 17 | //explain(true) 18 | 19 | val E = sc.textFile(args(0)) 20 | .map( line => { val a = line.split(",").toList 21 | ((a(0).toLong,a(1).toLong),true) } ) 22 | 23 | val t: Long = System.currentTimeMillis() 24 | 25 | v(sc,""" 26 | 27 | var P: vector[Double] = vector(); 28 | var C: vector[Int] = vector(); 29 | var N: Int = args(1).toInt; 30 | var b: Double = 0.85; 31 | 32 | for i = 1, N do { 33 | C[i] := 0; 34 | P[i] := 1.0/N; 35 | }; 36 | 37 | for i = 1, N do 38 | for j = 1, N do 39 | if (E[i,j]) 40 | C[i] += 1; 41 | 42 | var k: Int = 0; 43 | 44 | while (k < 10) { 45 | var Q: matrix[Double] = matrix(); 46 | k += 1; 47 | for i = 1, N do 48 | for j = 1, N do 49 | if (E[i,j]) 50 | Q[i,j] := P[i]; 51 | for i = 1, N do 52 | P[i] := (1-b)/N; 53 | for i = 1, N do 54 | for j = 1, N do 55 | P[i] += b*Q[j,i]/C[j]; 56 | }; 57 | 58 | println(P.count()); 59 | 60 | """) 61 | 62 | println("**** Pagerank run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 63 | sc.stop() 64 | 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/parallel/build: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../../..; pwd -P)" 4 | 5 | mkdir -p classes 6 | scalac -d classes -cp ${DIQL_HOME}/lib/diql-parallel.jar $* 7 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/parallel/build-seq: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../../..; pwd -P)" 4 | 5 | mkdir -p classes 6 | scalac -d classes -cp ${DIQL_HOME}/lib/diql-sequential.jar $* 7 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/parallel/factorization-seq.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import Math._ 3 | import scala.io.Source 4 | 5 | object Factorization { 6 | 7 | def main ( args: Array[String] ) { 8 | 9 | //explain(true) 10 | 11 | val n = args(1).toLong 12 | val m = args(2).toLong 13 | val l = args(3).toLong 14 | 15 | var R = Source.fromFile(args(0)).getLines.toIterable 16 | .map( line => { val a = line.split(",") 17 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ) 18 | 19 | val t: Long = System.currentTimeMillis() 20 | 21 | v(""" 22 | 23 | var P: matrix[Double] = matrix(); 24 | var Q: matrix[Double] = matrix(); 25 | var pq: matrix[Double] = matrix(); 26 | var E: matrix[Double] = matrix(); 27 | 28 | var a: Double = 0.002; 29 | var b: Double = 0.02; 30 | 31 | for i = 0, n-1 do 32 | for k = 0, l-1 do 33 | P[i,k] := random(); 34 | 35 | for k = 0, l-1 do 36 | for j = 0, m-1 do 37 | Q[k,j] := random(); 38 | 39 | var steps: Int = 0; 40 | while ( steps < 10 ) { 41 | steps += 1; 42 | for i = 0, n-1 do 43 | for j = 0, m-1 do { 44 | pq[i,j] := 0.0; 45 | for k = 0, l-1 do 46 | pq[i,j] += P[i,k]*Q[k,j]; 47 | E[i,j] := R[i,j]-pq[i,j]; 48 | for k = 0, l-1 do { 49 | P[i,k] += a*(2*E[i,j]*Q[k,j]-b*P[i,k]); 50 | Q[k,j] += a*(2*E[i,j]*P[i,k]-b*Q[k,j]); 51 | }; 52 | }; 53 | }; 54 | 55 | println(P.size); 56 | println(Q.size); 57 | 58 | """) 59 | 60 | println("**** Factorization run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 61 | 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/parallel/factorization.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import Math._ 3 | import scala.io.Source 4 | import scala.collection.parallel.ParIterable 5 | 6 | object Factorization { 7 | 8 | def main ( args: Array[String] ) { 9 | 10 | //explain(true) 11 | 12 | val n = args(1).toLong 13 | val m = args(2).toLong 14 | val l = args(3).toLong 15 | 16 | var R = Source.fromFile(args(0)).getLines 17 | .map( line => { val a = line.split(",") 18 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ).toIterable.par 19 | 20 | val t: Long = System.currentTimeMillis() 21 | 22 | v(""" 23 | 24 | var P: matrix[Double] = matrix(); 25 | var Q: matrix[Double] = matrix(); 26 | var pq: matrix[Double] = matrix(); 27 | var E: matrix[Double] = matrix(); 28 | 29 | var a: Double = 0.002; 30 | var b: Double = 0.02; 31 | 32 | for i = 0, n-1 do 33 | for k = 0, l-1 do 34 | P[i,k] := random(); 35 | 36 | for k = 0, l-1 do 37 | for j = 0, m-1 do 38 | Q[k,j] := random(); 39 | 40 | var steps: Int = 0; 41 | while ( steps < 10 ) { 42 | steps += 1; 43 | for i = 0, n-1 do 44 | for j = 0, m-1 do { 45 | pq[i,j] := 0.0; 46 | for k = 0, l-1 do 47 | pq[i,j] += P[i,k]*Q[k,j]; 48 | E[i,j] := R[i,j]-pq[i,j]; 49 | for k = 0, l-1 do { 50 | P[i,k] += a*(2*E[i,j]*Q[k,j]-b*P[i,k]); 51 | Q[k,j] += a*(2*E[i,j]*P[i,k]-b*Q[k,j]); 52 | }; 53 | }; 54 | }; 55 | 56 | println(P.size); 57 | println(Q.size); 58 | 59 | """) 60 | 61 | println("**** Factorization run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 62 | 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/parallel/multiply-seq.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import scala.io.Source 3 | 4 | object Multiply { 5 | 6 | def main ( args: Array[String] ) { 7 | 8 | //explain(true) 9 | 10 | val n = args(2).toLong 11 | val m = n 12 | 13 | val M = Source.fromFile(args(0)).getLines.toIterable 14 | .map( line => { val a = line.split(",") 15 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ) 16 | 17 | val N = Source.fromFile(args(1)).getLines.toIterable 18 | .map( line => { val a = line.split(",") 19 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ) 20 | 21 | val t: Long = System.currentTimeMillis() 22 | 23 | v(""" 24 | 25 | var R: matrix[Double] = matrix(); 26 | 27 | for i = 0, n-1 do 28 | for j = 0, n-1 do { 29 | R[i,j] := 0.0; 30 | for k = 0, m-1 do 31 | R[i,j] += M[i,k]*N[k,j]; 32 | }; 33 | 34 | println(R.size); 35 | 36 | """) 37 | 38 | println("**** Multiply run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 39 | 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/parallel/multiply.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import scala.io.Source 3 | import scala.collection.parallel.ParIterable 4 | 5 | object Multiply { 6 | 7 | def main ( args: Array[String] ) { 8 | 9 | //explain(true) 10 | 11 | val n = args(2).toLong 12 | val m = n 13 | 14 | val M = Source.fromFile(args(0)).getLines 15 | .map( line => { val a = line.split(",") 16 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ).toIterable.par 17 | 18 | val N = Source.fromFile(args(1)).getLines 19 | .map( line => { val a = line.split(",") 20 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ).toIterable.par 21 | 22 | val t: Long = System.currentTimeMillis() 23 | 24 | v(""" 25 | 26 | var R: matrix[Double] = matrix(); 27 | 28 | for i = 0, n-1 do 29 | for j = 0, n-1 do { 30 | R[i,j] := 0.0; 31 | for k = 0, m-1 do 32 | R[i,j] += M[i,k]*N[k,j]; 33 | }; 34 | 35 | println(R.size); 36 | 37 | """) 38 | 39 | println("**** Multiply run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 40 | 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/parallel/pagerank-seq.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import Math._ 3 | import scala.io.Source 4 | 5 | object Pagerank { 6 | 7 | def main ( args: Array[String] ) { 8 | 9 | //explain(true) 10 | 11 | val E = Source.fromFile(args(0)).getLines.toIterable 12 | .map( line => { val a = line.split(",").toList 13 | ((a(0).toLong,a(1).toLong),true) } ) 14 | 15 | val t: Long = System.currentTimeMillis() 16 | 17 | v(""" 18 | 19 | var P: vector[Double] = vector(); 20 | var Q: vector[Double] = vector(); 21 | var C: vector[Int] = vector(); 22 | var N: Int = args(1).toInt; 23 | var b: Double = 0.85; 24 | 25 | for i = 1, N do { 26 | C[i] := 0; 27 | P[i] := 1.0/N; 28 | }; 29 | 30 | for i = 1, N do 31 | for j = 1, N do 32 | if (E[i,j]) 33 | C[i] += 1; 34 | 35 | var k: Int = 0; 36 | while (k < 10) { 37 | k += 1; 38 | for i = 1, N do 39 | Q[i] := P[i]; 40 | for i = 1, N do 41 | P[i] := 0.0; 42 | for i = 1, N do 43 | for j = 1, N do 44 | if (E[j,i]) 45 | P[i] += b*Q[j]/C[j]; 46 | for i = 1, N do 47 | P[i] += (1-b)/N; 48 | }; 49 | 50 | println(P.size); 51 | 52 | """) 53 | 54 | println("**** Pagerank run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 55 | 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /benchmarks/old-diablo/parallel/pagerank.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import Math._ 3 | import scala.io.Source 4 | import scala.collection.parallel.ParIterable 5 | 6 | object Pagerank { 7 | 8 | def main ( args: Array[String] ) { 9 | 10 | //explain(true) 11 | 12 | val E = Source.fromFile(args(0)).getLines 13 | .map( line => { val a = line.split(",").toList 14 | ((a(0).toLong,a(1).toLong),true) } ).toIterable.par 15 | 16 | val t: Long = System.currentTimeMillis() 17 | 18 | v(""" 19 | 20 | var P: vector[Double] = vector(); 21 | var Q: vector[Double] = vector(); 22 | var C: vector[Int] = vector(); 23 | var N: Int = args(1).toInt; 24 | var b: Double = 0.85; 25 | 26 | for i = 1, N do { 27 | C[i] := 0; 28 | P[i] := 1.0/N; 29 | }; 30 | 31 | for i = 1, N do 32 | for j = 1, N do 33 | if (E[i,j]) 34 | C[i] += 1; 35 | 36 | var k: Int = 0; 37 | while (k < 10) { 38 | k += 1; 39 | for i = 1, N do 40 | Q[i] := P[i]; 41 | for i = 1, N do 42 | P[i] := 0.0; 43 | for i = 1, N do 44 | for j = 1, N do 45 | if (E[j,i]) 46 | P[i] += b*Q[j]/C[j]; 47 | for i = 1, N do 48 | P[i] += (1-b)/N; 49 | }; 50 | 51 | println(P.size); 52 | 53 | """) 54 | 55 | println("**** Pagerank run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 56 | 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /benchmarks/pagerank.mrql: -------------------------------------------------------------------------------- 1 | alpha = 0.85; 2 | 3 | edges = source(line,args[1],",",type((int,int))); 4 | 5 | dump "out" from 6 | repeat nodes = select ( s, count(d), 1-alpha ) 7 | from (s,d) in edges 8 | group by s 9 | step select ( ( id, m#1, nr ), true ) 10 | from (id,nr) in ( select ( key, (1-alpha)+alpha*sum(rank)/sum(degree) ) 11 | from (id,degree,rank) in nodes, 12 | e in edges 13 | where e#0 = id 14 | group by key: e#1 ), 15 | m in nodes 16 | where id = m#0 17 | limit toInt(args[0]); 18 | -------------------------------------------------------------------------------- /benchmarks/pagerank.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.log4j._ 5 | 6 | object Test { 7 | 8 | case class Edge ( src: Int, dest: Int ) 9 | 10 | case class Rank ( id: Int, degree: Int, rank: Double ) 11 | 12 | val alpha = 0.85D 13 | 14 | def main ( args: Array[String] ) { 15 | val iterations = args(0).toInt 16 | val input_file = args(1) 17 | val output_file = args(2) 18 | val conf = new SparkConf().setAppName("PageRank") 19 | val sc = new SparkContext(conf) 20 | 21 | conf.set("spark.logConf","false") 22 | conf.set("spark.eventLog.enabled","false") 23 | LogManager.getRootLogger().setLevel(Level.WARN) 24 | 25 | explain(true) 26 | val t: Long = System.currentTimeMillis() 27 | 28 | q(""" 29 | let edges = select Edge(s,d) 30 | from line <- sc.textFile(input_file), 31 | List(s,d) = line.split(",").toList.map(_.toInt) 32 | in repeat nodes = select Rank( id = s, 33 | degree = (count/d).toInt, 34 | rank = 1-alpha ) 35 | from Edge(s,d) <- edges 36 | group by s 37 | step select Rank( id, m.degree, nr ) 38 | from (id,nr) <- ( select ( key, 39 | (1-alpha)+alpha*(+/rank)/(+/degree) ) 40 | from Rank(id,degree,rank) <- nodes, 41 | e <- edges 42 | where e.src == id 43 | group by key: e.dest ), 44 | m <- nodes 45 | where id == m.id 46 | limit iterations 47 | """).saveAsTextFile(output_file) 48 | 49 | sc.stop() 50 | 51 | println("**** DIQL run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /benchmarks/pagerank2.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.log4j._ 5 | 6 | object Test { 7 | 8 | case class Edge ( src: Int, dest: Int ) 9 | 10 | case class Rank ( id: Int, degree: Int, rank: Double ) 11 | 12 | val alpha = 0.85D 13 | 14 | def main ( args: Array[String] ) { 15 | val iterations = 10 16 | val input_file = "g.txt" 17 | val output_file = "" 18 | val conf = new SparkConf().setAppName("PageRank") 19 | val sc = new SparkContext(conf) 20 | 21 | conf.set("spark.logConf","false") 22 | conf.set("spark.eventLog.enabled","false") 23 | LogManager.getRootLogger().setLevel(Level.WARN) 24 | 25 | debug(true) 26 | val t: Long = System.currentTimeMillis() 27 | 28 | q(""" 29 | let edges = select Edge(s,d) 30 | from line <- sc.textFile(input_file), 31 | List(s,d) = line.split(",").toList.map(_.toInt) 32 | in select Rank( id = s, degree = (count/d).toInt, rank = 1-alpha ) 33 | from Edge(s,d) <- edges 34 | group by s 35 | order by (count/d) desc 36 | """).take(10).foreach(println) 37 | 38 | println("DIQL run time: "+(System.currentTimeMillis()-t)/1000.0+" secs") 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /benchmarks/runFlink: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME=~/diql 4 | 5 | if [ -z ${FLINK_HOME} ]; then 6 | FLINK_HOME=~/flink-1.2.1 7 | fi 8 | 9 | export HADOOP_CONF_DIR= 10 | 11 | jar cf test.jar -C classes . 12 | 13 | JARS=. 14 | for I in ${FLINK_HOME}/lib/*.jar; do 15 | JARS=${JARS}:$I 16 | done 17 | 18 | ${FLINK_HOME}/bin/flink run -q -c Test -C file://${DIQL_HOME}/lib/diql-flink.jar test.jar $* 19 | -------------------------------------------------------------------------------- /benchmarks/runScalding: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME=~/diql 4 | 5 | if [ -z ${SCALDING_HOME} ]; then 6 | SCALDING_HOME=${HOME}/scalding 7 | fi 8 | if [ -z ${HADOOP_HOME} ]; then 9 | HADOOP_HOME=${HOME}/hadoop-2.6.0 10 | fi 11 | 12 | jars=${DIQL_HOME}/lib/diql-scalding.jar 13 | for I in ${SCALDING_HOME}/scalding-core/target/scala-2.11/scalding-core-assembly*.jar; do 14 | jars=${jars},$I 15 | done 16 | export HADOOP_CLASSPATH=`echo ${jars} | sed s/,/:/g` 17 | 18 | export HADOOP_CONF_DIR= 19 | 20 | jar cf test.jar -C classes . 21 | mkdir -p results 22 | 23 | $HADOOP_HOME/bin/hadoop jar test.jar Test -libjars ${jars} -mode --local $* 24 | -------------------------------------------------------------------------------- /benchmarks/runSpark: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME=~/diql 4 | 5 | if [ -z ${SPARK_HOME} ]; then 6 | SPARK_HOME=~/spark-2.1.0-bin-hadoop2.6 7 | fi 8 | 9 | jar cf test.jar -C classes . 10 | 11 | export HADOOP_CONF_DIR= 12 | 13 | ${SPARK_HOME}/bin/spark-submit --jars ${DIQL_HOME}/lib/diql-spark.jar --master local[2] --class Test test.jar $* 14 | -------------------------------------------------------------------------------- /benchmarks/runSparkDistr: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME=~/diql 4 | 5 | if [ -z ${SPARK_HOME} ]; then 6 | SPARK_HOME=~/spark-2.1.0-bin-hadoop2.6 7 | fi 8 | 9 | jar cf test.jar -C classes . 10 | 11 | export HADOOP_CONF_DIR=$HADOOP_PREFIX/etc/hadoop 12 | 13 | ${SPARK_HOME}/bin/spark-submit --jars ${DIQL_HOME}/lib/diql-spark.jar --class Test --master yarn --deploy-mode client --num-executors 16 --executor-cores 1 --executor-memory 1600m test.jar $* 14 | -------------------------------------------------------------------------------- /benchmarks/sqlgen/group-by-join.eps: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/benchmarks/sqlgen/group-by-join.eps -------------------------------------------------------------------------------- /benchmarks/sqlgen/group-by-join.run: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot166 3 | #SBATCH --job-name="sqlgen" 4 | #SBATCH --output="group-by-join.log" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=10 7 | #SBATCH --export=ALL 8 | #SBATCH --time=60 # time limit in minutes 9 | 10 | nodes=$SLURM_NNODES 11 | echo "Number of nodes = " $nodes 12 | 13 | # Comet node: 24 cores (23 available), 128 GB RAM 14 | # executor-cores = 4 (5 executors/node) 15 | # executor-memory = 23GB 16 | # num-executors = nodes*5-1 17 | executors=$((nodes*5-1)) 18 | echo "Number of executors = " $executors 19 | 20 | SPARK_OPTIONS="--driver-memory 23G --num-executors $executors --executor-cores 4 --executor-memory 23G --supervise" 21 | 22 | # location of spark, scala, and diql 23 | #SW=/oasis/projects/nsf/uot143/$USER 24 | SW=$HOME 25 | export HADOOP_CONF_DIR=$HOME/cometcluster 26 | module load hadoop/2.6.0 27 | 28 | DIQL_HOME=$SW/SQLGen/Count/DIQL 29 | 30 | export JAVA_HOME=/lib/jvm/java 31 | 32 | if [ -z ${SCALA_HOME} ]; then 33 | export SCALA_HOME=$SW/scala-2.11.8 34 | fi 35 | 36 | if [ -z ${SPARK_HOME} ]; then 37 | export SPARK_HOME=$SW/spark-2.2.0-bin-hadoop2.6 38 | fi 39 | 40 | myhadoop-configure.sh 41 | source $HOME/cometcluster/spark/spark-env.sh 42 | export SPARK_MASTER_HOST=$SPARK_MASTER_IP 43 | # start HDFS 44 | start-dfs.sh 45 | # start Spark 46 | $SPARK_HOME/sbin/start-all.sh -h $SPARK_MASTER_HOST 47 | 48 | JARS=. 49 | for I in $SPARK_HOME/jars/*.jar; do 50 | JARS=$JARS:$I 51 | done 52 | 53 | scale=100 # scale of datasets 54 | ns=5 # number of datasets per experiment 55 | repeat=4 # number of repetitions of each experiment 56 | 57 | for ((i=1; i<=$ns; i++)); do # for each different dataset 58 | n=$((200000*i*scale/ns)) # 2*10^8 59 | $SPARK_HOME/bin/spark-submit --jars ${DIQL_HOME}/lib/diql-spark.jar --class GroupByJoin --master $MASTER $SPARK_OPTIONS test.jar $repeat $n 60 | done 61 | 62 | $SPARK_HOME/sbin/stop-all.sh 63 | stop-dfs.sh 64 | myhadoop-cleanup.sh 65 | -------------------------------------------------------------------------------- /benchmarks/sqlgen/group-by.eps: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/benchmarks/sqlgen/group-by.eps -------------------------------------------------------------------------------- /benchmarks/sqlgen/group-by.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/benchmarks/sqlgen/group-by.png -------------------------------------------------------------------------------- /benchmarks/sqlgen/group-by.run: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot143 3 | #SBATCH --job-name="sqlgen" 4 | #SBATCH --output="group-by.log" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=10 7 | #SBATCH --export=ALL 8 | #SBATCH --time=600 # time limit in minutes 9 | 10 | nodes=$SLURM_NNODES 11 | echo "Number of nodes = " $nodes 12 | 13 | # Comet node: 24 cores (23 available), 128 GB RAM 14 | # executor-cores = 4 (5 executors/node) 15 | # executor-memory = 23GB 16 | # num-executors = nodes*5-1 17 | executors=$((nodes*5-1)) 18 | echo "Number of executors = " $executors 19 | 20 | SPARK_OPTIONS="--driver-memory 23G --num-executors $executors --executor-cores 4 --executor-memory 23G --supervise" 21 | 22 | # location of spark, scala, and diql 23 | #SW=/oasis/projects/nsf/uot143/$USER 24 | SW=$HOME 25 | export HADOOP_CONF_DIR=$HOME/cometcluster 26 | module load hadoop/2.6.0 27 | 28 | DIQL_HOME=$SW/SQLGen/Count/DIQL 29 | 30 | export JAVA_HOME=/lib/jvm/java 31 | 32 | if [ -z ${SCALA_HOME} ]; then 33 | export SCALA_HOME=$SW/scala-2.11.8 34 | fi 35 | 36 | if [ -z ${SPARK_HOME} ]; then 37 | export SPARK_HOME=$SW/spark-2.2.0-bin-hadoop2.6 38 | fi 39 | 40 | myhadoop-configure.sh 41 | source $HOME/cometcluster/spark/spark-env.sh 42 | export SPARK_MASTER_HOST=$SPARK_MASTER_IP 43 | # start HDFS 44 | start-dfs.sh 45 | # start Spark 46 | $SPARK_HOME/sbin/start-all.sh -h $SPARK_MASTER_HOST 47 | 48 | JARS=. 49 | for I in $SPARK_HOME/jars/*.jar; do 50 | JARS=$JARS:$I 51 | done 52 | 53 | scale=100 # scale of datasets 54 | ns=5 # number of datasets per experiment 55 | repeat=4 # number of repetitions of each experiment 56 | 57 | for ((i=1; i<=$ns; i++)); do # for each different dataset 58 | n=$((2000000*i*scale/ns)) # 2*10^8 59 | $SPARK_HOME/bin/spark-submit --jars ${DIQL_HOME}/lib/diql-spark.jar --class GroupBy --master $MASTER $SPARK_OPTIONS test.jar $repeat $n 60 | done 61 | 62 | $SPARK_HOME/sbin/stop-all.sh 63 | stop-dfs.sh 64 | myhadoop-cleanup.sh 65 | -------------------------------------------------------------------------------- /benchmarks/sqlgen/histogram.eps: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/benchmarks/sqlgen/histogram.eps -------------------------------------------------------------------------------- /benchmarks/sqlgen/histogram.run: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot166 3 | #SBATCH --job-name="sqlgen" 4 | #SBATCH --output="histogram.log" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=10 7 | #SBATCH --export=ALL 8 | #SBATCH --time=30 # time limit in minutes 9 | 10 | nodes=$SLURM_NNODES 11 | echo "Number of nodes = " $nodes 12 | 13 | # Comet node: 24 cores (23 available), 128 GB RAM 14 | # executor-cores = 4 (5 executors/node) 15 | # executor-memory = 23GB 16 | # num-executors = nodes*5-1 17 | executors=$((nodes*5-1)) 18 | echo "Number of executors = " $executors 19 | 20 | SPARK_OPTIONS="--driver-memory 23G --num-executors $executors --executor-cores 4 --executor-memory 23G --supervise" 21 | 22 | # location of spark, scala, and diql 23 | #SW=/oasis/projects/nsf/uot143/$USER 24 | SW=$HOME 25 | export HADOOP_CONF_DIR=$HOME/cometcluster 26 | module load hadoop/2.6.0 27 | 28 | DIQL_HOME=$SW/SQLGen/Count/DIQL 29 | 30 | export JAVA_HOME=/lib/jvm/java 31 | 32 | if [ -z ${SCALA_HOME} ]; then 33 | export SCALA_HOME=$SW/scala-2.11.8 34 | fi 35 | 36 | if [ -z ${SPARK_HOME} ]; then 37 | export SPARK_HOME=$SW/spark-2.2.0-bin-hadoop2.6 38 | fi 39 | 40 | myhadoop-configure.sh 41 | source $HOME/cometcluster/spark/spark-env.sh 42 | export SPARK_MASTER_HOST=$SPARK_MASTER_IP 43 | # start HDFS 44 | start-dfs.sh 45 | # start Spark 46 | $SPARK_HOME/sbin/start-all.sh -h $SPARK_MASTER_HOST 47 | 48 | JARS=. 49 | for I in $SPARK_HOME/jars/*.jar; do 50 | JARS=$JARS:$I 51 | done 52 | 53 | scale=100 # scale of datasets 54 | ns=5 # number of datasets per experiment 55 | repeat=4 # number of repetitions of each experiment 56 | 57 | for ((i=1; i<=$ns; i++)); do # for each different dataset 58 | n=$((900000*i*scale/ns)) # 2*10^8 59 | $SPARK_HOME/bin/spark-submit --jars ${DIQL_HOME}/lib/diql-spark.jar --class Histogram --master $MASTER $SPARK_OPTIONS test.jar $repeat $n 60 | done 61 | 62 | $SPARK_HOME/sbin/stop-all.sh 63 | stop-dfs.sh 64 | myhadoop-cleanup.sh 65 | -------------------------------------------------------------------------------- /benchmarks/sqlgen/linear-regression.eps: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/benchmarks/sqlgen/linear-regression.eps -------------------------------------------------------------------------------- /benchmarks/sqlgen/linear-regression.run: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot166 3 | #SBATCH --job-name="sqlgen" 4 | #SBATCH --output="linear-regression.log" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=10 7 | #SBATCH --export=ALL 8 | #SBATCH --time=60 # time limit in minutes 9 | 10 | nodes=$SLURM_NNODES 11 | echo "Number of nodes = " $nodes 12 | 13 | # Comet node: 24 cores (23 available), 128 GB RAM 14 | # executor-cores = 4 (5 executors/node) 15 | # executor-memory = 23GB 16 | # num-executors = nodes*5-1 17 | executors=$((nodes*5-1)) 18 | echo "Number of executors = " $executors 19 | 20 | SPARK_OPTIONS="--driver-memory 23G --num-executors $executors --executor-cores 4 --executor-memory 23G --supervise" 21 | 22 | # location of spark, scala, and diql 23 | #SW=/oasis/projects/nsf/uot143/$USER 24 | SW=$HOME 25 | export HADOOP_CONF_DIR=$HOME/cometcluster 26 | module load hadoop/2.6.0 27 | 28 | DIQL_HOME=$SW/SQLGen/Final/DIQL 29 | 30 | export JAVA_HOME=/lib/jvm/java 31 | 32 | if [ -z ${SCALA_HOME} ]; then 33 | export SCALA_HOME=$SW/scala-2.11.8 34 | fi 35 | 36 | if [ -z ${SPARK_HOME} ]; then 37 | export SPARK_HOME=$SW/spark-2.2.0-bin-hadoop2.6 38 | fi 39 | 40 | myhadoop-configure.sh 41 | source $HOME/cometcluster/spark/spark-env.sh 42 | export SPARK_MASTER_HOST=$SPARK_MASTER_IP 43 | # start HDFS 44 | start-dfs.sh 45 | # start Spark 46 | $SPARK_HOME/sbin/start-all.sh -h $SPARK_MASTER_HOST 47 | 48 | JARS=. 49 | for I in $SPARK_HOME/jars/*.jar; do 50 | JARS=$JARS:$I 51 | done 52 | 53 | scale=100 # scale of datasets 54 | ns=5 # number of datasets per experiment 55 | repeat=4 # number of repetitions of each experiment 56 | 57 | for ((i=1; i<=$ns; i++)); do # for each different dataset 58 | n=$((1200000*i*scale/ns)) # 2*10^8 59 | $SPARK_HOME/bin/spark-submit --jars ${DIQL_HOME}/lib/diql-spark.jar --class LinearRegression --master $MASTER $SPARK_OPTIONS test.jar $repeat $n 60 | done 61 | 62 | $SPARK_HOME/sbin/stop-all.sh 63 | stop-dfs.sh 64 | myhadoop-cleanup.sh 65 | -------------------------------------------------------------------------------- /benchmarks/sqlgen/matrix-addition.eps: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/benchmarks/sqlgen/matrix-addition.eps -------------------------------------------------------------------------------- /benchmarks/sqlgen/matrix-addition.run: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot166 3 | #SBATCH --job-name="sqlgen" 4 | #SBATCH --output="matrix-addition2.log" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=10 7 | #SBATCH --export=ALL 8 | #SBATCH --time=150 # time limit in minutes 9 | 10 | nodes=$SLURM_NNODES 11 | echo "Number of nodes = " $nodes 12 | 13 | # Comet node: 24 cores (23 available), 128 GB RAM 14 | # executor-cores = 4 (5 executors/node) 15 | # executor-memory = 23GB 16 | # num-executors = nodes*5-1 17 | executors=$((nodes*5-1)) 18 | echo "Number of executors = " $executors 19 | 20 | SPARK_OPTIONS="--driver-memory 23G --num-executors $executors --executor-cores 4 --executor-memory 23G --supervise" 21 | 22 | # location of spark, scala, and diql 23 | #SW=/oasis/projects/nsf/uot143/$USER 24 | SW=$HOME 25 | export HADOOP_CONF_DIR=$HOME/cometcluster 26 | module load hadoop/2.6.0 27 | 28 | DIQL_HOME=$SW/SQLGen/Count/DIQL 29 | 30 | export JAVA_HOME=/lib/jvm/java 31 | 32 | if [ -z ${SCALA_HOME} ]; then 33 | export SCALA_HOME=$SW/scala-2.11.8 34 | fi 35 | 36 | if [ -z ${SPARK_HOME} ]; then 37 | export SPARK_HOME=$SW/spark-2.2.0-bin-hadoop2.6 38 | fi 39 | 40 | myhadoop-configure.sh 41 | source $HOME/cometcluster/spark/spark-env.sh 42 | export SPARK_MASTER_HOST=$SPARK_MASTER_IP 43 | # start HDFS 44 | start-dfs.sh 45 | # start Spark 46 | $SPARK_HOME/sbin/start-all.sh -h $SPARK_MASTER_HOST 47 | 48 | JARS=. 49 | for I in $SPARK_HOME/jars/*.jar; do 50 | JARS=$JARS:$I 51 | done 52 | 53 | scale=100 # scale of datasets 54 | ns=5 # number of datasets per experiment 55 | repeat=4 # number of repetitions of each experiment 56 | 57 | for ((i=3; i<=$ns; i++)); do # for each different dataset 58 | j=$(echo "scale=3;sqrt($i/$ns)*70*$scale" | bc); n=${j%.*} # 1000x1000 59 | $SPARK_HOME/bin/spark-submit --jars ${DIQL_HOME}/lib/diql-spark.jar --class MatrixAddition --master $MASTER $SPARK_OPTIONS test.jar $repeat $n $n 60 | done 61 | 62 | $SPARK_HOME/sbin/stop-all.sh 63 | stop-dfs.sh 64 | myhadoop-cleanup.sh 65 | -------------------------------------------------------------------------------- /benchmarks/sqlgen/matrix-factorization.eps: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/benchmarks/sqlgen/matrix-factorization.eps -------------------------------------------------------------------------------- /benchmarks/sqlgen/matrix-factorization.run: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot166 3 | #SBATCH --job-name="sqlgen" 4 | #SBATCH --output="matrix-factorization.log" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=10 7 | #SBATCH --export=ALL 8 | #SBATCH --time=200 # time limit in minutes 9 | 10 | nodes=$SLURM_NNODES 11 | echo "Number of nodes = " $nodes 12 | 13 | # Comet node: 24 cores (23 available), 128 GB RAM 14 | # executor-cores = 4 (5 executors/node) 15 | # executor-memory = 23GB 16 | # num-executors = nodes*5-1 17 | executors=$((nodes*5-1)) 18 | echo "Number of executors = " $executors 19 | 20 | SPARK_OPTIONS="--driver-memory 23G --num-executors $executors --executor-cores 4 --executor-memory 23G --supervise" 21 | 22 | # location of spark, scala, and diql 23 | #SW=/oasis/projects/nsf/uot143/$USER 24 | SW=$HOME 25 | export HADOOP_CONF_DIR=$HOME/cometcluster 26 | module load hadoop/2.6.0 27 | 28 | DIQL_HOME=$SW/SQLGen/Final/DIQL 29 | 30 | export JAVA_HOME=/lib/jvm/java 31 | 32 | if [ -z ${SCALA_HOME} ]; then 33 | export SCALA_HOME=$SW/scala-2.11.8 34 | fi 35 | 36 | if [ -z ${SPARK_HOME} ]; then 37 | export SPARK_HOME=$SW/spark-2.2.0-bin-hadoop2.6 38 | fi 39 | 40 | myhadoop-configure.sh 41 | source $HOME/cometcluster/spark/spark-env.sh 42 | export SPARK_MASTER_HOST=$SPARK_MASTER_IP 43 | # start HDFS 44 | start-dfs.sh 45 | # start Spark 46 | $SPARK_HOME/sbin/start-all.sh -h $SPARK_MASTER_HOST 47 | 48 | JARS=. 49 | for I in $SPARK_HOME/jars/*.jar; do 50 | JARS=$JARS:$I 51 | done 52 | 53 | scale=100 # scale of datasets 54 | ns=5 # number of datasets per experiment 55 | repeat=4 # number of repetitions of each experiment 56 | 57 | for ((i=1; i<=$ns; i++)); do # for each different dataset 58 | j=$(echo "scale=3;sqrt($i/$ns)*60*$scale" | bc); n=${j%.*} # 8000x8000 59 | $SPARK_HOME/bin/spark-submit --jars ${DIQL_HOME}/lib/diql-spark.jar --class MatrixFactorization --master $MASTER $SPARK_OPTIONS test.jar $repeat $n $n 60 | done 61 | 62 | $SPARK_HOME/sbin/stop-all.sh 63 | stop-dfs.sh 64 | myhadoop-cleanup.sh 65 | -------------------------------------------------------------------------------- /benchmarks/sqlgen/matrix-multiplication.eps: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/benchmarks/sqlgen/matrix-multiplication.eps -------------------------------------------------------------------------------- /benchmarks/sqlgen/matrix-multiplication.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/benchmarks/sqlgen/matrix-multiplication.png -------------------------------------------------------------------------------- /benchmarks/sqlgen/matrix-multiplication.run: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot143 3 | #SBATCH --job-name="sqlgen" 4 | #SBATCH --output="matrix-multiplication.log" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=10 7 | #SBATCH --export=ALL 8 | #SBATCH --time=300 # time limit in minutes 9 | 10 | nodes=$SLURM_NNODES 11 | echo "Number of nodes = " $nodes 12 | 13 | # Comet node: 24 cores (23 available), 128 GB RAM 14 | # executor-cores = 4 (5 executors/node) 15 | # executor-memory = 23GB 16 | # num-executors = nodes*5-1 17 | executors=$((nodes*5-1)) 18 | echo "Number of executors = " $executors 19 | 20 | SPARK_OPTIONS="--driver-memory 23G --num-executors $executors --executor-cores 4 --executor-memory 23G --supervise" 21 | 22 | # location of spark, scala, and diql 23 | #SW=/oasis/projects/nsf/uot143/$USER 24 | SW=$HOME 25 | export HADOOP_CONF_DIR=$HOME/cometcluster 26 | module load hadoop/2.6.0 27 | 28 | DIQL_HOME=$SW/SQLGen/Count/DIQL 29 | 30 | export JAVA_HOME=/lib/jvm/java 31 | 32 | if [ -z ${SCALA_HOME} ]; then 33 | export SCALA_HOME=$SW/scala-2.11.8 34 | fi 35 | 36 | if [ -z ${SPARK_HOME} ]; then 37 | export SPARK_HOME=$SW/spark-2.2.0-bin-hadoop2.6 38 | fi 39 | 40 | myhadoop-configure.sh 41 | source $HOME/cometcluster/spark/spark-env.sh 42 | export SPARK_MASTER_HOST=$SPARK_MASTER_IP 43 | # start HDFS 44 | start-dfs.sh 45 | # start Spark 46 | $SPARK_HOME/sbin/start-all.sh -h $SPARK_MASTER_HOST 47 | 48 | JARS=. 49 | for I in $SPARK_HOME/jars/*.jar; do 50 | JARS=$JARS:$I 51 | done 52 | 53 | scale=100 # scale of datasets 54 | ns=5 # number of datasets per experiment 55 | repeat=4 # number of repetitions of each experiment 56 | 57 | for ((i=1; i<=$ns; i++)); do # for each different dataset 58 | j=$(echo "scale=3;sqrt($i/$ns)*10*$scale" | bc); n=${j%.*} # 1000x1000 59 | $SPARK_HOME/bin/spark-submit --jars ${DIQL_HOME}/lib/diql-spark.jar --class MatrixMultiplication --master $MASTER $SPARK_OPTIONS test.jar $repeat $n $n 60 | done 61 | 62 | $SPARK_HOME/sbin/stop-all.sh 63 | stop-dfs.sh 64 | myhadoop-cleanup.sh 65 | -------------------------------------------------------------------------------- /benchmarks/sqlgen/page-rank.eps: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/benchmarks/sqlgen/page-rank.eps -------------------------------------------------------------------------------- /benchmarks/sqlgen/page-rank.run: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot166 3 | #SBATCH --job-name="sqlgen" 4 | #SBATCH --output="page-rank.log" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=10 7 | #SBATCH --export=ALL 8 | #SBATCH --time=15 # time limit in minutes 9 | 10 | nodes=$SLURM_NNODES 11 | echo "Number of nodes = " $nodes 12 | 13 | # Comet node: 24 cores (23 available), 128 GB RAM 14 | # executor-cores = 4 (5 executors/node) 15 | # executor-memory = 23GB 16 | # num-executors = nodes*5-1 17 | executors=$((nodes*5-1)) 18 | echo "Number of executors = " $executors 19 | 20 | SPARK_OPTIONS="--driver-memory 23G --num-executors $executors --executor-cores 4 --executor-memory 23G --supervise" 21 | 22 | # location of spark, scala, and diql 23 | #SW=/oasis/projects/nsf/uot143/$USER 24 | SW=$HOME 25 | export HADOOP_CONF_DIR=$HOME/cometcluster 26 | module load hadoop/2.6.0 27 | 28 | DIQL_HOME=$SW/SQLGen/Final/DIQL 29 | 30 | export JAVA_HOME=/lib/jvm/java 31 | 32 | if [ -z ${SCALA_HOME} ]; then 33 | export SCALA_HOME=$SW/scala-2.11.8 34 | fi 35 | 36 | if [ -z ${SPARK_HOME} ]; then 37 | export SPARK_HOME=$SW/spark-2.2.0-bin-hadoop2.6 38 | fi 39 | 40 | myhadoop-configure.sh 41 | source $HOME/cometcluster/spark/spark-env.sh 42 | export SPARK_MASTER_HOST=$SPARK_MASTER_IP 43 | # start HDFS 44 | start-dfs.sh 45 | # start Spark 46 | $SPARK_HOME/sbin/start-all.sh -h $SPARK_MASTER_HOST 47 | 48 | JARS=. 49 | for I in $SPARK_HOME/jars/*.jar; do 50 | JARS=$JARS:$I 51 | done 52 | 53 | scale=100 # scale of datasets 54 | ns=5 # number of datasets per experiment 55 | repeat=4 # number of repetitions of each experiment 56 | 57 | #for ((i=1; i<=$ns; i++)); do # for each different dataset 58 | n=$((20000*1*scale/ns)) # 2*10^8 59 | $SPARK_HOME/bin/spark-submit --jars ${DIQL_HOME}/lib/diql-spark.jar --class PageRank --master $MASTER $SPARK_OPTIONS test.jar $repeat $n $((10*n)) 60 | #done 61 | 62 | $SPARK_HOME/sbin/stop-all.sh 63 | stop-dfs.sh 64 | myhadoop-cleanup.sh 65 | -------------------------------------------------------------------------------- /benchmarks/sqlgen/page-rank2.eps: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/benchmarks/sqlgen/page-rank2.eps -------------------------------------------------------------------------------- /benchmarks/sqlgen/pca.eps: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/benchmarks/sqlgen/pca.eps -------------------------------------------------------------------------------- /benchmarks/sqlgen/pca.run: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot166 3 | #SBATCH --job-name="sqlgen" 4 | #SBATCH --output="pca.log" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=10 7 | #SBATCH --export=ALL 8 | #SBATCH --time=100 # time limit in minutes 9 | 10 | nodes=$SLURM_NNODES 11 | echo "Number of nodes = " $nodes 12 | 13 | # Comet node: 24 cores (23 available), 128 GB RAM 14 | # executor-cores = 4 (5 executors/node) 15 | # executor-memory = 23GB 16 | # num-executors = nodes*5-1 17 | executors=$((nodes*5-1)) 18 | echo "Number of executors = " $executors 19 | 20 | SPARK_OPTIONS="--driver-memory 23G --num-executors $executors --executor-cores 4 --executor-memory 23G --supervise" 21 | 22 | # location of spark, scala, and diql 23 | #SW=/oasis/projects/nsf/uot143/$USER 24 | SW=$HOME 25 | export HADOOP_CONF_DIR=$HOME/cometcluster 26 | module load hadoop/2.6.0 27 | 28 | DIQL_HOME=$SW/SQLGen/Count/DIQL 29 | 30 | export JAVA_HOME=/lib/jvm/java 31 | 32 | if [ -z ${SCALA_HOME} ]; then 33 | export SCALA_HOME=$SW/scala-2.11.8 34 | fi 35 | 36 | if [ -z ${SPARK_HOME} ]; then 37 | export SPARK_HOME=$SW/spark-2.2.0-bin-hadoop2.6 38 | fi 39 | 40 | myhadoop-configure.sh 41 | source $HOME/cometcluster/spark/spark-env.sh 42 | export SPARK_MASTER_HOST=$SPARK_MASTER_IP 43 | # start HDFS 44 | start-dfs.sh 45 | # start Spark 46 | $SPARK_HOME/sbin/start-all.sh -h $SPARK_MASTER_HOST 47 | 48 | JARS=. 49 | for I in $SPARK_HOME/jars/*.jar; do 50 | JARS=$JARS:$I 51 | done 52 | 53 | scale=100 # scale of datasets 54 | ns=5 # number of datasets per experiment 55 | repeat=4 # number of repetitions of each experiment 56 | 57 | for ((i=1; i<=$ns; i++)); do # for each different dataset 58 | j=$(echo "scale=3;sqrt($i/$ns)*100*$scale" | bc); n=${j%.*}; d=$((10*i)) # 1000x1000 59 | $SPARK_HOME/bin/spark-submit --jars ${DIQL_HOME}/lib/diql-spark.jar --class PCA --master $MASTER $SPARK_OPTIONS test.jar $repeat $n $d 60 | done 61 | 62 | $SPARK_HOME/sbin/stop-all.sh 63 | stop-dfs.sh 64 | myhadoop-cleanup.sh 65 | -------------------------------------------------------------------------------- /benchmarks/sqlgen/string-match.eps: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/benchmarks/sqlgen/string-match.eps -------------------------------------------------------------------------------- /benchmarks/sqlgen/string-match.run: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot166 3 | #SBATCH --job-name="sqlgen" 4 | #SBATCH --output="string-match.log" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=10 7 | #SBATCH --export=ALL 8 | #SBATCH --time=60 # time limit in minutes 9 | 10 | nodes=$SLURM_NNODES 11 | echo "Number of nodes = " $nodes 12 | 13 | # Comet node: 24 cores (23 available), 128 GB RAM 14 | # executor-cores = 4 (5 executors/node) 15 | # executor-memory = 23GB 16 | # num-executors = nodes*5-1 17 | executors=$((nodes*5-1)) 18 | echo "Number of executors = " $executors 19 | 20 | SPARK_OPTIONS="--driver-memory 23G --num-executors $executors --executor-cores 4 --executor-memory 23G --supervise" 21 | 22 | # location of spark, scala, and diql 23 | #SW=/oasis/projects/nsf/uot143/$USER 24 | SW=$HOME 25 | export HADOOP_CONF_DIR=$HOME/cometcluster 26 | module load hadoop/2.6.0 27 | 28 | DIQL_HOME=$SW/SQLGen/Count/DIQL 29 | 30 | export JAVA_HOME=/lib/jvm/java 31 | 32 | if [ -z ${SCALA_HOME} ]; then 33 | export SCALA_HOME=$SW/scala-2.11.8 34 | fi 35 | 36 | if [ -z ${SPARK_HOME} ]; then 37 | export SPARK_HOME=$SW/spark-2.2.0-bin-hadoop2.6 38 | fi 39 | 40 | myhadoop-configure.sh 41 | source $HOME/cometcluster/spark/spark-env.sh 42 | export SPARK_MASTER_HOST=$SPARK_MASTER_IP 43 | # start HDFS 44 | start-dfs.sh 45 | # start Spark 46 | $SPARK_HOME/sbin/start-all.sh -h $SPARK_MASTER_HOST 47 | 48 | JARS=. 49 | for I in $SPARK_HOME/jars/*.jar; do 50 | JARS=$JARS:$I 51 | done 52 | 53 | scale=100 # scale of datasets 54 | ns=5 # number of datasets per experiment 55 | repeat=4 # number of repetitions of each experiment 56 | 57 | for ((i=1; i<=$ns; i++)); do # for each different dataset 58 | n=$((1500000*$i*scale/ns)) # 2*10^8 59 | $SPARK_HOME/bin/spark-submit --jars ${DIQL_HOME}/lib/diql-spark.jar --class StringMatch --master $MASTER $SPARK_OPTIONS test.jar $repeat $n 60 | done 61 | 62 | $SPARK_HOME/sbin/stop-all.sh 63 | stop-dfs.sh 64 | myhadoop-cleanup.sh 65 | -------------------------------------------------------------------------------- /benchmarks/sqlgen/sum.eps: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/benchmarks/sqlgen/sum.eps -------------------------------------------------------------------------------- /benchmarks/sqlgen/sum.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/benchmarks/sqlgen/sum.png -------------------------------------------------------------------------------- /benchmarks/sqlgen/sum.run: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot166 3 | #SBATCH --job-name="sqlgen" 4 | #SBATCH --output="sum.log" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=10 7 | #SBATCH --export=ALL 8 | #SBATCH --time=30 # time limit in minutes 9 | 10 | nodes=$SLURM_NNODES 11 | echo "Number of nodes = " $nodes 12 | 13 | # Comet node: 24 cores (23 available), 128 GB RAM 14 | # executor-cores = 4 (5 executors/node) 15 | # executor-memory = 23GB 16 | # num-executors = nodes*5-1 17 | executors=$((nodes*5-1)) 18 | echo "Number of executors = " $executors 19 | 20 | SPARK_OPTIONS="--driver-memory 23G --num-executors $executors --executor-cores 4 --executor-memory 23G --supervise" 21 | 22 | # location of spark, scala, and diql 23 | #SW=/oasis/projects/nsf/uot143/$USER 24 | SW=$HOME 25 | export HADOOP_CONF_DIR=$HOME/cometcluster 26 | module load hadoop/2.6.0 27 | 28 | DIQL_HOME=$SW/SQLGen/Count/DIQL 29 | 30 | export JAVA_HOME=/lib/jvm/java 31 | 32 | if [ -z ${SCALA_HOME} ]; then 33 | export SCALA_HOME=$SW/scala-2.11.8 34 | fi 35 | 36 | if [ -z ${SPARK_HOME} ]; then 37 | export SPARK_HOME=$SW/spark-2.2.0-bin-hadoop2.6 38 | fi 39 | 40 | myhadoop-configure.sh 41 | source $HOME/cometcluster/spark/spark-env.sh 42 | export SPARK_MASTER_HOST=$SPARK_MASTER_IP 43 | # start HDFS 44 | start-dfs.sh 45 | # start Spark 46 | $SPARK_HOME/sbin/start-all.sh -h $SPARK_MASTER_HOST 47 | 48 | JARS=. 49 | for I in $SPARK_HOME/jars/*.jar; do 50 | JARS=$JARS:$I 51 | done 52 | 53 | scale=100 # scale of datasets 54 | ns=5 # number of datasets per experiment 55 | repeat=4 # number of repetitions of each experiment 56 | 57 | for ((i=1; i<=$ns; i++)); do # for each different dataset 58 | n=$((2000000*i*scale/ns)) 59 | $SPARK_HOME/bin/spark-submit --jars ${DIQL_HOME}/lib/diql-spark.jar --class Sum --master $MASTER $SPARK_OPTIONS test.jar $repeat $n 60 | done 61 | 62 | $SPARK_HOME/sbin/stop-all.sh 63 | stop-dfs.sh 64 | myhadoop-cleanup.sh 65 | -------------------------------------------------------------------------------- /benchmarks/sqlgen/word-count.eps: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/benchmarks/sqlgen/word-count.eps -------------------------------------------------------------------------------- /benchmarks/sqlgen/word-count.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/benchmarks/sqlgen/word-count.png -------------------------------------------------------------------------------- /benchmarks/sqlgen/word-count.run: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | #SBATCH -A uot143 3 | #SBATCH --job-name="sqlgen" 4 | #SBATCH --output="word-count.log" 5 | #SBATCH --partition=compute 6 | #SBATCH --nodes=10 7 | #SBATCH --export=ALL 8 | #SBATCH --time=30 # time limit in minutes 9 | 10 | nodes=$SLURM_NNODES 11 | echo "Number of nodes = " $nodes 12 | 13 | # Comet node: 24 cores (23 available), 128 GB RAM 14 | # executor-cores = 4 (5 executors/node) 15 | # executor-memory = 23GB 16 | # num-executors = nodes*5-1 17 | executors=$((nodes*5-1)) 18 | echo "Number of executors = " $executors 19 | 20 | SPARK_OPTIONS="--driver-memory 23G --num-executors $executors --executor-cores 4 --executor-memory 23G --supervise" 21 | 22 | # location of spark, scala, and diql 23 | #SW=/oasis/projects/nsf/uot143/$USER 24 | SW=$HOME 25 | export HADOOP_CONF_DIR=$HOME/cometcluster 26 | module load hadoop/2.6.0 27 | 28 | DIQL_HOME=$SW/SQLGen/Count/DIQL 29 | 30 | export JAVA_HOME=/lib/jvm/java 31 | 32 | if [ -z ${SCALA_HOME} ]; then 33 | export SCALA_HOME=$SW/scala-2.11.8 34 | fi 35 | 36 | if [ -z ${SPARK_HOME} ]; then 37 | export SPARK_HOME=$SW/spark-2.2.0-bin-hadoop2.6 38 | fi 39 | 40 | myhadoop-configure.sh 41 | source $HOME/cometcluster/spark/spark-env.sh 42 | export SPARK_MASTER_HOST=$SPARK_MASTER_IP 43 | # start HDFS 44 | start-dfs.sh 45 | # start Spark 46 | $SPARK_HOME/sbin/start-all.sh -h $SPARK_MASTER_HOST 47 | 48 | JARS=. 49 | for I in $SPARK_HOME/jars/*.jar; do 50 | JARS=$JARS:$I 51 | done 52 | 53 | scale=100 # scale of datasets 54 | ns=5 # number of datasets per experiment 55 | repeat=4 # number of repetitions of each experiment 56 | 57 | for ((i=1; i<=$ns; i++)); do # for each different dataset 58 | n=$((800000*i*scale/ns)) # 2*10^8 59 | $SPARK_HOME/bin/spark-submit --jars ${DIQL_HOME}/lib/diql-spark.jar --class WordCount --master $MASTER $SPARK_OPTIONS test.jar $repeat $n 60 | done 61 | 62 | $SPARK_HOME/sbin/stop-all.sh 63 | stop-dfs.sh 64 | myhadoop-cleanup.sh 65 | -------------------------------------------------------------------------------- /lib/diql-spark.jar: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/lib/diql-spark.jar -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | pom-spark.xml -------------------------------------------------------------------------------- /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version=1.3.3 2 | -------------------------------------------------------------------------------- /src/main/scala/edu/uta/diql/InMemoryBag.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright © 2017 University of Texas at Arlington 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package edu.uta.diql.core 17 | 18 | object inMemory { 19 | 20 | def flatMap[A,B] ( f: A => Traversable[B], S: Traversable[A] ): Traversable[B] 21 | = S.flatMap(f) 22 | 23 | def groupBy[K,A] ( S: Traversable[(K,A)] ): Traversable[(K,Traversable[A])] 24 | = S.groupBy{ case (k,_) => k }.mapValues( _.map{ case (_,a) => a }) 25 | 26 | def orderBy[K,A] ( S: Traversable[(K,A)] ) ( implicit cmp: Ordering[K] ): Traversable[A] 27 | = S.toSeq.sortWith{ case ((k1,_),(k2,_)) => cmp.lt(k1,k2) }.map(_._2) 28 | 29 | def reduce[A] ( acc: (A,A) => A, S: Traversable[A] ): A 30 | = S.reduce(acc) 31 | 32 | def coGroup[K,A,B] ( X: Traversable[(K,A)], Y: Traversable[(K,B)] ): Traversable[(K,(Traversable[A],Traversable[B]))] 33 | = { val xi = X.map{ case (k,x) => (k,Left(x)) } 34 | val yi = Y.map{ case (k,y) => (k,Right(y)) } 35 | val g = groupBy(xi++yi) 36 | g.map{ case (k,xy) 37 | => (k,(xy.foldLeft(Nil:List[A]){ case (r,Left(x)) => x::r; case (r,_) => r }, 38 | xy.foldLeft(Nil:List[B]){ case (r,Right(y)) => y::r; case (r,_) => r })) } 39 | } 40 | 41 | def cross[A,B] ( X: Traversable[A], Y: Traversable[B] ): Traversable[(A,B)] 42 | = X.flatMap(x => Y.map(y => (x,y))) 43 | 44 | def merge[A] ( X: Traversable[A], Y: Traversable[A] ): Traversable[A] 45 | = X++Y 46 | } 47 | -------------------------------------------------------------------------------- /tests/diablo/flink/build: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../../..; pwd -P)" 4 | 5 | if [ -z ${FLINK_HOME} ]; then 6 | FLINK_HOME=~/flink-1.6.0 7 | fi 8 | 9 | JARS=. 10 | for I in ${FLINK_HOME}/lib/*.jar; do 11 | JARS=${JARS}:$I 12 | done 13 | 14 | mkdir -p classes 15 | env JAVA_OPTS="-Xmx2G" scalac -d classes -cp ${JARS}:${DIQL_HOME}/lib/diql-flink.jar $* 16 | -------------------------------------------------------------------------------- /tests/diablo/flink/multiply.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.flink.api.scala._ 3 | 4 | object Test { 5 | 6 | def main ( args: Array[String] ) { 7 | val env = ExecutionEnvironment.getExecutionEnvironment 8 | 9 | explain(true) 10 | 11 | val n = args(2).toLong 12 | val m = n 13 | 14 | var M = env.readTextFile(args(0)) 15 | .map( line => { val a = line.split(",") 16 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ) 17 | var N = env.readTextFile(args(1)) 18 | .map( line => { val a = line.split(",") 19 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ) 20 | 21 | v(env,""" 22 | 23 | var R: matrix[Double] = matrix(); 24 | 25 | for i = 0, n-1 do 26 | for j = 0, n-1 do { 27 | R[i,j] := 0.0; 28 | for k = 0, m-1 do 29 | R[i,j] += M[i,k]*N[k,j]; 30 | }; 31 | 32 | R.print; 33 | 34 | """) 35 | 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /tests/diablo/flink/run: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../../..; pwd -P)" 4 | 5 | if [ -z ${FLINK_HOME} ]; then 6 | FLINK_HOME=~/flink-1.6.0 7 | fi 8 | 9 | export HADOOP_CONF_DIR= 10 | 11 | jar cf test.jar -C classes . 12 | 13 | JARS=. 14 | for I in ${FLINK_HOME}/lib/*.jar; do 15 | JARS=${JARS}:$I 16 | done 17 | 18 | ${FLINK_HOME}/bin/flink run -q -c Test -C file://${DIQL_HOME}/lib/diql-flink.jar test.jar $* 19 | -------------------------------------------------------------------------------- /tests/diablo/parallel/add.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import scala.io.Source 3 | import scala.collection.parallel.ParIterable 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ) { 8 | 9 | explain(true) 10 | 11 | val n = args(2).toLong 12 | val m = n 13 | 14 | var M = Source.fromFile(args(0)).getLines 15 | .map( line => { val a = line.split(",") 16 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ).toIterable.par 17 | var N = Source.fromFile(args(1)).getLines 18 | .map( line => { val a = line.split(",") 19 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ).toIterable.par 20 | 21 | 22 | v(""" 23 | 24 | var R: matrix[Double] = matrix(); 25 | 26 | for i = 0, n-1 do 27 | for j = 0, m-1 do 28 | R[i,j] := M[i,j]+N[i,j]; 29 | 30 | R.take(30).foreach(println); 31 | 32 | """) 33 | 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /tests/diablo/parallel/avg.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import scala.io.Source 3 | import scala.collection.parallel.ParIterable 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ) { 8 | 9 | explain(true) 10 | 11 | val A = Source.fromFile(args(0)).getLines 12 | .zipWithIndex.map{ case (line,i) => (i.toLong,line.toDouble) } 13 | .toIterable.par 14 | 15 | v(""" 16 | 17 | var sum: Double = 0.0; 18 | var count: Int = 0; 19 | 20 | for i = 1, 100 do { 21 | sum += A[i]; 22 | count += 1; 23 | }; 24 | println(sum/count); 25 | 26 | """) 27 | 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /tests/diablo/parallel/build: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../../..; pwd -P)" 4 | 5 | mkdir -p classes 6 | scalac -d classes -cp ${DIQL_HOME}/lib/diql-parallel.jar $* 7 | -------------------------------------------------------------------------------- /tests/diablo/parallel/build-seq: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../../..; pwd -P)" 4 | 5 | mkdir -p classes 6 | scalac -d classes -cp ${DIQL_HOME}/lib/diql-sequential.jar $* 7 | -------------------------------------------------------------------------------- /tests/diablo/parallel/c: -------------------------------------------------------------------------------- 1 | 1.2,1.2 2 | 1.2,3.2 3 | 1.2,5.2 4 | 1.2,7.2 5 | 1.2,9.2 6 | 1.2,11.2 7 | 1.2,13.2 8 | 1.2,15.2 9 | 1.2,17.2 10 | 1.2,19.2 11 | 3.2,1.2 12 | 3.2,3.2 13 | 3.2,5.2 14 | 3.2,7.2 15 | 3.2,9.2 16 | 3.2,11.2 17 | 3.2,13.2 18 | 3.2,15.2 19 | 3.2,17.2 20 | 3.2,19.2 21 | 5.2,1.2 22 | 5.2,3.2 23 | 5.2,5.2 24 | 5.2,7.2 25 | 5.2,9.2 26 | 5.2,11.2 27 | 5.2,13.2 28 | 5.2,15.2 29 | 5.2,17.2 30 | 5.2,19.2 31 | 7.2,1.2 32 | 7.2,3.2 33 | 7.2,5.2 34 | 7.2,7.2 35 | 7.2,9.2 36 | 7.2,11.2 37 | 7.2,13.2 38 | 7.2,15.2 39 | 7.2,17.2 40 | 7.2,19.2 41 | 9.2,1.2 42 | 9.2,3.2 43 | 9.2,5.2 44 | 9.2,7.2 45 | 9.2,9.2 46 | 9.2,11.2 47 | 9.2,13.2 48 | 9.2,15.2 49 | 9.2,17.2 50 | 9.2,19.2 51 | 11.2,1.2 52 | 11.2,3.2 53 | 11.2,5.2 54 | 11.2,7.2 55 | 11.2,9.2 56 | 11.2,11.2 57 | 11.2,13.2 58 | 11.2,15.2 59 | 11.2,17.2 60 | 11.2,19.2 61 | 13.2,1.2 62 | 13.2,3.2 63 | 13.2,5.2 64 | 13.2,7.2 65 | 13.2,9.2 66 | 13.2,11.2 67 | 13.2,13.2 68 | 13.2,15.2 69 | 13.2,17.2 70 | 13.2,19.2 71 | 15.2,1.2 72 | 15.2,3.2 73 | 15.2,5.2 74 | 15.2,7.2 75 | 15.2,9.2 76 | 15.2,11.2 77 | 15.2,13.2 78 | 15.2,15.2 79 | 15.2,17.2 80 | 15.2,19.2 81 | 17.2,1.2 82 | 17.2,3.2 83 | 17.2,5.2 84 | 17.2,7.2 85 | 17.2,9.2 86 | 17.2,11.2 87 | 17.2,13.2 88 | 17.2,15.2 89 | 17.2,17.2 90 | 17.2,19.2 91 | 19.2,1.2 92 | 19.2,3.2 93 | 19.2,5.2 94 | 19.2,7.2 95 | 19.2,9.2 96 | 19.2,11.2 97 | 19.2,13.2 98 | 19.2,15.2 99 | 19.2,17.2 100 | 19.2,19.2 101 | -------------------------------------------------------------------------------- /tests/diablo/parallel/f.txt: -------------------------------------------------------------------------------- 1 | 0,0,5 2 | 0,1,3 3 | 0,3,1 4 | 1,0,4 5 | 1,3,1 6 | 2,0,1 7 | 2,1,1 8 | 2,3,5 9 | 3,0,1 10 | 3,3,4 11 | 4,1,1 12 | 4,2,5 13 | 4,3,4 14 | -------------------------------------------------------------------------------- /tests/diablo/parallel/factorization.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import scala.io.Source 3 | import scala.collection.parallel.ParIterable 4 | import Math._ 5 | 6 | object Test { 7 | 8 | def main ( args: Array[String] ) { 9 | 10 | explain(true) 11 | 12 | val n = args(1).toLong 13 | val m = args(2).toLong 14 | val l = args(3).toLong 15 | 16 | var R = Source.fromFile(args(0)).getLines 17 | .map( line => { val a = line.split(",") 18 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ).toIterable.par 19 | 20 | v(""" 21 | 22 | var P: matrix[Double] = matrix(); 23 | var Q: matrix[Double] = matrix(); 24 | var pq: matrix[Double] = matrix(); 25 | var E: matrix[Double] = matrix(); 26 | 27 | var a: Double = 0.002; 28 | var b: Double = 0.02; 29 | 30 | for i = 0, n-1 do 31 | for k = 0, l-1 do 32 | P[i,k] := random(); 33 | 34 | for k = 0, l-1 do 35 | for j = 0, m-1 do 36 | Q[k,j] := random(); 37 | 38 | var steps: Int = 0; 39 | while ( steps < 10 ) { 40 | steps += 1; 41 | for i = 0, n-1 do 42 | for j = 0, m-1 do { 43 | pq[i,j] := 0.0; 44 | for k = 0, l-1 do 45 | pq[i,j] += P[i,k]*Q[k,j]; 46 | E[i,j] := R[i,j]-pq[i,j]; 47 | for k = 0, l-1 do { 48 | P[i,k] += a*(2*E[i,j]*Q[k,j]-b*P[i,k]); 49 | Q[k,j] += a*(2*E[i,j]*P[i,k]-b*Q[k,j]); 50 | }; 51 | }; 52 | }; 53 | 54 | P.take(30).foreach(println); 55 | Q.take(30).foreach(println); 56 | """) 57 | 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /tests/diablo/parallel/multiply.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import scala.io.Source 3 | import scala.collection.parallel.ParIterable 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ) { 8 | 9 | explain(true) 10 | 11 | val n = args(2).toLong 12 | val m = n 13 | 14 | var M = Source.fromFile(args(0)).getLines 15 | .map( line => { val a = line.split(",") 16 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ).toIterable.par 17 | var N = Source.fromFile(args(1)).getLines 18 | .map( line => { val a = line.split(",") 19 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ).toIterable.par 20 | 21 | v(""" 22 | 23 | var R: matrix[Double] = matrix(); 24 | 25 | for i = 0, n-1 do 26 | for j = 0, n-1 do { 27 | R[i,j] := 0.0; 28 | for k = 0, m-1 do 29 | R[i,j] += M[i,k]*N[k,j]; 30 | }; 31 | 32 | R.toList.sortBy(_._1).take(30).foreach(println); 33 | 34 | """) 35 | 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /tests/diablo/parallel/pagerank.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import scala.io.Source 3 | import scala.collection.parallel.ParIterable 4 | import Math._ 5 | 6 | object Test { 7 | 8 | def main ( args: Array[String] ) { 9 | 10 | explain(true) 11 | 12 | val E = Source.fromFile(args(0)).getLines 13 | .map( line => { val a = line.split(",").toList 14 | ((a(0).toLong,a(1).toLong),true) } ).toIterable.par 15 | 16 | v(""" 17 | 18 | var P: vector[Double] = vector(); 19 | var Q: vector[Double] = vector(); 20 | var C: vector[Int] = vector(); 21 | var N: Int = args(1).toInt; 22 | var b: Double = 0.85; 23 | 24 | for i = 1, N do { 25 | C[i] := 0; 26 | P[i] := 1.0/N; 27 | }; 28 | 29 | for i = 1, N do 30 | for j = 1, N do 31 | if (E[i,j]) 32 | C[i] += 1; 33 | 34 | var k: Int = 0; 35 | 36 | while (k < 10) { 37 | k += 1; 38 | for i = 1, N do 39 | Q[i] := P[i]; 40 | for i = 1, N do 41 | P[i] := (1-b)/N; 42 | for i = 1, N do 43 | for j = 1, N do 44 | if (E[j,i]) 45 | P[i] += b*Q[j]/C[j]; 46 | }; 47 | 48 | P.toList.sortBy(_._2).reverse.take(30).foreach(println); 49 | 50 | """) 51 | 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /tests/diablo/parallel/pagerank2.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import scala.io.Source 3 | import scala.collection.parallel.ParIterable 4 | import Math._ 5 | 6 | object Test { 7 | 8 | def main ( args: Array[String] ) { 9 | 10 | 11 | explain(true) 12 | 13 | val E = Source.fromFile(args(0)).getLines 14 | .map( line => { val a = line.split(",").toList 15 | ((a(0).toLong,a(1).toLong),true) } ).toIterable.par 16 | 17 | v(""" 18 | 19 | var P: vector[Double] = vector(); 20 | var C: vector[Int] = vector(); 21 | var N: Int = args(1).toInt; 22 | var b: Double = 0.85; 23 | 24 | for i = 1, N do { 25 | C[i] := 0; 26 | P[i] := 1.0/N; 27 | }; 28 | 29 | for i = 1, N do 30 | for j = 1, N do 31 | if (E[i,j]) 32 | C[i] += 1; 33 | 34 | var k: Int = 0; 35 | 36 | while (k < 10) { 37 | var Q: matrix[Double] = matrix(); 38 | k += 1; 39 | for i = 1, N do 40 | for j = 1, N do 41 | if (E[i,j]) 42 | Q[i,j] := P[i]; 43 | for i = 1, N do 44 | P[i] := (1-b)/N; 45 | for i = 1, N do 46 | for j = 1, N do 47 | P[i] += b*Q[j,i]/C[j]; 48 | }; 49 | 50 | P.toList.sortBy(_._2).reverse.take(30).foreach(println); 51 | 52 | """) 53 | 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /tests/diablo/parallel/run: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../../..; pwd -P)" 4 | 5 | jar cf test.jar -C classes . 6 | 7 | scala -cp ${DIQL_HOME}/lib/diql-parallel.jar:test.jar Test $* 8 | -------------------------------------------------------------------------------- /tests/diablo/parallel/run-seq: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../../..; pwd -P)" 4 | 5 | jar cf test.jar -C classes . 6 | 7 | scala -cp ${DIQL_HOME}/lib/diql-sequential.jar:test.jar -J-Xss1G Test $* 8 | -------------------------------------------------------------------------------- /tests/diablo/parallel/t.txt: -------------------------------------------------------------------------------- 1 | The focus of this course is on data management techniques and tools for storing and analyzing very large amounts of data. Topics that will be covered include: cloud computing; virtualization; distributed file systems; large data processing using Map-Reduce; data modeling, storage, indexing, and query processing for big data; key-value storage systems, columnar databases, NoSQL systems (Cassandra, BigTable, HBase, MongoDB); big data technologies and tools (Hive, Pig, Spark, Flink); large-scale stream processing systems (Storm, Spark Streaming); data analytics frameworks (Mahout); big data applications, including graph processing, recommendation systems, machine learning, clustering, classification, prediction, and stream mining. 2 | -------------------------------------------------------------------------------- /tests/diablo/parallel/wordCount.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import scala.io.Source 3 | import scala.collection.parallel.ParIterable 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ) { 8 | 9 | explain(true) 10 | 11 | val words = Source.fromFile(args(0)).getLines 12 | .flatMap( line => line.split(" ") ).toIterable.par 13 | 14 | v(""" 15 | 16 | var C: map[String,Int] = map(); 17 | 18 | for w in words do 19 | C[w] := 0; 20 | 21 | for w in words do 22 | C[w] += 1; 23 | 24 | C.foreach(println); 25 | 26 | """) 27 | 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /tests/diablo/spark/Average.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ) { 8 | val conf = new SparkConf().setAppName("Test") 9 | val sc = new SparkContext(conf) 10 | 11 | explain(true) 12 | 13 | 14 | val A = sc.textFile(args(0)).zipWithIndex.map{ case (line,i) => (i,line.toDouble) } 15 | 16 | v(sc,""" 17 | 18 | var sum: Double = 0.0; 19 | var count: Int = 0; 20 | 21 | for i = 1, 100 do { 22 | sum += A[i]; 23 | count += 1; 24 | }; 25 | 26 | println(sum/count); 27 | 28 | """) 29 | 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /tests/diablo/spark/Average2.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ) { 8 | val conf = new SparkConf().setAppName("Test") 9 | val sc = new SparkContext(conf) 10 | 11 | explain(true) 12 | 13 | val A = sc.textFile(args(0)).map( line => line.toDouble ) 14 | 15 | v(sc,""" 16 | 17 | var sum: Double = 0.0; 18 | var count: Int = 0; 19 | 20 | for a in A do { 21 | sum += a; 22 | count += 1; 23 | }; 24 | 25 | println(sum/count); 26 | 27 | """) 28 | 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /tests/diablo/spark/ConditionalCount.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | 5 | 6 | object Test { 7 | 8 | def main ( args: Array[String] ) { 9 | val conf = new SparkConf().setAppName("Test") 10 | val sc = new SparkContext(conf) 11 | 12 | explain(true) 13 | 14 | var V = sc.textFile(args(0)).map(_.split(",")).map(_(0).toDouble) 15 | 16 | v(sc,""" 17 | var count: Int = 0; 18 | 19 | for v in V do 20 | if (v < 100) 21 | count += 1; 22 | 23 | println(count); 24 | """) 25 | 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /tests/diablo/spark/ConditionalSum.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ) { 8 | val conf = new SparkConf().setAppName("Test") 9 | val sc = new SparkContext(conf) 10 | 11 | explain(true) 12 | 13 | var V = sc.textFile(args(0)).map(_.split(",")).map(_(0).toDouble) 14 | 15 | v(sc,""" 16 | var sum: Double = 0.0; 17 | 18 | for v in V do 19 | if (v < 100) 20 | sum += v; 21 | 22 | println(sum); 23 | """) 24 | 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /tests/diablo/spark/Count.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ) { 8 | val conf = new SparkConf().setAppName("Test") 9 | val sc = new SparkContext(conf) 10 | 11 | explain(true) 12 | 13 | var V = sc.textFile(args(0)) 14 | 15 | v(sc,""" 16 | var count: Int = 0; 17 | 18 | for v in V do 19 | count += 1; 20 | 21 | println(count); 22 | """) 23 | 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /tests/diablo/spark/Equal.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ) { 8 | val conf = new SparkConf().setAppName("Test") 9 | val sc = new SparkContext(conf) 10 | 11 | explain(true) 12 | 13 | var V = sc.textFile(args(0)).flatMap(_.split(",")) 14 | 15 | var x = V.first() 16 | 17 | v(sc,""" 18 | var eq: Boolean = true; 19 | 20 | for v in V do 21 | eq := eq && v == x; 22 | 23 | println(eq); 24 | 25 | """) 26 | 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /tests/diablo/spark/EqualFrequency.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ) { 8 | val conf = new SparkConf().setAppName("Test") 9 | val sc = new SparkContext(conf) 10 | 11 | explain(true) 12 | 13 | val V = sc.textFile(args(0)).map( line => line.toDouble ) 14 | 15 | v(sc,""" 16 | var count1: Int = 0; 17 | var count2: Int = 0; 18 | 19 | for v in V do { 20 | if (v==100) 21 | count1 += 1; 22 | if (v==110) 23 | count2 += 1; 24 | }; 25 | 26 | """) 27 | sc.stop() 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /tests/diablo/spark/ImageHistogram.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | 5 | case class Color ( red: Int, green: Int, blue: Int ) 6 | 7 | object Test { 8 | 9 | def main ( args: Array[String] ) { 10 | val conf = new SparkConf().setAppName("Test") 11 | val sc = new SparkContext(conf) 12 | 13 | explain(true) 14 | 15 | var P = sc.textFile(args(0)) 16 | .map( line => { val a = line.split(",") 17 | Color(a(0).toInt,a(1).toInt,a(2).toInt) } ) 18 | 19 | v(sc,""" 20 | 21 | var R: map[Int,Int] = map(); 22 | var G: map[Int,Int] = map(); 23 | var B: map[Int,Int] = map(); 24 | 25 | for p in P do { 26 | R[p.red] += 1; 27 | G[p.green] += 1; 28 | B[p.blue] += 1; 29 | }; 30 | 31 | println("R values"); 32 | R.foreach(println); 33 | println("G values"); 34 | G.foreach(println); 35 | println("B values"); 36 | B.foreach(println); 37 | 38 | """) 39 | 40 | } 41 | } 42 | 43 | -------------------------------------------------------------------------------- /tests/diablo/spark/ImageHistogramSpark.scala: -------------------------------------------------------------------------------- 1 | import org.apache.spark.{SparkConf, SparkContext} 2 | 3 | case class Color ( red: Int, green: Int, blue: Int ) 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ) { 8 | 9 | val conf = new SparkConf().setAppName("Test") 10 | val sc = new SparkContext(conf) 11 | 12 | var P = sc.textFile(args(0)) 13 | .map( line => { val a = line.split(",") 14 | Color(a(0).toInt,a(1).toInt,a(2).toInt) } ) 15 | 16 | val R = P.map(_.red).countByValue() 17 | val G = P.map(_.green).countByValue() 18 | val B = P.map(_.blue).countByValue() 19 | 20 | println(R) 21 | println(G) 22 | println(B) 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /tests/diablo/spark/KMeansSpark.scala: -------------------------------------------------------------------------------- 1 | import org.apache.spark.SparkContext 2 | import org.apache.spark.SparkConf 3 | 4 | object KmeansSpark { 5 | 6 | type Point = (Double, Double) 7 | var centroids: Array[Point] = Array[Point]() 8 | 9 | //Receives each point and clusters, returns cluster(closest to that point and the point itself) 10 | def assignment(x: Point, centers: Array[Point]) = { 11 | var minDistance: Double = 0 12 | var distance: Double = 0 13 | var count = 0 14 | var point: Point = (0.0, 0.0) 15 | for (c: Point <- centers) { 16 | distance = math.sqrt(math.pow((c._1 - x._1), 2) + math.pow((c._2 - x._2), 2)) 17 | if (count == 0) { 18 | minDistance = distance 19 | point = c 20 | count = count + 1 21 | } 22 | if (distance < minDistance) { 23 | minDistance = distance 24 | point = c 25 | } 26 | } 27 | point 28 | } 29 | 30 | //takes points assigned to one cluster and get the new cluster coordinates 31 | // x,y coordinates of cluster is the mean x,y of all points 32 | def update(points: Iterable[Point]) = { 33 | var x: Double = 0.0 34 | var y: Double = 0.0 35 | val total = points.size 36 | for (p: Point <- points) { 37 | x += p._1 38 | y += p._2 39 | } 40 | (x / total, y / total) 41 | } 42 | 43 | def main(args: Array[String]) { 44 | val conf = new SparkConf().setAppName("Kmeans Clustering").setMaster("local[2]") 45 | val sc = new SparkContext(conf) 46 | 47 | centroids = sc.textFile("c").map(line => { 48 | val a = line.split(",") 49 | (a(0).toDouble, a(1).toDouble)}) 50 | .collect() 51 | 52 | var points = sc.textFile("p").map(line => { 53 | val a = line.split(",") 54 | (a(0).toDouble, a(1).toDouble) 55 | }) 56 | 57 | for( a <- 1 to 10) { 58 | points.map(point => (assignment(point, centroids), point)) 59 | .groupByKey() 60 | .map(points => update(points._2)) 61 | .foreach(println) 62 | } 63 | 64 | sc.stop() 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /tests/diablo/spark/LinearRegression.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ) { 8 | 9 | val conf = new SparkConf().setAppName("Test") 10 | val sc = new SparkContext(conf) 11 | 12 | explain(true) 13 | 14 | val P = sc.textFile(args(0)) 15 | .map( line => { val a = line.split(",") 16 | (a(0).toDouble, a(1).toDouble) } ) 17 | 18 | var N = P.count() 19 | 20 | v(sc,""" 21 | 22 | var sum_x: Double = 0.0; 23 | var sum_y: Double = 0.0; 24 | var x_bar: Double = 0.0; 25 | var y_bar: Double = 0.0; 26 | var xx_bar: Double = 0.0; 27 | var yy_bar: Double = 0.0; 28 | var xy_bar: Double = 0.0; 29 | var slope: Double = 0.0; 30 | var intercept: Double = 0.0; 31 | 32 | for p in P do { 33 | sum_x += p._1; 34 | sum_y += p._2; 35 | }; 36 | 37 | x_bar := sum_x/N; 38 | y_bar := sum_y/N; 39 | 40 | for p in P do { 41 | xx_bar += (p._1 - x_bar)*(p._1 - x_bar); 42 | yy_bar += (p._2 - y_bar)*(p._2 - y_bar); 43 | xy_bar += (p._1 - x_bar)*(p._2 - y_bar); 44 | }; 45 | 46 | slope := xy_bar/xx_bar; 47 | intercept := y_bar - slope*x_bar; 48 | 49 | println(slope); 50 | println(intercept); 51 | 52 | """) 53 | 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /tests/diablo/spark/LinearRegressionSpark.scala: -------------------------------------------------------------------------------- 1 | import org.apache.spark.SparkContext 2 | import org.apache.spark.SparkConf 3 | 4 | object Test { 5 | 6 | def main ( args: Array[String] ) { 7 | val conf = new SparkConf().setAppName("LinearRegression") 8 | val sc = new SparkContext(conf) 9 | 10 | val p = sc.textFile(args(0)) 11 | .map( line => { val a = line.split(",") 12 | (a(0).toDouble, a(1).toDouble) } ) 13 | 14 | val n = p.count() 15 | val x_bar = p.map(_._1).reduce(_+_)/n 16 | val y_bar = p.map(_._2).reduce(_+_)/n 17 | 18 | val xx_bar = p.map(x => (x._1 - x_bar)*(x._1 - x_bar)).reduce(_+_) 19 | val yy_bar = p.map(y => (y._2 - y_bar)*(y._2 - y_bar)).reduce(_+_) 20 | val xy_bar = p.map(p => (p._1 - x_bar)*(p._2 - y_bar)).reduce(_+_) 21 | 22 | val slope = xy_bar/xx_bar 23 | val intercept = y_bar - slope * x_bar 24 | 25 | println(slope) 26 | println(intercept) 27 | 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /tests/diablo/spark/MatrixAddition.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ) { 8 | val conf = new SparkConf().setAppName("Test") 9 | val sc = new SparkContext(conf) 10 | 11 | explain(true) 12 | 13 | val n = args(2).toLong 14 | val m = n 15 | 16 | var M = sc.textFile(args(0)) 17 | .map( line => { val a = line.split(",") 18 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ) 19 | var N = sc.textFile(args(1)) 20 | .map( line => { val a = line.split(",") 21 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ) 22 | 23 | v(sc,""" 24 | 25 | var R: matrix[Double] = matrix(); 26 | 27 | for i = 0, n do 28 | for j = 0, n do 29 | R[i,j] := M[i,j] + N[i,j]; 30 | 31 | println(R.count()); 32 | 33 | """) 34 | 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /tests/diablo/spark/MatrixFactorization.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import Math._ 5 | 6 | object Test { 7 | 8 | def main ( args: Array[String] ) { 9 | val conf = new SparkConf().setAppName("Test") 10 | val sc = new SparkContext(conf) 11 | 12 | explain(true) 13 | 14 | val n = args(1).toLong 15 | val m = args(2).toLong 16 | val l = args(3).toLong 17 | 18 | var R = sc.textFile(args(0)) 19 | .map( line => { val a = line.split(",") 20 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ) 21 | 22 | v(sc,""" 23 | 24 | var P: matrix[Double] = matrix(); 25 | var Q: matrix[Double] = matrix(); 26 | var pq: matrix[Double] = matrix(); 27 | var E: matrix[Double] = matrix(); 28 | 29 | var a: Double = 0.002; 30 | var b: Double = 0.02; 31 | 32 | for i = 0, n-1 do 33 | for k = 0, l-1 do 34 | P[i,k] := random(); 35 | 36 | for k = 0, l-1 do 37 | for j = 0, m-1 do 38 | Q[k,j] := random(); 39 | 40 | var steps: Int = 0; 41 | while ( steps < 10 ) { 42 | steps += 1; 43 | for i = 0, n-1 do 44 | for j = 0, m-1 do { 45 | pq[i,j] := 0.0; 46 | for k = 0, l-1 do 47 | pq[i,j] += P[i,k]*Q[k,j]; 48 | E[i,j] := R[i,j]-pq[i,j]; 49 | for k = 0, l-1 do { 50 | P[i,k] += a*(2*E[i,j]*Q[k,j]-b*P[i,k]); 51 | Q[k,j] += a*(2*E[i,j]*P[i,k]-b*Q[k,j]); 52 | }; 53 | }; 54 | }; 55 | 56 | P.take(30).foreach(println); 57 | Q.take(30).foreach(println); 58 | """) 59 | 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /tests/diablo/spark/MatrixFactorization2.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import Math._ 5 | 6 | object Test { 7 | 8 | val a: Double = 0.002 9 | val b: Double = 0.02 10 | 11 | implicit class Mult ( private val value: Double ) extends AnyVal { 12 | def ^ ( that: Double ): Double 13 | = value+(1-a*b)*that 14 | } 15 | 16 | def main ( args: Array[String] ) { 17 | val conf = new SparkConf().setAppName("Test") 18 | val sc = new SparkContext(conf) 19 | 20 | explain(true) 21 | 22 | val n = args(1).toLong 23 | val m = args(2).toLong 24 | val l = args(3).toLong 25 | 26 | var R = sc.textFile(args(0)) 27 | .map( line => { val a = line.split(",") 28 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ) 29 | 30 | v(sc,""" 31 | 32 | var P: matrix[Double] = matrix(); 33 | var Q: matrix[Double] = matrix(); 34 | var pq: matrix[Double] = matrix(); 35 | var E: matrix[Double] = matrix(); 36 | 37 | for i = 0, n-1 do 38 | for k = 0, l-1 do 39 | P[i,k] := random(); 40 | 41 | for k = 0, l-1 do 42 | for j = 0, m-1 do 43 | Q[k,j] := random(); 44 | 45 | var steps: Int = 0; 46 | while ( steps < 10 ) { 47 | steps += 1; 48 | for i = 0, n-1 do 49 | for j = 0, m-1 do { 50 | pq[i,j] := 0.0; 51 | for k = 0, l-1 do 52 | pq[i,j] += P[i,k]*Q[k,j]; 53 | E[i,j] := R[i,j]-pq[i,j]; 54 | for k = 0, l-1 do { 55 | P[i,k] := P[i,k] ^ 2*a*E[i,j]*Q[k,j]; 56 | Q[k,j] := Q[k,j] ^ 2*a*E[i,j]*P[i,k]; 57 | }; 58 | }; 59 | }; 60 | 61 | P.take(30).foreach(println); 62 | Q.take(30).foreach(println); 63 | """) 64 | 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /tests/diablo/spark/MatrixMultiplication.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ) { 8 | val conf = new SparkConf().setAppName("Test") 9 | val sc = new SparkContext(conf) 10 | 11 | explain(true) 12 | 13 | val n = args(2).toLong 14 | val m = n 15 | 16 | var M = sc.textFile(args(0)) 17 | .map( line => { val a = line.split(",") 18 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ) 19 | var N = sc.textFile(args(1)) 20 | .map( line => { val a = line.split(",") 21 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ) 22 | 23 | v(sc,""" 24 | 25 | var R: matrix[Double] = matrix(); 26 | 27 | for i = 0, n-1 do 28 | for j = 0, n-1 do { 29 | R[i,j] := 0.0; 30 | for k = 0, m-1 do 31 | R[i,j] += M[i,k]*N[k,j]; 32 | }; 33 | 34 | R.sortBy(_._1,true,1).take(30).foreach(println); 35 | 36 | """) 37 | 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /tests/diablo/spark/MatrixMultiplicationSpark.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ) { 8 | val conf = new SparkConf().setAppName("Test") 9 | val sc = new SparkContext(conf) 10 | 11 | explain(true) 12 | 13 | var M = sc.textFile(args(0)) 14 | .map( line => { val a = line.split(",") 15 | ((a(0).toInt,a(1).toInt),a(2).toDouble) } ) 16 | var N = sc.textFile(args(1)) 17 | .map( line => { val a = line.split(",") 18 | ((a(0).toInt,a(1).toInt),a(2).toDouble) } ) 19 | 20 | M.map{ case ((i,j),m) => (j,(i,m)) } 21 | .join( N.map{ case ((i,j),n) => (i,(j,n)) } ) 22 | .map{ case (k,((i,m),(j,n))) => ((i,j),m*n) } 23 | .reduceByKey(_+_) 24 | .sortBy(_._1,true,1).take(30).foreach(println) 25 | 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /tests/diablo/spark/MatrixMultiplicationSumma.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ) { 8 | val conf = new SparkConf().setAppName("Test") 9 | val sc = new SparkContext(conf) 10 | 11 | explain(true) 12 | 13 | var M = sc.textFile(args(0)) 14 | .map( line => { val a = line.split(",") 15 | ((a(0).toInt,a(1).toInt),a(2).toDouble) } ) 16 | var N = sc.textFile(args(1)) 17 | .map( line => { val a = line.split(",") 18 | ((a(0).toInt,a(1).toInt),a(2).toDouble) } ) 19 | 20 | val R = core.GroupByJoin.groupByJoin[((Int,Int),Double),((Int,Int),Double),Int,Int,Int,Double]( 21 | { case ((i,j),m) => i }, 22 | { case ((i,j),n) => j }, 23 | { case ((_,m),(_,n)) => m*n }, 24 | _+_, 25 | M.map{ case x@((i,j),m) => (j,x)}, 26 | N.map{ case x@((i,j),b) => (i,x)} 27 | ) 28 | R.sortBy(_._1,true,1).take(30).foreach(println) 29 | 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /tests/diablo/spark/PCA.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ) { 8 | val conf = new SparkConf().setAppName("Test") 9 | val sc = new SparkContext(conf) 10 | 11 | explain(true) 12 | 13 | 14 | var P = sc.textFile(args(0)) 15 | .map( line => { val a = line.split(",") 16 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ) 17 | var n = P.count() 18 | 19 | v(sc,""" 20 | 21 | var mean: vector[Double] = vector(); 22 | var d: Int = 3; 23 | var r: Double = n/d; 24 | var ri: Int = r.toInt; 25 | var sum: matrix[Double] = matrix(); 26 | 27 | for i = 0, d-1 do { 28 | for j = 0, ri-1 do 29 | mean[i] += P[j,i]; 30 | mean[i] := 0.0+mean[i]/r; 31 | }; 32 | 33 | for i = 0, d-1 do { 34 | for j = 0, d-1 do { 35 | sum[i,j] := 0.0; 36 | for k = 0, ri-1 do 37 | sum[i,j] += (P[k,i]-mean[i])*(P[k,j]-mean[j]); 38 | sum[i,j] := 0.0 + sum[i,j]/(r-1); 39 | }; 40 | }; 41 | 42 | sum.foreach(println); 43 | 44 | """) 45 | 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /tests/diablo/spark/PCASpark.scala: -------------------------------------------------------------------------------- 1 | import org.apache.spark.{SparkConf, SparkContext} 2 | 3 | object PCASpark { 4 | def main(args: Array[String]) { 5 | 6 | val conf = new SparkConf().setAppName("PCA") 7 | val sc = new SparkContext(conf) 8 | 9 | var P = sc.textFile(args(0)) 10 | .map( line => { val a = line.split(",") 11 | (a(0).toLong,a(1).toLong,a(2).toDouble) } ) 12 | 13 | val d = 3 14 | val n = P.count() 15 | val r = n/d 16 | 17 | val mean = P.map(x => (x._2, x._3)).reduceByKey(_+_).map(x => (x._1, x._2/r)).sortByKey().collect() 18 | val x_bar = mean(0)._2 19 | val y_bar = mean(1)._2 20 | val z_bar = mean(2)._2 21 | 22 | def v ( i: Double, value: Double ) 23 | = if (i == 0) 24 | value - x_bar 25 | else if (i == 1) 26 | value - y_bar 27 | else value - z_bar 28 | 29 | val cov = P.map{ case (i,j,v) => (i,(j,v)) } 30 | .cogroup( P.map{ case (i,j,v) => (i,(j,v)) } ) 31 | .flatMap{ case (k,(ms,ns)) 32 | => ms.flatMap { 33 | case (i,m) 34 | => ns.map { 35 | case (j,n) 36 | => ((i,j), v(i,m)*v(j,n)) 37 | } 38 | } 39 | } 40 | .reduceByKey(_+_) 41 | .map{ case ((i,j),v) => (i,j,v/(r-1)) } 42 | 43 | cov.foreach(println) 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /tests/diablo/spark/PageRankSpark.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import Math._ 5 | 6 | object Test { 7 | 8 | def main ( args: Array[String] ) { 9 | val conf = new SparkConf().setAppName("Test") 10 | val sc = new SparkContext(conf) 11 | 12 | explain(true) 13 | 14 | val E = sc.textFile(args(0)) 15 | .map( line => { val a = line.split(",").toList 16 | (a(0).toLong,a(1).toLong) } ) 17 | 18 | val N = args(1).toInt 19 | 20 | val links = E.groupByKey().cache() 21 | 22 | println(links.count) 23 | 24 | var ranks = links.mapValues(v => 1.0/N) 25 | 26 | for (i <- 1 to 10) { 27 | val contribs = links.join(ranks).values.flatMap{ case (urls, rank) => 28 | val size = urls.size 29 | urls.map(url => (url, rank / size)) 30 | } 31 | ranks = contribs.reduceByKey(_ + _).mapValues(0.15/N + 0.85 * _) 32 | } 33 | 34 | ranks.sortBy(_._2,false,1).take(30).foreach(println) 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /tests/diablo/spark/ReverseIndex.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | 5 | object Test { 6 | def main ( args: Array[String] ) { 7 | 8 | val conf = new SparkConf().setAppName("reverse-index") 9 | val sc = new SparkContext(conf) 10 | 11 | val regex = "]*?\\s+)?href=([\"'])(.*?)\\1".r 12 | 13 | var links = sc.wholeTextFiles("reverseIndex/") 14 | .map{ case (k, v) => (k, regex.findAllMatchIn(v).toList) } 15 | .flatMap{ case (k, v) => v.map(s => (k, s.toString)) } 16 | 17 | explain(true) 18 | 19 | v(sc,""" 20 | 21 | var R: map[String,String] = map(); 22 | 23 | for link in links do 24 | R[link._1] += link._2 + " "; 25 | 26 | R.foreach(println); 27 | 28 | """) 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /tests/diablo/spark/ReverseIndexSpark.scala: -------------------------------------------------------------------------------- 1 | import org.apache.spark.{SparkConf, SparkContext} 2 | 3 | object Test { 4 | def main ( args: Array[String] ) { 5 | 6 | val conf = new SparkConf().setAppName("reverse-index") 7 | val sc = new SparkContext(conf) 8 | 9 | val regex = "]*?\\s+)?href=([\"'])(.*?)\\1".r 10 | 11 | var ri = sc.wholeTextFiles("reverseIndex/") 12 | .map{ case (k, v) => (k, regex.findAllMatchIn(v).toList) } 13 | .flatMap{ case (k, v) => v.map(s => (s.toString, k)) } 14 | .reduceByKey{ case (x, y) => x +","+ y } 15 | 16 | ri.foreach(println) 17 | } 18 | } 19 | -------------------------------------------------------------------------------- /tests/diablo/spark/StringMatch.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | 5 | object Test { 6 | def main(args: Array[String]) { 7 | val conf = new SparkConf().setAppName("StringMatch") 8 | val sc = new SparkContext(conf) 9 | 10 | val words = sc.textFile(args(0)) 11 | .flatMap( line => line.split(" ") ) 12 | 13 | val keys = sc.textFile(args(1)).collect() 14 | 15 | explain(true) 16 | 17 | v(sc,""" 18 | 19 | var C: map[String,Boolean] = map(); 20 | 21 | for k in keys do { 22 | C[k] := false; 23 | for w in words do 24 | if (w == k) 25 | C[k] := true; 26 | }; 27 | 28 | """) 29 | 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /tests/diablo/spark/StringMatchSpark.scala: -------------------------------------------------------------------------------- 1 | import org.apache.spark.SparkContext 2 | import org.apache.spark.SparkConf 3 | 4 | object Test { 5 | def main(args: Array[String]) { 6 | 7 | val conf = new SparkConf().setAppName("StringMatch") 8 | val sc = new SparkContext(conf) 9 | 10 | val words = sc.textFile(args(0)) 11 | .flatMap( line => line.split(" ") ) 12 | 13 | val keys = sc.textFile(args(1)).collect() 14 | 15 | val res = words.filter{ w => keys.contains(w) }.distinct() 16 | 17 | res.foreach(println) 18 | 19 | sc.stop() 20 | 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /tests/diablo/spark/build: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../../..; pwd -P)" 4 | 5 | if [ -z ${SPARK_HOME} ]; then 6 | SPARK_HOME=~/spark-2.2.0-bin-hadoop2.6 7 | fi 8 | 9 | JARS=. 10 | for I in ${SPARK_HOME}/jars/*.jar; do 11 | JARS=${JARS}:$I 12 | done 13 | for I in ${SPARK_HOME}/lib/*.jar; do 14 | JARS=${JARS}:$I 15 | done 16 | 17 | mkdir -p classes 18 | scalac -d classes -cp ${JARS}:${DIQL_HOME}/lib/diql-spark.jar $* 19 | -------------------------------------------------------------------------------- /tests/diablo/spark/c: -------------------------------------------------------------------------------- 1 | 1.2,1.2 2 | 1.2,3.2 3 | 1.2,5.2 4 | 1.2,7.2 5 | 1.2,9.2 6 | 1.2,11.2 7 | 1.2,13.2 8 | 1.2,15.2 9 | 1.2,17.2 10 | 1.2,19.2 11 | 3.2,1.2 12 | 3.2,3.2 13 | 3.2,5.2 14 | 3.2,7.2 15 | 3.2,9.2 16 | 3.2,11.2 17 | 3.2,13.2 18 | 3.2,15.2 19 | 3.2,17.2 20 | 3.2,19.2 21 | 5.2,1.2 22 | 5.2,3.2 23 | 5.2,5.2 24 | 5.2,7.2 25 | 5.2,9.2 26 | 5.2,11.2 27 | 5.2,13.2 28 | 5.2,15.2 29 | 5.2,17.2 30 | 5.2,19.2 31 | 7.2,1.2 32 | 7.2,3.2 33 | 7.2,5.2 34 | 7.2,7.2 35 | 7.2,9.2 36 | 7.2,11.2 37 | 7.2,13.2 38 | 7.2,15.2 39 | 7.2,17.2 40 | 7.2,19.2 41 | 9.2,1.2 42 | 9.2,3.2 43 | 9.2,5.2 44 | 9.2,7.2 45 | 9.2,9.2 46 | 9.2,11.2 47 | 9.2,13.2 48 | 9.2,15.2 49 | 9.2,17.2 50 | 9.2,19.2 51 | 11.2,1.2 52 | 11.2,3.2 53 | 11.2,5.2 54 | 11.2,7.2 55 | 11.2,9.2 56 | 11.2,11.2 57 | 11.2,13.2 58 | 11.2,15.2 59 | 11.2,17.2 60 | 11.2,19.2 61 | 13.2,1.2 62 | 13.2,3.2 63 | 13.2,5.2 64 | 13.2,7.2 65 | 13.2,9.2 66 | 13.2,11.2 67 | 13.2,13.2 68 | 13.2,15.2 69 | 13.2,17.2 70 | 13.2,19.2 71 | 15.2,1.2 72 | 15.2,3.2 73 | 15.2,5.2 74 | 15.2,7.2 75 | 15.2,9.2 76 | 15.2,11.2 77 | 15.2,13.2 78 | 15.2,15.2 79 | 15.2,17.2 80 | 15.2,19.2 81 | 17.2,1.2 82 | 17.2,3.2 83 | 17.2,5.2 84 | 17.2,7.2 85 | 17.2,9.2 86 | 17.2,11.2 87 | 17.2,13.2 88 | 17.2,15.2 89 | 17.2,17.2 90 | 17.2,19.2 91 | 19.2,1.2 92 | 19.2,3.2 93 | 19.2,5.2 94 | 19.2,7.2 95 | 19.2,9.2 96 | 19.2,11.2 97 | 19.2,13.2 98 | 19.2,15.2 99 | 19.2,17.2 100 | 19.2,19.2 101 | -------------------------------------------------------------------------------- /tests/diablo/spark/f.txt: -------------------------------------------------------------------------------- 1 | 0,0,5 2 | 0,1,3 3 | 0,3,1 4 | 1,0,4 5 | 1,3,1 6 | 2,0,1 7 | 2,1,1 8 | 2,3,5 9 | 3,0,1 10 | 3,3,4 11 | 4,1,1 12 | 4,2,5 13 | 4,3,4 14 | -------------------------------------------------------------------------------- /tests/diablo/spark/groupBy.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | 5 | case class R ( K: Long, A: Double ) 6 | 7 | object Test { 8 | 9 | def main ( args: Array[String] ) { 10 | val conf = new SparkConf().setAppName("Test") 11 | val sc = new SparkContext(conf) 12 | 13 | explain(true) 14 | 15 | val V = sc.textFile(args(0)).map { 16 | case line 17 | => val a = line.split(",") 18 | R(a(0).toLong, a(1).toDouble) 19 | } 20 | 21 | v(sc,""" 22 | 23 | var C: vector[Double] = vector(); 24 | 25 | for v in V do { 26 | C[v.K] += v.A; 27 | 28 | C.foreach(println); 29 | 30 | }; 31 | 32 | """) 33 | 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /tests/diablo/spark/groupBy2.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.sql._ 4 | import org.apache.spark.rdd._ 5 | 6 | case class R ( K: Long, A:Double) 7 | 8 | object Test { 9 | 10 | def main ( args: Array[String] ) { 11 | val conf = new SparkConf().setAppName("Test") 12 | val sc = new SparkContext(conf) 13 | 14 | explain(true) 15 | 16 | val V = sc.textFile(args(0)).zipWithIndex.map { 17 | case (line,i) 18 | => val a = line.split(",") 19 | (i.toLong, R(a(0).toLong, a(1).toDouble)) 20 | } 21 | 22 | val n = V.count() 23 | 24 | v(sc,""" 25 | 26 | var C: vector[Double] = vector(); 27 | 28 | for i = 0, n-1 do { 29 | C[V[i].K] += V[i].A; 30 | 31 | C.foreach(println); 32 | 33 | }; 34 | 35 | """) 36 | 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /tests/diablo/spark/ih.txt: -------------------------------------------------------------------------------- 1 | 1,4,7 2 | 1,4,7 3 | 2,5,8 4 | 2,5,8 5 | 3,6,9 -------------------------------------------------------------------------------- /tests/diablo/spark/k.txt: -------------------------------------------------------------------------------- 1 | key1 2 | key2 3 | key3 -------------------------------------------------------------------------------- /tests/diablo/spark/lr.txt: -------------------------------------------------------------------------------- 1 | 1,1 2 | 2,2 3 | 3,3 4 | 4,4 5 | 5,5 6 | -------------------------------------------------------------------------------- /tests/diablo/spark/pagerank.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import Math._ 5 | 6 | object Test { 7 | 8 | def main ( args: Array[String] ) { 9 | val conf = new SparkConf().setAppName("Test") 10 | val sc = new SparkContext(conf) 11 | 12 | explain(true) 13 | 14 | val E = sc.textFile(args(0)) 15 | .map( line => { val a = line.split(",").toList 16 | ((a(0).toLong,a(1).toLong),true) } ) 17 | 18 | v(sc,""" 19 | 20 | var P: vector[Double] = vector(); 21 | var Q: vector[Double] = vector(); 22 | var C: vector[Int] = vector(); 23 | var N: Int = args(1).toInt; 24 | var b: Double = 0.85; 25 | 26 | for i = 1, N do { 27 | C[i] := 0; 28 | P[i] := 1.0/N; 29 | }; 30 | 31 | for i = 1, N do 32 | for j = 1, N do 33 | if (E[i,j]) 34 | C[i] += 1; 35 | 36 | var k: Int = 0; 37 | 38 | while (k < 10) { 39 | k += 1; 40 | for i = 1, N do 41 | Q[i] := P[i]; 42 | for i = 1, N do 43 | P[i] := (1-b)/N; 44 | for i = 1, N do 45 | for j = 1, N do 46 | if (E[j,i]) 47 | P[i] += b*Q[j]/C[j]; 48 | }; 49 | 50 | P.sortBy(_._2,false,1).take(30).foreach(println); 51 | 52 | """) 53 | 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /tests/diablo/spark/pagerank2.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import Math._ 5 | 6 | object Test { 7 | 8 | def main ( args: Array[String] ) { 9 | val conf = new SparkConf().setAppName("Test") 10 | val sc = new SparkContext(conf) 11 | 12 | explain(true) 13 | 14 | val E = sc.textFile(args(0)) 15 | .map( line => { val a = line.split(",").toList 16 | ((a(0).toLong,a(1).toLong),true) } ) 17 | 18 | v(sc,""" 19 | 20 | var P: vector[Double] = vector(); 21 | var C: vector[Int] = vector(); 22 | var N: Int = args(1).toInt; 23 | var b: Double = 0.85; 24 | 25 | for i = 1, N do { 26 | C[i] := 0; 27 | P[i] := 1.0/N; 28 | }; 29 | 30 | for i = 1, N do 31 | for j = 1, N do 32 | if (E[i,j]) 33 | C[i] += 1; 34 | 35 | var k: Int = 0; 36 | 37 | while (k < 10) { 38 | var Q: matrix[Double] = matrix(); 39 | k += 1; 40 | for i = 1, N do 41 | for j = 1, N do 42 | if (E[i,j]) 43 | Q[i,j] := P[i]; 44 | for i = 1, N do 45 | P[i] := (1-b)/N; 46 | for i = 1, N do 47 | for j = 1, N do 48 | P[i] += b*Q[j,i]/C[j]; 49 | }; 50 | 51 | P.sortBy(_._2,false,1).take(30).foreach(println); 52 | 53 | """) 54 | 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /tests/diablo/spark/pca.txt: -------------------------------------------------------------------------------- 1 | 0,0,1.0 2 | 0,2,3.0 3 | 3,0,30.0 4 | 0,1,2.0 5 | 1,0,2.0 6 | 1,1,4.0 7 | 1,2,6.0 8 | 3,1,6.0 9 | 3,2,90.0 10 | -------------------------------------------------------------------------------- /tests/diablo/spark/q: -------------------------------------------------------------------------------- 1 | 3.5 2 | 2.1 3 | 4.3 4 | -------------------------------------------------------------------------------- /tests/diablo/spark/reverseIndex/1.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | -------------------------------------------------------------------------------- /tests/diablo/spark/reverseIndex/2.html: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | -------------------------------------------------------------------------------- /tests/diablo/spark/run: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../../..; pwd -P)" 4 | 5 | if [ -z ${SPARK_HOME} ]; then 6 | SPARK_HOME=~/spark-2.2.0-bin-hadoop2.6 7 | fi 8 | 9 | export HADOOP_CONF_DIR= 10 | 11 | jar cf test.jar -C classes . 12 | 13 | ${SPARK_HOME}/bin/spark-submit --jars ${DIQL_HOME}/lib/diql-spark.jar --class Test --master local[2] test.jar $* 14 | -------------------------------------------------------------------------------- /tests/diablo/spark/small-graph.txt: -------------------------------------------------------------------------------- 1 | 1,2 2 | 2,3 3 | 2,4 4 | 3,4 5 | 4,1 6 | -------------------------------------------------------------------------------- /tests/diablo/spark/sum.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ) { 8 | val conf = new SparkConf().setAppName("Test") 9 | val sc = new SparkContext(conf) 10 | 11 | explain(true) 12 | 13 | val A = sc.textFile(args(0)).map( line => line.toDouble ) 14 | 15 | val N = A.count() 16 | 17 | v(sc,""" 18 | var sum: Double = 0.0; 19 | var count: Int = 0; 20 | 21 | for a in A do { 22 | sum += a; 23 | count += 1; 24 | }; 25 | 26 | println(sum+" "+count); 27 | """) 28 | 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /tests/diablo/spark/t.txt: -------------------------------------------------------------------------------- 1 | The focus of this course is on data management techniques and tools for storing and analyzing very large amounts of data Topics that will be covered include cloud computing virtualization distributed file systems large data processing using Map Reduce 2 | data modeling storage indexing and query processing for big data key value storage systems columnar databases NoSQL systems Cassandra BigTable HBase MongoDB big data technologies and tools Hive Pig Spark Flink large scale 3 | stream processing systems Storm Spark Streaming data analytics frameworks Mahout big data applications including graph processing recommendation systems machine learning clustering classification prediction and stream mining 4 | -------------------------------------------------------------------------------- /tests/diablo/spark/w: -------------------------------------------------------------------------------- 1 | a 2 | bb 3 | c 4 | -------------------------------------------------------------------------------- /tests/diablo/spark/w.txt: -------------------------------------------------------------------------------- 1 | a 2 | bb 3 | c 4 | key1 5 | -------------------------------------------------------------------------------- /tests/diablo/spark/wordCount.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import Math._ 5 | 6 | object Test { 7 | 8 | def main ( args: Array[String] ) { 9 | val conf = new SparkConf().setAppName("Test") 10 | val sc = new SparkContext(conf) 11 | 12 | explain(true) 13 | 14 | val words = sc.textFile(args(0)) 15 | .flatMap( line => line.split(" ") ) 16 | 17 | v(sc,""" 18 | 19 | var C: map[String,Int] = map(); 20 | 21 | for w in words do 22 | C[w] += 1; 23 | 24 | C.foreach(println); 25 | 26 | """) 27 | 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /tests/diablo/spark/wordCountSpark.scala: -------------------------------------------------------------------------------- 1 | import org.apache.spark.{SparkConf, SparkContext} 2 | 3 | object Test { 4 | def main ( args: Array[String] ) { 5 | 6 | val conf = new SparkConf().setAppName("Test") 7 | val sc = new SparkContext(conf) 8 | 9 | val words = sc.textFile(args(0)) 10 | .flatMap( line => line.split(" ") ) 11 | 12 | val counts = words.map((_,1)).reduceByKey(_+_) 13 | 14 | counts.foreach(println) 15 | 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /tests/flink/build: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../..; pwd -P)" 4 | 5 | if [ -z ${FLINK_HOME} ]; then 6 | FLINK_HOME=~/flink-1.6.0 7 | fi 8 | 9 | JARS=. 10 | for I in ${FLINK_HOME}/lib/*.jar; do 11 | JARS=${JARS}:$I 12 | done 13 | 14 | mkdir -p classes 15 | env JAVA_OPTS="-Xmx2G" scalac -d classes -cp ${JARS}:${DIQL_HOME}/lib/diql-flink.jar $* 16 | -------------------------------------------------------------------------------- /tests/flink/g.txt: -------------------------------------------------------------------------------- 1 | 1,2 2 | 1,3 3 | 1,4 4 | 2,3 5 | 4,5 6 | 5,6 7 | 4,8 8 | -------------------------------------------------------------------------------- /tests/flink/graph.txt: -------------------------------------------------------------------------------- 1 | 3,2,1 2 | 2,4,3 3 | 1,3,4,6 4 | 5,6 5 | 6,5,7,1 6 | 0,8,9 7 | 4,2,1 8 | 8,0 9 | 9,0 10 | 7,6 11 | -------------------------------------------------------------------------------- /tests/flink/kmeans.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright © 2017 University of Texas at Arlington 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | import edu.uta.diql._ 17 | import org.apache.flink.api.scala._ 18 | 19 | object Test { 20 | 21 | def main ( args: Array[String] ) { 22 | val env = ExecutionEnvironment.getExecutionEnvironment 23 | 24 | explain(true) 25 | 26 | case class Point ( X: Double, Y: Double ) 27 | 28 | def distance ( x: Point, y: Point ): Double 29 | = Math.sqrt(Math.pow(x.X-y.X,2)+Math.pow(x.Y-y.Y,2)) 30 | 31 | q("""let points = env.readTextFile("points.txt") 32 | .map( _.split(",") ) 33 | .map( p => Point(p(0).toDouble,p(1).toDouble) ) 34 | in repeat centroids = Array( Point(0,0), Point(10,0), Point(0,10), Point(10,10) ) 35 | step select Point( avg/x, avg/y ) 36 | from p@Point(x,y) <- points 37 | group by k: ( select c 38 | from c <- centroids 39 | order by distance(c,p) ).head 40 | limit 10 41 | """).foreach(println) 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /tests/flink/run: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../..; pwd -P)" 4 | 5 | if [ -z ${FLINK_HOME} ]; then 6 | FLINK_HOME=~/flink-1.6.0 7 | fi 8 | 9 | export HADOOP_CONF_DIR= 10 | 11 | jar cf test.jar -C classes . 12 | 13 | JARS=. 14 | for I in ${FLINK_HOME}/lib/*.jar; do 15 | JARS=${JARS}:$I 16 | done 17 | 18 | ${FLINK_HOME}/bin/flink run -q -c Test -C file://${DIQL_HOME}/lib/diql-flink.jar test.jar $* 19 | -------------------------------------------------------------------------------- /tests/parallel/build: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../..; pwd -P)" 4 | 5 | mkdir -p classes 6 | scalac -d classes -cp ${DIQL_HOME}/lib/diql-parallel.jar $* 7 | -------------------------------------------------------------------------------- /tests/parallel/graph.txt: -------------------------------------------------------------------------------- 1 | 3,2,1 2 | 2,4,3 3 | 1,3,4,6 4 | 5,6 5 | 6,5,7,1 6 | 0,8,9 7 | 4,2,1 8 | 8,0 9 | 9,0 10 | 7,6 11 | -------------------------------------------------------------------------------- /tests/parallel/kmeans.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright © 2017 University of Texas at Arlington 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | import edu.uta.diql._ 17 | import scala.io.Source 18 | import scala.collection.parallel.ParIterable 19 | 20 | object Test { 21 | 22 | def main ( args: Array[String] ) { 23 | explain(true) 24 | 25 | case class Point ( X: Double, Y: Double ) 26 | 27 | def distance ( x: Point, y: Point ): Double 28 | = Math.sqrt(Math.pow(x.X-y.X,2)+Math.pow(x.Y-y.Y,2)) 29 | 30 | q("""let points = Source.fromFile("points.txt") 31 | .getLines.toList.par 32 | .map( _.split(",") ) 33 | .map( p => Point(p(0).toDouble,p(1).toDouble) ) 34 | in repeat centroids = Array( Point(0,0), Point(10,0), Point(0,10), Point(10,10) ) 35 | step select Point( avg/x, avg/y ) 36 | from p@Point(x,y) <- points 37 | group by k: ( select c 38 | from c <- centroids 39 | order by distance(c,p) ).head 40 | limit 10 41 | """).map(println) 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /tests/parallel/map.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import scala.io.Source 3 | import scala.collection.parallel.ParIterable 4 | 5 | object Test { 6 | 7 | def main ( args: Array[String] ) { 8 | println("Number of cores: "+Runtime.getRuntime().availableProcessors()) 9 | println("Memory: "+(Runtime.getRuntime.maxMemory / 1024)) 10 | 11 | var t: Long = System.currentTimeMillis() 12 | val a = (1 to args(0).toInt).toList 13 | 14 | println("**** construct sequential: "+(System.currentTimeMillis()-t)/1000.0+" secs") 15 | 16 | t = System.currentTimeMillis() 17 | 18 | a.map(_+1) 19 | 20 | println("**** sequential: "+(System.currentTimeMillis()-t)/1000.0+" secs") 21 | 22 | t = System.currentTimeMillis() 23 | 24 | a.groupBy(x => x).map(x => (x,+1)) 25 | 26 | println("**** sequential group by: "+(System.currentTimeMillis()-t)/1000.0+" secs") 27 | 28 | t = System.currentTimeMillis() 29 | 30 | val c = (1 to args(0).toInt).toList.par 31 | 32 | println("**** construct parallel: "+(System.currentTimeMillis()-t)/1000.0+" secs") 33 | 34 | explain(true) 35 | 36 | t = System.currentTimeMillis() 37 | 38 | q(""" 39 | select v+1 from v <- c 40 | """) 41 | 42 | println("**** parallel: "+(System.currentTimeMillis()-t)/1000.0+" secs") 43 | 44 | t = System.currentTimeMillis() 45 | 46 | q(""" 47 | select (k, +/v) from v <- c group by k: v+1 48 | """) 49 | 50 | println("**** parallel group by: "+(System.currentTimeMillis()-t)/1000.0+" secs") 51 | 52 | } 53 | } 54 | 55 | -------------------------------------------------------------------------------- /tests/parallel/points.txt: -------------------------------------------------------------------------------- 1 | ../../benchmarks/diablo/data/P1 -------------------------------------------------------------------------------- /tests/parallel/run: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../..; pwd -P)" 4 | 5 | jar cf test.jar -C classes . 6 | 7 | scala -cp ${DIQL_HOME}/lib/diql-parallel.jar:test.jar -J-Xmx20G -J-Xms20G -J-Xss128M Test $* 8 | -------------------------------------------------------------------------------- /tests/scalding/build: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../..; pwd -P)" 4 | 5 | if [ -z ${SCALDING_HOME} ]; then 6 | SCALDING_HOME=${HOME}/scalding 7 | fi 8 | if [ -z ${HADOOP_HOME} ]; then 9 | HADOOP_HOME=${HOME}/hadoop-2.6.0 10 | fi 11 | 12 | HADOOP_JARS=`${HADOOP_HOME}/bin/hadoop classpath` 13 | JARS=.:${HADOOP_JARS} 14 | for I in ${SCALDING_HOME}/scalding-core/target/scala-2.11/scalding-core-assembly*.jar; do 15 | JARS=${JARS}:$I 16 | done 17 | 18 | mkdir -p classes 19 | scalac -d classes -cp ${JARS}:${DIQL_HOME}/lib/diql-scalding.jar $* 20 | -------------------------------------------------------------------------------- /tests/scalding/graph.txt: -------------------------------------------------------------------------------- 1 | 3,2,1 2 | 2,4,3 3 | 1,3,4,6 4 | 5,6 5 | 6,5,7,1 6 | 0,8,9 7 | 4,2,1 8 | 8,0 9 | 9,0 10 | 7,6 11 | -------------------------------------------------------------------------------- /tests/scalding/kmeans.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright © 2017 University of Texas at Arlington 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | import edu.uta.diql._ 17 | import com.twitter.scalding._ 18 | 19 | 20 | object Test extends ExecutionApp { 21 | explain(true) 22 | 23 | case class Point ( X: Double, Y: Double ) extends Comparable[Point] { 24 | def compareTo ( that: Point ): Int = (if (X != that.X) X-that.X else Y-that.Y).toInt 25 | } 26 | 27 | def distance ( x: Point, y: Point ): Double 28 | = Math.sqrt(Math.pow(x.X-y.X,2)+Math.pow(x.Y-y.Y,2)) 29 | 30 | val pointsDS = TypedPipe.from(TextLine("graph.txt")) 31 | 32 | def job: Execution[Unit] = { 33 | q("""let points = pointsDS.map( _.split(",") ) 34 | .map( p => Point(p(0).toDouble,p(1).toDouble) ) 35 | in repeat centroids = Array( Point(0,0), Point(10,0), Point(0,10), Point(10,10) ) 36 | step select Point( avg/x, avg/y ) 37 | from p@Point(x,y) <- points 38 | group by k: ( select c 39 | from c <- centroids 40 | order by distance(c,p) ).head 41 | limit 10 42 | """).debug.writeExecution(TypedTsv("results/output")) 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /tests/scalding/run: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../..; pwd -P)" 4 | SCALA_LIB="$(dirname `which scalac`)/../lib" 5 | 6 | if [ -z ${SCALDING_HOME} ]; then 7 | SCALDING_HOME=${HOME}/scalding 8 | fi 9 | if [ -z ${HADOOP_HOME} ]; then 10 | HADOOP_HOME=${HOME}/hadoop-2.6.0 11 | fi 12 | 13 | jars=${DIQL_HOME}/lib/diql-scalding.jar 14 | for I in ${SCALDING_HOME}/scalding-core/target/scala-2.11/scalding-core-assembly*.jar; do 15 | jars=${jars},$I 16 | done 17 | for I in ${SCALA_LIB}/scala-parser-combinators*.jar; do 18 | jars=${jars},$I 19 | done 20 | 21 | export HADOOP_CLASSPATH=`echo ${jars} | sed s/,/:/g` 22 | 23 | export HADOOP_CONF_DIR= 24 | 25 | jar cf test.jar -C classes . 26 | mkdir -p results 27 | 28 | $HADOOP_HOME/bin/hadoop jar test.jar Test -libjars ${jars} -mode --local $* 29 | -------------------------------------------------------------------------------- /tests/spark/build: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../..; pwd -P)" 4 | 5 | if [ -z ${SPARK_HOME} ]; then 6 | SPARK_HOME=~/spark-3.0.0-bin-hadoop2.7 7 | fi 8 | 9 | JARS=. 10 | for I in ${SPARK_HOME}/jars/*.jar; do 11 | JARS=${JARS}:$I 12 | done 13 | for I in ${SPARK_HOME}/lib/*.jar; do 14 | JARS=${JARS}:$I 15 | done 16 | 17 | mkdir -p classes 18 | scalac -d classes -cp ${JARS}:${DIQL_HOME}/lib/diql-spark.jar $* 19 | -------------------------------------------------------------------------------- /tests/spark/g.txt: -------------------------------------------------------------------------------- 1 | 1,2 2 | 1,3 3 | 1,4 4 | 2,3 5 | 4,5 6 | 5,6 7 | 4,8 8 | -------------------------------------------------------------------------------- /tests/spark/graph.txt: -------------------------------------------------------------------------------- 1 | 3,2,1 2 | 2,4,3 3 | 1,3,4,6 4 | 5,6 5 | 6,5,7,1 6 | 0,8,9 7 | 4,2,1 8 | 8,0 9 | 9,0 10 | 7,6 11 | -------------------------------------------------------------------------------- /tests/spark/kmeans.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright © 2017 University of Texas at Arlington 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | import edu.uta.diql._ 17 | import org.apache.spark._ 18 | import org.apache.spark.rdd._ 19 | 20 | object Test { 21 | 22 | def main ( args: Array[String] ) { 23 | val conf = new SparkConf().setAppName("k-means") 24 | val sc = new SparkContext(conf) 25 | 26 | explain(true) 27 | 28 | case class Point ( X: Double, Y: Double ) 29 | 30 | def distance ( x: Point, y: Point ): Double 31 | = Math.sqrt(Math.pow(x.X-y.X,2)+Math.pow(x.Y-y.Y,2)) 32 | 33 | q("""let points = sc.textFile("points.txt") 34 | .map( _.split(",") ) 35 | .map( p => Point(p(0).toDouble,p(1).toDouble) ) 36 | in repeat centroids = Array( Point(0,0), Point(10,0), Point(0,10), Point(10,10) ) 37 | step select Point( avg/x, avg/y ) 38 | from p@Point(x,y) <- points 39 | group by k: ( select c 40 | from c <- centroids 41 | order by distance(c,p) ).head 42 | limit 10 43 | """).map(println) 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /tests/spark/multiply.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright © 2017 University of Texas at Arlington 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | import edu.uta.diql._ 17 | import org.apache.spark._ 18 | import org.apache.spark.rdd._ 19 | 20 | object Test { 21 | 22 | def main ( args: Array[String] ) { 23 | val conf = new SparkConf().setAppName("Test") 24 | val sc = new SparkContext(conf) 25 | 26 | explain(true) 27 | 28 | var M = sc.textFile(args(0)) 29 | .map( line => { val a = line.split(",") 30 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ) 31 | var N = sc.textFile(args(1)) 32 | .map( line => { val a = line.split(",") 33 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ) 34 | 35 | q(""" 36 | 37 | select ( (i,j), +/v ) 38 | from ((i,k),m) <-M, ((kk,j),n) <- N, v = m*n 39 | where k == kk 40 | group by (i,j) 41 | 42 | """).sortBy(_._1,true,1).take(30).foreach(println) 43 | 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /tests/spark/run: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../..; pwd -P)" 4 | 5 | if [ -z ${SPARK_HOME} ]; then 6 | SPARK_HOME=~/spark-3.0.0-bin-hadoop2.7 7 | fi 8 | 9 | export HADOOP_CONF_DIR= 10 | 11 | jar cf test.jar -C classes . 12 | 13 | ${SPARK_HOME}/bin/spark-submit --jars ${DIQL_HOME}/lib/diql-spark.jar --class Test --master local[2] test.jar $* 14 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/GroupBy.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.spark.sql._ 5 | 6 | case class Cclass ( K: Long, A:Double) 7 | 8 | object Test { 9 | 10 | def main ( args: Array[String] ) { 11 | val conf = new SparkConf() 12 | .setAppName("Test") 13 | .setMaster("local[2]") 14 | 15 | val sc = new SparkContext(conf) 16 | 17 | val spark = SparkSession 18 | .builder() 19 | .config(conf) 20 | .getOrCreate() 21 | 22 | import spark.implicits._ 23 | explain(true) 24 | 25 | 26 | val V = sc.textFile(args(0)).zipWithIndex.map{ 27 | case (line,i) => {val a = line.split(",") 28 | (i.toLong, Cclass(a(0).toLong, a(1).toDouble)) }}.toDS() 29 | 30 | V.createOrReplaceTempView("V") 31 | 32 | 33 | s(sc,""" 34 | 35 | var C: vector[Double] = vector(); 36 | 37 | for i = 0, 10 do { 38 | C[V[i].K] += V[i].A; 39 | }; 40 | 41 | println(C); 42 | """) 43 | 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/GroupBy2.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.spark.sql._ 5 | 6 | case class Cclass ( K: Long, A:Double) 7 | 8 | object Test { 9 | 10 | def main ( args: Array[String] ) { 11 | val conf = new SparkConf() 12 | .setAppName("Test") 13 | .setMaster("local[2]") 14 | 15 | val sc = new SparkContext(conf) 16 | 17 | val spark = SparkSession 18 | .builder() 19 | .config(conf) 20 | .getOrCreate() 21 | 22 | import spark.implicits._ 23 | explain(true) 24 | 25 | 26 | val V = sc.textFile(args(0)).zipWithIndex.map{ 27 | case (line,i) => {val a = line.split(",") 28 | (i.toLong, Cclass(a(0).toLong, a(1).toDouble)) }}.toDS() 29 | val n = V.count() 30 | 31 | V.createOrReplaceTempView("V") 32 | V.printSchema() 33 | 34 | s(sc,""" 35 | 36 | var C: vector[Double] = vector(); 37 | 38 | for i = 0, n-1 do { 39 | C[V[i].K] += V[i].A; 40 | }; 41 | println(C); 42 | """) 43 | 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/GroupByJoin.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.spark.sql._ 5 | import Math._ 6 | 7 | object Test { 8 | def main ( args: Array[String] ) { 9 | val conf = new SparkConf() 10 | .setAppName("Test") 11 | .setMaster("local[2]") 12 | 13 | val sc = new SparkContext(conf) 14 | 15 | val spark = SparkSession 16 | .builder() 17 | .config(conf) 18 | .getOrCreate() 19 | 20 | import spark.implicits._ 21 | 22 | explain(true) 23 | 24 | //v.txt 25 | val V = sc.textFile(args(0)).map( line => { val a = line.split(",");(a(0).toLong,a(1).toLong)}).toDS() 26 | //k.txt 27 | val K = sc.textFile(args(1)).zipWithIndex.map{ case (line,i) => (i, line.toLong) }.toDS() 28 | 29 | val n = V.count() 30 | 31 | 32 | K.createOrReplaceTempView("K") 33 | V.createOrReplaceTempView("V") 34 | 35 | s(sc,""" 36 | var W: vector[Double] = vector(); 37 | 38 | for i = 0, 9 do { 39 | W[K[i]] += V[i]; 40 | }; 41 | 42 | println(W); 43 | """) 44 | 45 | } 46 | } 47 | 48 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/GroupByJoin2.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.spark.sql._ 5 | import Math._ 6 | 7 | object Test { 8 | def main ( args: Array[String] ) { 9 | val conf = new SparkConf() 10 | .setAppName("Test") 11 | .setMaster("local[2]") 12 | 13 | val sc = new SparkContext(conf) 14 | 15 | val spark = SparkSession 16 | .builder() 17 | .config(conf) 18 | .getOrCreate() 19 | 20 | import spark.implicits._ 21 | 22 | explain(true) 23 | 24 | //v.txt 25 | val V = sc.textFile(args(0)).map( line => { val a = line.split(",");(a(0).toLong,a(1).toLong)}).toDS() 26 | //k.txt 27 | val K = sc.textFile(args(1)).zipWithIndex.map{ case (line,i) => (i, line.toLong) }.toDS() 28 | 29 | val n = V.count() 30 | 31 | V.createOrReplaceTempView("V") 32 | K.createOrReplaceTempView("K") 33 | 34 | s(sc,""" 35 | var W: vector[Double] = vector(); 36 | 37 | for i = 0, n-1 do { 38 | W[K[i]] += V[i]; 39 | }; 40 | println(W); 41 | """) 42 | 43 | } 44 | } 45 | 46 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/Histogram.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.spark.sql._ 5 | 6 | case class Color ( red: Int, green: Int, blue: Int ) 7 | 8 | object Test { 9 | 10 | def main ( args: Array[String] ) { 11 | val conf = new SparkConf().setAppName("Test") 12 | val sc = new SparkContext(conf) 13 | 14 | val spark = SparkSession 15 | .builder() 16 | .config(conf) 17 | .getOrCreate() 18 | 19 | import spark.implicits._ 20 | explain(true) 21 | 22 | var P = sc.textFile(args(0)).zipWithIndex.map{ 23 | case (line,i) => {val a = line.split(",") 24 | (i.toLong, Color(a(0).toInt,a(1).toInt,a(2).toInt)) }}.toDS() 25 | 26 | P.createOrReplaceTempView("P") 27 | 28 | 29 | s(sc,""" 30 | 31 | var R: map[Int,Int] = map(); 32 | var G: map[Int,Int] = map(); 33 | var B: map[Int,Int] = map(); 34 | 35 | for i = 1, 5 do { 36 | R[P[i].red] += 1; 37 | G[P[i].green] += 1; 38 | B[P[i].blue] += 1; 39 | }; 40 | 41 | println(R); 42 | println(G); 43 | println(B); 44 | 45 | """) 46 | 47 | } 48 | } 49 | 50 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/Histogram2.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.spark.sql._ 5 | 6 | case class Color ( red: Int, green: Int, blue: Int ) 7 | 8 | object Test { 9 | 10 | def main ( args: Array[String] ) { 11 | val conf = new SparkConf().setAppName("Test") 12 | val sc = new SparkContext(conf) 13 | 14 | val spark = SparkSession 15 | .builder() 16 | .config(conf) 17 | .getOrCreate() 18 | 19 | import spark.implicits._ 20 | explain(true) 21 | 22 | var P = sc.textFile(args(0)).zipWithIndex.map{ 23 | case (line,i) => {val a = line.split(",") 24 | (i.toLong, Color(a(0).toInt,a(1).toInt,a(2).toInt)) }}.toDS() 25 | val n = P.count() 26 | 27 | P.createOrReplaceTempView("P") 28 | 29 | 30 | s(sc,""" 31 | 32 | var R: map[Int,Int] = map(); 33 | var G: map[Int,Int] = map(); 34 | var B: map[Int,Int] = map(); 35 | 36 | for i = 0, n-1 do { 37 | R[P[i].red] += 1; 38 | G[P[i].green] += 1; 39 | B[P[i].blue] += 1; 40 | }; 41 | 42 | println(R); 43 | println(G); 44 | println(B); 45 | 46 | """) 47 | 48 | } 49 | } 50 | 51 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/MatrixAddition.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.sql._ 4 | import org.apache.spark.rdd._ 5 | 6 | object Test { 7 | 8 | def main ( args: Array[String] ) { 9 | val conf = new SparkConf() 10 | .setAppName("Test") 11 | .setMaster("local[2]") 12 | 13 | val sc = new SparkContext(conf) 14 | 15 | val spark = SparkSession 16 | .builder() 17 | .config(conf) 18 | .getOrCreate() 19 | 20 | import spark.implicits._ 21 | explain(true) 22 | 23 | val n = args(2).toLong 24 | val m = n 25 | 26 | val M = sc.textFile(args(0)).zipWithIndex.map{ 27 | case (line,i) => {val a = line.split(",") 28 | ((a(0).toLong, a(1).toLong), a(2).toDouble) }}.toDS() 29 | 30 | val N = sc.textFile(args(1)).zipWithIndex.map{ 31 | case (line,i) => {val a = line.split(",") 32 | ((a(0).toLong, a(1).toLong), a(2).toDouble) }}.toDS() 33 | 34 | M.createOrReplaceTempView("M") 35 | N.createOrReplaceTempView("N") 36 | 37 | 38 | s(sc,""" 39 | var R: matrix[Double] = matrix(); 40 | for i = 0, 10 do 41 | for j = 0, 10 do 42 | R[i,j] := M[i,j] + N[i,j]; 43 | println(R); 44 | """) 45 | 46 | } 47 | } 48 | 49 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/MatrixAddition2.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.sql._ 4 | import org.apache.spark.rdd._ 5 | 6 | object Test { 7 | 8 | def main ( args: Array[String] ) { 9 | val conf = new SparkConf() 10 | .setAppName("Test") 11 | .setMaster("local[2]") 12 | 13 | val sc = new SparkContext(conf) 14 | 15 | val spark = SparkSession 16 | .builder() 17 | .config(conf) 18 | .getOrCreate() 19 | 20 | import spark.implicits._ 21 | explain(false) 22 | 23 | val n = args(2).toLong 24 | val m = n 25 | 26 | val M = sc.textFile(args(0)) 27 | .map( line => { val a = line.split(",") 28 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ).toDS() 29 | val N = sc.textFile(args(1)) 30 | .map( line => { val a = line.split(",") 31 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ).toDS() 32 | M.createOrReplaceTempView("M") 33 | N.createOrReplaceTempView("N") 34 | 35 | 36 | s(sc,""" 37 | var R: matrix[Double] = matrix(); 38 | for i = 0, n-1 do 39 | for j = 0, n-1 do 40 | R[i,j] := M[i,j] + N[i,j]; 41 | 42 | println(R); 43 | """) 44 | 45 | } 46 | } 47 | 48 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/MatrixMultiplication.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.sql._ 4 | import org.apache.spark.rdd._ 5 | 6 | case class M_Matrix ( i: Long, j: Long, v: Double ) 7 | case class N_Matrix ( i: Long, j: Long, v: Double ) 8 | 9 | 10 | object Test { 11 | 12 | def main ( args: Array[String] ) { 13 | val conf = new SparkConf() 14 | .setAppName("Test") 15 | .setMaster("local[2]") 16 | 17 | val sc = new SparkContext(conf) 18 | 19 | val spark = SparkSession 20 | .builder() 21 | .config(conf) 22 | .getOrCreate() 23 | 24 | import spark.implicits._ 25 | explain(true) 26 | 27 | val n = args(2).toLong 28 | val m = n 29 | 30 | val M = sc.textFile(args(0)).zipWithIndex.map{ 31 | case (line,i) => {val a = line.split(",") 32 | ((a(0).toLong, a(1).toLong), M_Matrix(a(0).toLong, a(1).toLong, a(2).toDouble)) }}.toDS() 33 | 34 | val N = sc.textFile(args(1)).zipWithIndex.map{ 35 | case (line,i) => {val a = line.split(",") 36 | ((a(0).toLong, a(1).toLong), N_Matrix(a(0).toLong, a(1).toLong, a(2).toDouble)) }}.toDS() 37 | 38 | M.printSchema() 39 | M.createOrReplaceTempView("M") 40 | N.createOrReplaceTempView("N") 41 | 42 | s(sc,""" 43 | var R: matrix[Double] = matrix(); 44 | for i = 0, 3 do 45 | for j = 0, 3 do { 46 | R[i,j] := 0.0; 47 | for k = 0, 3 do 48 | R[i,j] += M[i,k].v*N[k,j].v; 49 | }; 50 | println(R); 51 | """) 52 | 53 | } 54 | } 55 | 56 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/MatrixMultiplication2.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.sql._ 4 | import org.apache.spark.rdd._ 5 | 6 | case class M_Matrix ( i: Long, j: Long, v: Double ) 7 | case class N_Matrix ( i: Long, j: Long, v: Double ) 8 | 9 | 10 | object Test { 11 | 12 | def main ( args: Array[String] ) { 13 | val conf = new SparkConf() 14 | .setAppName("Test") 15 | .setMaster("local[2]") 16 | 17 | val sc = new SparkContext(conf) 18 | 19 | val spark = SparkSession 20 | .builder() 21 | .config(conf) 22 | .getOrCreate() 23 | 24 | import spark.implicits._ 25 | explain(true) 26 | 27 | val n = args(2).toLong 28 | val m = n 29 | 30 | val M = sc.textFile(args(0)).zipWithIndex.map{ 31 | case (line,i) => {val a = line.split(",") 32 | ((a(0).toLong, a(1).toLong), M_Matrix(a(0).toLong, a(1).toLong, a(2).toDouble)) }}.toDS() 33 | 34 | val N = sc.textFile(args(1)).zipWithIndex.map{ 35 | case (line,i) => {val a = line.split(",") 36 | ((a(0).toLong, a(1).toLong), N_Matrix(a(0).toLong, a(1).toLong, a(2).toDouble)) }}.toDS() 37 | 38 | M.createOrReplaceTempView("M") 39 | N.createOrReplaceTempView("N") 40 | 41 | M.printSchema() 42 | N.printSchema() 43 | 44 | s(sc,""" 45 | var R: matrix[Double] = matrix(); 46 | for i = 0, n-1 do 47 | for j = 0, n-1 do { 48 | R[i,j] := 0.0; 49 | for k = 0, m-1 do 50 | R[i,j] += M[i,k].v*N[k,j].v; 51 | }; 52 | println(R); 53 | """) 54 | 55 | } 56 | } 57 | 58 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/MatrixMultiplication3.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.sql._ 4 | import org.apache.spark.rdd._ 5 | 6 | object Test { 7 | 8 | def main ( args: Array[String] ) { 9 | val conf = new SparkConf() 10 | .setAppName("Test") 11 | .setMaster("local[2]") 12 | 13 | val sc = new SparkContext(conf) 14 | 15 | val spark = SparkSession 16 | .builder() 17 | .config(conf) 18 | .getOrCreate() 19 | 20 | import spark.implicits._ 21 | explain(true) 22 | 23 | val n = args(2).toLong 24 | val m = n 25 | 26 | var M = sc.textFile(args(0)) 27 | .map( line => { val a = line.split(",") 28 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ).toDS() 29 | var N = sc.textFile(args(1)) 30 | .map( line => { val a = line.split(",") 31 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ).toDS() 32 | M.createOrReplaceTempView("M") 33 | N.createOrReplaceTempView("N") 34 | M.printSchema() 35 | 36 | s(sc,""" 37 | var R: matrix[Double] = matrix(); 38 | for i = 0, n-1 do 39 | for j = 0, n-1 do 40 | for k = 0, m-1 do 41 | R[i,j] += M[i,k]*N[k,j]; 42 | println(R); 43 | """) 44 | 45 | } 46 | } 47 | 48 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/PCA.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.spark.sql._ 5 | 6 | case class Dataset(I: Long, J: Long, V: Double) 7 | 8 | object Test { 9 | def main ( args: Array[String] ) { 10 | val conf = new SparkConf() 11 | .setAppName("Test") 12 | .setMaster("local[2]") 13 | 14 | val sc = new SparkContext(conf) 15 | 16 | val spark = SparkSession 17 | .builder() 18 | .config(conf) 19 | .getOrCreate() 20 | 21 | import spark.implicits._ 22 | explain(true) 23 | 24 | 25 | val P = sc.textFile(args(0)).zipWithIndex.map{ 26 | case (line,i) => {val a = line.split(",") 27 | ((a(0).toLong, a(1).toLong), Dataset(a(0).toLong, a(1).toLong, a(2).toDouble)) }}.toDS() 28 | var n = P.count() 29 | var r: Double = n/3; 30 | 31 | P.createOrReplaceTempView("P") 32 | P.printSchema() 33 | s(sc,""" 34 | 35 | var mean: vector[Double] = vector(); 36 | var ri: Int = r.toInt; 37 | var sum: matrix[Double] = matrix(); 38 | 39 | for i = 0, 8 do { 40 | for j = 0, 2 do 41 | mean[j] += P[i,j].V; 42 | }; 43 | 44 | for i = 0, 8 do { 45 | mean[i] := 0.0+mean[i]/3; 46 | }; 47 | 48 | for i = 0, 2 do { 49 | for j = 0, 2 do { 50 | sum[i,j] := 0.0; 51 | for k = 0, 2 do 52 | sum[i,j] += (P[k,i].V - mean[i])*(P[k,j].V-mean[j]); 53 | }; 54 | }; 55 | 56 | for i = 0, 2 do { 57 | for j = 0, 2 do { 58 | sum[i,j] := 0.0 + sum[i,j]/2; 59 | }; 60 | }; 61 | println(sum); 62 | """) 63 | 64 | } 65 | } 66 | 67 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/PCA2.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.spark.sql._ 5 | 6 | object Test { 7 | def main ( args: Array[String] ) { 8 | val conf = new SparkConf() 9 | .setAppName("Test") 10 | .setMaster("local[2]") 11 | 12 | val sc = new SparkContext(conf) 13 | 14 | val spark = SparkSession 15 | .builder() 16 | .config(conf) 17 | .getOrCreate() 18 | 19 | import spark.implicits._ 20 | explain(true) 21 | 22 | var P = sc.textFile(args(0)) 23 | .map( line => { val a = line.split(",") 24 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ).toDS() 25 | var n = P.count() 26 | var d: Int =4; 27 | var r: Double = n/d; 28 | var m: Int = r.toInt; 29 | 30 | P.createOrReplaceTempView("P") 31 | 32 | s(sc,""" 33 | 34 | var mean: vector[Double] = vector(); 35 | var sum: matrix[Double] = matrix(); 36 | 37 | for i = 0, n-1 do { 38 | for j = 0, m-1 do 39 | mean[j] += P[i,j]; 40 | }; 41 | 42 | for i = 0, n-1 do { 43 | mean[i] := 0.0+mean[i]/4; 44 | }; 45 | 46 | for i = 0, d-1 do { 47 | for j = 0, d-1 do { 48 | sum[i,j] := 0.0; 49 | for k = 0, m-1 do 50 | sum[i,j] += (P[k,i]-mean[i])*(P[k,j]-mean[j]); 51 | }; 52 | }; 53 | 54 | for i = 0, d-1 do { 55 | for j = 0, d-1 do { 56 | sum[i,j] := 0.0 + sum[i,j]/3; 57 | }; 58 | }; 59 | println(sum); 60 | """) 61 | 62 | } 63 | } 64 | 65 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/PCA3.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.spark.sql._ 5 | 6 | object Test { 7 | def main ( args: Array[String] ) { 8 | val conf = new SparkConf() 9 | .setAppName("Test") 10 | .setMaster("local[2]") 11 | 12 | val sc = new SparkContext(conf) 13 | 14 | val spark = SparkSession 15 | .builder() 16 | .config(conf) 17 | .getOrCreate() 18 | 19 | import spark.implicits._ 20 | explain(true) 21 | 22 | var P = sc.textFile(args(0)) 23 | .map( line => { val a = line.split(",") 24 | ((a(0).toLong,a(1).toLong),a(2).toDouble) } ).toDS() 25 | var n = P.count() 26 | var d: Int = 4; 27 | var r: Double = n/d; 28 | var m: Int = r.toInt+1; 29 | var z = r-1 30 | P.createOrReplaceTempView("P") 31 | 32 | s(sc,""" 33 | 34 | var mean: vector[Double] = vector(); 35 | var sum: matrix[Double] = matrix(); 36 | 37 | for i = 0, n-1 do { 38 | for j = 0, m-1 do 39 | mean[j] += P[i,j]; 40 | }; 41 | 42 | for i = 0, n-1 do { 43 | mean[i] := 0.0+mean[i]/r; 44 | }; 45 | 46 | for i = 0, d-1 do { 47 | for j = 0, d-1 do { 48 | sum[i,j] := 0.0; 49 | for k = 0, m-1 do 50 | sum[i,j] += (P[k,i]-mean[i])*(P[k,j]-mean[j]); 51 | }; 52 | }; 53 | 54 | for i = 0, d-1 do { 55 | for j = 0, d-1 do { 56 | sum[i,j] := 0.0 + sum[i,j]/z; 57 | }; 58 | }; 59 | println(sum); 60 | """) 61 | 62 | } 63 | } 64 | 65 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/PageRank.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.spark.sql._ 5 | import Math._ 6 | 7 | object Test { 8 | 9 | def main ( args: Array[String] ) { 10 | val conf = new SparkConf() 11 | .setAppName("Test") 12 | .setMaster("local[2]") 13 | 14 | val sc = new SparkContext(conf) 15 | val spark = SparkSession 16 | .builder() 17 | .config(conf) 18 | .getOrCreate() 19 | 20 | import spark.implicits._ 21 | 22 | explain(true) 23 | var n: Int = args(1).toInt; 24 | var b: Double = 0.8; 25 | var c: Double = b/n; 26 | val E = sc.textFile(args(0)) 27 | .map( line => { val a = line.split(",").toList 28 | ((a(0).toLong,a(1).toLong),true)}).toDS() 29 | E.createOrReplaceTempView("E") 30 | val R = sc.parallelize(1L to n).map(v => (v,(1.0-b)/n)).toDS() 31 | R.createOrReplaceTempView("R") 32 | 33 | s(sc,""" 34 | var C: vector[Int] = vector(); 35 | var S: vector[Double] = vector(); 36 | var I: vector[Double] = vector(); 37 | 38 | for i = 1, n do 39 | for j = 1, n do 40 | if (E[i,j]) 41 | C[i] += 1; 42 | 43 | for i = 1, n do{ 44 | for j = 1, n do{ 45 | if (E[j,i]) 46 | S[i] += c/C[j]; 47 | }; 48 | }; 49 | 50 | for i = 1, n do 51 | I[i] += S[i] + R[i]; 52 | 53 | println(I); 54 | """) 55 | 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/StringMatch.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.sql._ 4 | import org.apache.spark.rdd._ 5 | 6 | case class words (W: String) 7 | case class keys (K:String) 8 | 9 | object Test { 10 | 11 | def main ( args: Array[String] ) { 12 | val conf = new SparkConf() 13 | .setAppName("Test") 14 | 15 | val sc = new SparkContext(conf) 16 | 17 | val spark = SparkSession 18 | .builder() 19 | .config(conf) 20 | .getOrCreate() 21 | 22 | import spark.implicits._ 23 | explain(true) 24 | 25 | //string.txt 26 | val W = sc.textFile(args(0)).zipWithIndex.map{ case (line,i) => (i, words(line))}.toDS() 27 | //key.txt 28 | val K = sc.textFile(args(1)).zipWithIndex.map{ case (line,i) => (i, keys(line))}.toDS() 29 | 30 | W.createOrReplaceTempView("W") 31 | K.createOrReplaceTempView("K") 32 | 33 | W.printSchema() 34 | 35 | s(sc,""" 36 | var C: map[String,Int] = map(); 37 | 38 | for j = 0, 3 do{ 39 | C[K[j].K] := 0; 40 | for i = 0, 5 do 41 | if (W[i].W == K[j].K) 42 | C[K[j].K] += 1; 43 | }; 44 | 45 | println(C); 46 | 47 | """) 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/StringMatch2.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.sql._ 4 | import org.apache.spark.rdd._ 5 | 6 | case class words (W: String) 7 | case class keys (K:String) 8 | 9 | object Test { 10 | 11 | def main ( args: Array[String] ) { 12 | val conf = new SparkConf() 13 | .setAppName("Test") 14 | 15 | val sc = new SparkContext(conf) 16 | 17 | val spark = SparkSession 18 | .builder() 19 | .config(conf) 20 | .getOrCreate() 21 | 22 | import spark.implicits._ 23 | explain(true) 24 | 25 | val W = sc.textFile(args(0)).zipWithIndex.map{ case (line,i) => (i, words(line))}.toDS() 26 | val K = sc.textFile(args(1)).zipWithIndex.map{ case (line,i) => (i, keys(line))}.toDS() 27 | 28 | val w = W.count() 29 | val k = K.count() 30 | 31 | W.createOrReplaceTempView("W") 32 | K.createOrReplaceTempView("K") 33 | 34 | s(sc,""" 35 | var C: map[String,Int] = map(); 36 | 37 | for j = 0, k-1 do{ 38 | C[K[j].K] := 0; 39 | for i = 0, w-1 do 40 | if (W[i].W == K[j].K) 41 | C[K[j].K] += 1; 42 | }; 43 | println(C); 44 | """) 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/StringMatch3.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.sql._ 4 | import org.apache.spark.rdd._ 5 | 6 | object Test { 7 | 8 | def main ( args: Array[String] ) { 9 | val conf = new SparkConf() 10 | .setAppName("Test") 11 | 12 | val sc = new SparkContext(conf) 13 | 14 | val spark = SparkSession 15 | .builder() 16 | .config(conf) 17 | .getOrCreate() 18 | 19 | import spark.implicits._ 20 | explain(true) 21 | 22 | val W = sc.textFile(args(0)).zipWithIndex.map{ case (line,i) => (i, line)}.toDS() 23 | val keys = List("key1", "key2", "key3") 24 | val K = keys.zipWithIndex.map{ case (line,i) => (i.toLong, line)}.toDS() 25 | 26 | val w = W.count() 27 | val k = K.count() 28 | 29 | W.createOrReplaceTempView("W") 30 | K.createOrReplaceTempView("K") 31 | 32 | s(sc,""" 33 | var C: map[String,Int] = map(); 34 | 35 | for j = 0, k-1 do{ 36 | C[K[j]] := 0; 37 | for i = 0, w-1 do 38 | if (W[i] == K[j]) 39 | C[K[j]] += 1; 40 | }; 41 | println(C); 42 | """) 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/StringMatch4.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.sql._ 4 | import org.apache.spark.rdd._ 5 | 6 | object Test { 7 | 8 | def main ( args: Array[String] ) { 9 | val conf = new SparkConf() 10 | .setAppName("Test") 11 | 12 | val sc = new SparkContext(conf) 13 | 14 | val spark = SparkSession 15 | .builder() 16 | .config(conf) 17 | .getOrCreate() 18 | 19 | import spark.implicits._ 20 | explain(true) 21 | 22 | val W = sc.textFile(args(0)).zipWithIndex.map{ case (line,i) => (i, line)}.toDS() 23 | val keys = List("key1", "key2", "key3") 24 | val K = keys.zipWithIndex.map{ case (line,i) => (i.toLong, line)}.toDS() 25 | 26 | val w = W.count() 27 | val k = K.count() 28 | 29 | W.createOrReplaceTempView("W") 30 | K.createOrReplaceTempView("K") 31 | 32 | s(sc,""" 33 | var C: map[String,Int] = map(); 34 | 35 | for j = 0, k-1 do{ 36 | C[K[j]] := 0; 37 | for i = 0, w-1 do 38 | if (W[i] == K[j]) 39 | C[K[j]] += 1; 40 | }; 41 | 42 | println(C); 43 | """) 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/StringMatch5.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.sql._ 4 | import org.apache.spark.rdd._ 5 | 6 | object Test { 7 | 8 | def main ( args: Array[String] ) { 9 | val conf = new SparkConf() 10 | .setAppName("Test") 11 | 12 | val sc = new SparkContext(conf) 13 | 14 | val spark = SparkSession 15 | .builder() 16 | .config(conf) 17 | .getOrCreate() 18 | 19 | import spark.implicits._ 20 | explain(true) 21 | 22 | val W = sc.textFile(args(0)).zipWithIndex.map{ case (line,i) => (i, line)}.toDS() 23 | val keys = List("key1", "key2", "key3") 24 | val K = keys.zipWithIndex.map{ case (line,i) => (i.toLong, line)}.toDS() 25 | 26 | val w = W.count() 27 | val k = K.count() 28 | 29 | W.createOrReplaceTempView("W") 30 | K.createOrReplaceTempView("K") 31 | 32 | s(sc,""" 33 | var C: vector[String] = vector(); 34 | 35 | for i = 0, w-1 do { 36 | for j = 0, k-1 do 37 | if (W[i] == K[j]) 38 | C[j] := K[j]; 39 | }; 40 | println(C); 41 | """) 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/Sum.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.sql._ 4 | import org.apache.spark.rdd._ 5 | 6 | object Test { 7 | 8 | def main ( args: Array[String] ) { 9 | val conf = new SparkConf() 10 | .setAppName("Spark SQL Sum") 11 | .setMaster("local[2]") 12 | 13 | val sc = new SparkContext(conf) 14 | 15 | val spark = SparkSession 16 | .builder() 17 | .config(conf) 18 | .getOrCreate() 19 | 20 | import spark.implicits._ 21 | explain(true) 22 | 23 | val A = sc.textFile(args(0)).zipWithIndex.map{ case (line,i) => (i, line.toDouble) }.toDS() 24 | 25 | val n = A.count() 26 | A.createOrReplaceTempView("A") 27 | 28 | 29 | s(sc,""" 30 | var sum: Double = 0.0; 31 | 32 | for i = 0, 10 do { 33 | sum += A[i]; 34 | }; 35 | 36 | println(sum); 37 | """) 38 | 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/Sum2.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.sql._ 4 | import org.apache.spark.rdd._ 5 | 6 | object Test { 7 | 8 | def main ( args: Array[String] ) { 9 | val conf = new SparkConf() 10 | .setAppName("Spark SQL Sum") 11 | .setMaster("local[2]") 12 | 13 | val sc = new SparkContext(conf) 14 | 15 | val spark = SparkSession 16 | .builder() 17 | .config(conf) 18 | .getOrCreate() 19 | 20 | import spark.implicits._ 21 | explain(true) 22 | 23 | val A = sc.textFile(args(0)).zipWithIndex.map{ case (line,i) => (i, line.toDouble) }.toDS() 24 | val n = A.count() 25 | 26 | A.createOrReplaceTempView("A") 27 | A.printSchema() 28 | 29 | s(sc,""" 30 | var sum: Double = 0.0; 31 | 32 | for i = 0, n-1 do { 33 | sum += A[i]; 34 | }; 35 | 36 | println(sum); 37 | """) 38 | 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/WordCount.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.spark.sql._ 5 | import Math._ 6 | 7 | case class wclass(W: String) 8 | 9 | object Test { 10 | 11 | def main ( args: Array[String] ) { 12 | val conf = new SparkConf().setAppName("Test") 13 | val sc = new SparkContext(conf) 14 | 15 | val spark = SparkSession 16 | .builder() 17 | .config(conf) 18 | .getOrCreate() 19 | 20 | import spark.implicits._ 21 | explain(true) 22 | 23 | val D = sc.textFile(args(0)) 24 | .flatMap( line => line.split(" ") ).zipWithIndex.map{ case (line,i) => (i,wclass(line)) }.toDS() 25 | 26 | D.createOrReplaceTempView("D") 27 | 28 | s(sc,""" 29 | 30 | var C: map[String,Int] = map(); 31 | 32 | for i = 0, 5 do 33 | C[D[i].W] += 1; 34 | 35 | println(C); 36 | """) 37 | 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/WordCount2.scala: -------------------------------------------------------------------------------- 1 | import edu.uta.diql._ 2 | import org.apache.spark._ 3 | import org.apache.spark.rdd._ 4 | import org.apache.spark.sql._ 5 | import Math._ 6 | 7 | //case class wclass(W: String) 8 | 9 | object Test { 10 | 11 | def main ( args: Array[String] ) { 12 | val conf = new SparkConf().setAppName("Test") 13 | val sc = new SparkContext(conf) 14 | 15 | val spark = SparkSession 16 | .builder() 17 | .config(conf) 18 | .getOrCreate() 19 | 20 | import spark.implicits._ 21 | explain(true) 22 | 23 | val D = sc.textFile(args(0)) 24 | .flatMap( line => line.split(" ") ).zipWithIndex.map{ case (line,i) => (i,(line)) }.toDS() 25 | 26 | val n = D.count() 27 | D.createOrReplaceTempView("D") 28 | 29 | s(sc,""" 30 | 31 | var C: map[String,Int] = map(); 32 | 33 | for i = 0, n-1 do 34 | C[D[i]] += 1; 35 | 36 | println(C); 37 | """) 38 | 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/build: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../../..; pwd -P)" 4 | 5 | if [ -z ${SPARK_HOME} ]; then 6 | SPARK_HOME=~/spark-2.2.0-bin-hadoop2.6 7 | fi 8 | 9 | JARS=. 10 | for I in ${SPARK_HOME}/jars/*.jar; do 11 | JARS=${JARS}:$I 12 | done 13 | for I in ${SPARK_HOME}/lib/*.jar; do 14 | JARS=${JARS}:$I 15 | done 16 | 17 | mkdir -p classes 18 | scalac -d classes -cp ${JARS}:${DIQL_HOME}/lib/diql-spark.jar $* 19 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/classes/Cclass$.class: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/tests/sqlgen/spark/classes/Cclass$.class -------------------------------------------------------------------------------- /tests/sqlgen/spark/classes/Cclass.class: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/tests/sqlgen/spark/classes/Cclass.class -------------------------------------------------------------------------------- /tests/sqlgen/spark/classes/Test$$typecreator5$1.class: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/tests/sqlgen/spark/classes/Test$$typecreator5$1.class -------------------------------------------------------------------------------- /tests/sqlgen/spark/classes/Test$.class: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/tests/sqlgen/spark/classes/Test$.class -------------------------------------------------------------------------------- /tests/sqlgen/spark/classes/Test.class: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/tests/sqlgen/spark/classes/Test.class -------------------------------------------------------------------------------- /tests/sqlgen/spark/group-by.txt: -------------------------------------------------------------------------------- 1 | 1,50 2 | 2,100 3 | 3,150 4 | 4,200 5 | 5,250 6 | 1,50 7 | 2,100 8 | 3,150 9 | 4,200 10 | 5,250 11 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/histogram.txt: -------------------------------------------------------------------------------- 1 | 1,4,7 2 | 1,4,7 3 | 2,5,8 4 | 2,5,8 5 | 3,6,9 -------------------------------------------------------------------------------- /tests/sqlgen/spark/k.txt: -------------------------------------------------------------------------------- 1 | 0 2 | 1 3 | 2 4 | 3 5 | 4 6 | 5 7 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/key.txt: -------------------------------------------------------------------------------- 1 | key1 2 | key2 3 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/m-matrix-small.txt: -------------------------------------------------------------------------------- 1 | 1,1,-2.0 2 | 0,0,5.0 3 | 2,2,6.0 4 | 0,1,-3.0 5 | 3,2,7.0 6 | 0,2,-1.0 7 | 1,0,3.0 8 | 1,2,4.0 9 | 2,0,1.0 10 | 3,0,-4.0 11 | 3,1,2.0 12 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/matrix-fact.txt: -------------------------------------------------------------------------------- 1 | 0,0,5.0 2 | 0,1,3.0 3 | 0,2,1.0 4 | 1,0,9.0 5 | 1,1,2.0 6 | 1,2,4.0 7 | 2,0,2.0 8 | 2,1,7.0 9 | 2,2,6.0 10 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/n-matrix-small.txt: -------------------------------------------------------------------------------- 1 | 1,0,3.0 2 | 0,0,5.0 3 | 1,2,-2.0 4 | 2,0,9.0 5 | 0,1,-3.0 6 | 0,2,-1.0 7 | 1,1,8.0 8 | 2,1,4.0 9 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/page-rank.txt: -------------------------------------------------------------------------------- 1 | 1,1 2 | 1,2 3 | 2,1 4 | 2,3 5 | 3,3 6 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/pca.txt: -------------------------------------------------------------------------------- 1 | 0,0,1 2 | 0,1,2 3 | 0,2,3 4 | 0,3,0 5 | 1,0,2 6 | 1,1,4 7 | 1,2,6 8 | 1,3,0 9 | 2,0,30 10 | 2,1,6 11 | 2,2,90 12 | 2,3,0 13 | 3,0,1 14 | 3,1,3 15 | 3,2,2 16 | 3,3,4 17 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/run: -------------------------------------------------------------------------------- 1 | #/bin/bash 2 | 3 | DIQL_HOME="$(cd `dirname $0`/../../..; pwd -P)" 4 | 5 | if [ -z ${SPARK_HOME} ]; then 6 | SPARK_HOME=~/spark-2.2.0-bin-hadoop2.6 7 | fi 8 | 9 | export HADOOP_CONF_DIR= 10 | 11 | jar cf test.jar -C classes . 12 | 13 | ${SPARK_HOME}/bin/spark-submit --jars ${DIQL_HOME}/lib/diql-spark.jar --class Test --master local[2] test.jar $* 14 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/string.txt: -------------------------------------------------------------------------------- 1 | a 2 | bb 3 | c 4 | key1 5 | key2 6 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/sum.txt: -------------------------------------------------------------------------------- 1 | 1 2 | 2 3 | 3 4 | 4 5 | 5 6 | 6 7 | 7 8 | 8 9 | 9 10 | 10 11 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/test.jar: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/fegaras/DIQL/2763ecdaf07287f4955c47d62312da3509c00f31/tests/sqlgen/spark/test.jar -------------------------------------------------------------------------------- /tests/sqlgen/spark/v.txt: -------------------------------------------------------------------------------- 1 | 0,0 2 | 1,50 3 | 2,100 4 | 3,150 5 | 4,200 6 | 5,250 7 | 1,50 8 | 2,100 9 | 3,150 10 | 4,200 11 | 5,250 12 | -------------------------------------------------------------------------------- /tests/sqlgen/spark/word-count.txt: -------------------------------------------------------------------------------- 1 | c 2 | b 3 | a 4 | a 5 | b 6 | a 7 | c 8 | b 9 | g 10 | --------------------------------------------------------------------------------