Repository: incubator-hivemall Updated Branches: refs/heads/master fc881c33d -> bd1431467
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/bd143146/spark/spark-2.3/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala ---------------------------------------------------------------------- diff --git a/spark/spark-2.3/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala b/spark/spark-2.3/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala new file mode 100644 index 0000000..ad23e8f --- /dev/null +++ b/spark/spark-2.3/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala @@ -0,0 +1,286 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.hive + +import java.io.{BufferedInputStream, BufferedReader, InputStream, InputStreamReader} +import java.net.URL +import java.util.UUID +import java.util.concurrent.{Executors, ExecutorService} + +import hivemall.mix.server.MixServer +import hivemall.utils.lang.CommandLineUtils +import hivemall.utils.net.NetUtils +import org.apache.commons.cli.Options +import org.apache.commons.compress.compressors.CompressorStreamFactory +import org.scalatest.BeforeAndAfter + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.HivemallLabeledPoint +import org.apache.spark.sql.{Column, DataFrame, Row} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.hive.HivemallGroupedDataset._ +import org.apache.spark.sql.hive.HivemallOps._ +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive.implicits._ +import org.apache.spark.test.TestUtils + +final class ModelMixingSuite extends SparkFunSuite with BeforeAndAfter { + + // Load A9a training and test data + val a9aLineParser = (line: String) => { + val elements = line.split(" ") + val (label, features) = (elements.head, elements.tail) + HivemallLabeledPoint(if (label == "+1") 1.0f else 0.0f, features) + } + + lazy val trainA9aData: DataFrame = + getDataFromURI( + new URL("http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary/a9a").openStream, + a9aLineParser) + + lazy val testA9aData: DataFrame = + getDataFromURI( + new URL("http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary/a9a.t").openStream, + a9aLineParser) + + // Load A9a training and test data + val kdd2010aLineParser = (line: String) => { + val elements = line.split(" ") + val (label, features) = (elements.head, elements.tail) + HivemallLabeledPoint(if (label == "1") 1.0f else 0.0f, features) + } + + lazy val trainKdd2010aData: DataFrame = + getDataFromURI( + new CompressorStreamFactory().createCompressorInputStream( + new BufferedInputStream( + new URL("http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary/kdda.bz2") + .openStream + ) + ), + kdd2010aLineParser, + 8) + + lazy val testKdd2010aData: DataFrame = + getDataFromURI( + new CompressorStreamFactory().createCompressorInputStream( + new BufferedInputStream( + new URL("http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary/kdda.t.bz2") + .openStream + ) + ), + kdd2010aLineParser, + 8) + + // Placeholder for a mix server + var mixServExec: ExecutorService = _ + var assignedPort: Int = _ + + private def getDataFromURI( + in: InputStream, lineParseFunc: String => HivemallLabeledPoint, numPart: Int = 2) + : DataFrame = { + val reader = new BufferedReader(new InputStreamReader(in)) + try { + // Cache all data because stream closed soon + val lines = FileIterator(reader.readLine()).toSeq + val rdd = TestHive.sparkContext.parallelize(lines, numPart).map(lineParseFunc) + val df = rdd.toDF.cache + df.foreach(_ => {}) + df + } finally { + reader.close() + } + } + + before { + assert(mixServExec == null) + + // Launch a MIX server as thread + assignedPort = NetUtils.getAvailablePort + val method = classOf[MixServer].getDeclaredMethod("getOptions") + method.setAccessible(true) + val options = method.invoke(null).asInstanceOf[Options] + val cl = CommandLineUtils.parseOptions( + Array( + "-port", Integer.toString(assignedPort), + "-sync_threshold", "1" + ), + options + ) + val server = new MixServer(cl) + mixServExec = Executors.newSingleThreadExecutor() + mixServExec.submit(server) + var retry = 0 + while (server.getState() != MixServer.ServerState.RUNNING && retry < 32) { + Thread.sleep(100L) + retry += 1 + } + assert(MixServer.ServerState.RUNNING == server.getState) + } + + after { + mixServExec.shutdownNow() + mixServExec = null + } + + TestUtils.benchmark("model mixing test w/ regression") { + Seq( + "train_adadelta", + "train_adagrad", + "train_arow_regr", + "train_arowe_regr", + "train_arowe2_regr", + "train_logregr", + "train_pa1_regr", + "train_pa1a_regr", + "train_pa2_regr", + "train_pa2a_regr" + ).map { func => + // Build a model + val model = { + val groupId = s"${TestHive.sparkContext.applicationId}-${UUID.randomUUID}" + val res = TestUtils.invokeFunc( + new HivemallOps(trainA9aData.part_amplify(lit(1))), + func, + Seq[Column]( + add_bias($"features"), + $"label", + lit(s"-mix localhost:${assignedPort} -mix_session ${groupId} -mix_threshold 2 " + + "-mix_cancel") + ) + ) + if (!res.columns.contains("conv")) { + res.groupBy("feature").agg("weight" -> "avg") + } else { + res.groupBy("feature").argmin_kld("weight", "conv") + } + }.toDF("feature", "weight") + + // Data preparation + val testDf = testA9aData + .select(rowid(), $"label".as("target"), $"features") + .cache + + val testDf_exploded = testDf + .explode_array($"features") + .select($"rowid", extract_feature($"feature"), extract_weight($"feature")) + + // Do prediction + val predict = testDf_exploded + .join(model, testDf_exploded("feature") === model("feature"), "LEFT_OUTER") + .select($"rowid", ($"weight" * $"value").as("value")) + .groupBy("rowid").sum("value") + .toDF("rowid", "predicted") + + // Evaluation + val eval = predict + .join(testDf, predict("rowid") === testDf("rowid")) + .groupBy() + .agg(Map("target" -> "avg", "predicted" -> "avg")) + .toDF("target", "predicted") + + val (target, predicted) = eval.map { + case Row(target: Double, predicted: Double) => (target, predicted) + }.first + + // scalastyle:off println + println(s"func:${func} target:${target} predicted:${predicted} " + + s"diff:${Math.abs(target - predicted)}") + + testDf.unpersist() + } + } + + TestUtils.benchmark("model mixing test w/ binary classification") { + Seq( + "train_perceptron", + "train_pa", + "train_pa1", + "train_pa2", + "train_cw", + "train_arow", + "train_arowh", + "train_scw", + "train_scw2", + "train_adagrad_rda" + ).map { func => + // Build a model + val model = { + val groupId = s"${TestHive.sparkContext.applicationId}-${UUID.randomUUID}" + val res = TestUtils.invokeFunc( + new HivemallOps(trainKdd2010aData.part_amplify(lit(1))), + func, + Seq[Column]( + add_bias($"features"), + $"label", + lit(s"-mix localhost:${assignedPort} -mix_session ${groupId} -mix_threshold 2 " + + "-mix_cancel") + ) + ) + if (!res.columns.contains("conv")) { + res.groupBy("feature").agg("weight" -> "avg") + } else { + res.groupBy("feature").argmin_kld("weight", "conv") + } + }.toDF("feature", "weight") + + // Data preparation + val testDf = testKdd2010aData + .select(rowid(), $"label".as("target"), $"features") + .cache + + val testDf_exploded = testDf + .explode_array($"features") + .select($"rowid", extract_feature($"feature"), extract_weight($"feature")) + + // Do prediction + val predict = testDf_exploded + .join(model, testDf_exploded("feature") === model("feature"), "LEFT_OUTER") + .select($"rowid", ($"weight" * $"value").as("value")) + .groupBy("rowid").sum("value") + .select($"rowid", when(sigmoid($"sum(value)") > 0.50, 1.0).otherwise(0.0)) + .toDF("rowid", "predicted") + + // Evaluation + val eval = predict + .join(testDf, predict("rowid") === testDf("rowid")) + .where($"target" === $"predicted") + + // scalastyle:off println + println(s"func:${func} precision:${(eval.count + 0.0) / predict.count}") + + testDf.unpersist() + predict.unpersist() + } + } +} + +object FileIterator { + + def apply[A](f: => A): Iterator[A] = new Iterator[A] { + var opt = Option(f) + def hasNext = opt.nonEmpty + def next() = { + val r = opt.get + opt = Option(f) + r + } + } +} http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/bd143146/spark/spark-2.3/src/test/scala/org/apache/spark/sql/hive/XGBoostSuite.scala ---------------------------------------------------------------------- diff --git a/spark/spark-2.3/src/test/scala/org/apache/spark/sql/hive/XGBoostSuite.scala b/spark/spark-2.3/src/test/scala/org/apache/spark/sql/hive/XGBoostSuite.scala new file mode 100644 index 0000000..89ed086 --- /dev/null +++ b/spark/spark-2.3/src/test/scala/org/apache/spark/sql/hive/XGBoostSuite.scala @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.hive + +import java.io.File + +import hivemall.xgboost._ + +import org.apache.spark.sql.Row +import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.hive.HivemallGroupedDataset._ +import org.apache.spark.sql.hive.HivemallOps._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.VectorQueryTest +import org.apache.spark.sql.types._ + +final class XGBoostSuite extends VectorQueryTest { + import hiveContext.implicits._ + + private val defaultOptions = XGBoostOptions() + .set("num_round", "10") + .set("max_depth", "4") + + private val numModles = 3 + + private def countModels(dirPath: String): Int = { + new File(dirPath).listFiles().toSeq.count(_.getName.endsWith(".xgboost")) + } + + test("resolve libxgboost") { + def getProvidingClass(name: String): Class[_] = + DataSource(sparkSession = null, className = name).providingClass + assert(getProvidingClass("libxgboost") === + classOf[org.apache.spark.sql.hive.source.XGBoostFileFormat]) + } + + test("check XGBoost options") { + assert(s"$defaultOptions" == "-max_depth 4 -num_round 10") + val errMsg = intercept[IllegalArgumentException] { + defaultOptions.set("unknown", "3") + } + assert(errMsg.getMessage == "requirement failed: " + + "non-existing key detected in XGBoost options: unknown") + } + + test("train_xgboost_regr") { + withTempModelDir { tempDir => + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + + // Save built models in persistent storage + mllibTrainDf.repartition(numModles) + .train_xgboost_regr($"features", $"label", lit(s"${defaultOptions}")) + .write.format("libxgboost").save(tempDir) + + // Check #models generated by XGBoost + assert(countModels(tempDir) == numModles) + + // Load the saved models + val model = hiveContext.sparkSession.read.format("libxgboost").load(tempDir) + val predict = model.join(mllibTestDf) + .xgboost_predict($"rowid", $"features", $"model_id", $"pred_model") + .groupBy("rowid").avg() + .toDF("rowid", "predicted") + + val result = predict.join(mllibTestDf, predict("rowid") === mllibTestDf("rowid"), "INNER") + .select(predict("rowid"), $"predicted", $"label") + + result.select(avg(abs($"predicted" - $"label"))).collect.map { + case Row(diff: Double) => assert(diff > 0.0) + } + } + } + } + + test("train_xgboost_classifier") { + withTempModelDir { tempDir => + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + + mllibTrainDf.repartition(numModles) + .train_xgboost_regr($"features", $"label", lit(s"${defaultOptions}")) + .write.format("libxgboost").save(tempDir) + + // Check #models generated by XGBoost + assert(countModels(tempDir) == numModles) + + val model = hiveContext.sparkSession.read.format("libxgboost").load(tempDir) + val predict = model.join(mllibTestDf) + .xgboost_predict($"rowid", $"features", $"model_id", $"pred_model") + .groupBy("rowid").avg() + .toDF("rowid", "predicted") + + val result = predict.join(mllibTestDf, predict("rowid") === mllibTestDf("rowid"), "INNER") + .select( + when($"predicted" >= 0.50, 1).otherwise(0), + $"label".cast(IntegerType) + ) + .toDF("predicted", "label") + + assert((result.where($"label" === $"predicted").count + 0.0) / result.count > 0.0) + } + } + } + + test("train_xgboost_multiclass_classifier") { + withTempModelDir { tempDir => + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + + mllibTrainDf.repartition(numModles) + .train_xgboost_multiclass_classifier( + $"features", $"label", lit(s"${defaultOptions.set("num_class", "2")}")) + .write.format("libxgboost").save(tempDir) + + // Check #models generated by XGBoost + assert(countModels(tempDir) == numModles) + + val model = hiveContext.sparkSession.read.format("libxgboost").load(tempDir) + val predict = model.join(mllibTestDf) + .xgboost_multiclass_predict($"rowid", $"features", $"model_id", $"pred_model") + .groupBy("rowid").max_label("probability", "label") + .toDF("rowid", "predicted") + + val result = predict.join(mllibTestDf, predict("rowid") === mllibTestDf("rowid"), "INNER") + .select( + predict("rowid"), + $"predicted", + $"label".cast(IntegerType) + ) + + assert((result.where($"label" === $"predicted").count + 0.0) / result.count > 0.0) + } + } + } +} http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/bd143146/spark/spark-2.3/src/test/scala/org/apache/spark/sql/hive/benchmark/MiscBenchmark.scala ---------------------------------------------------------------------- diff --git a/spark/spark-2.3/src/test/scala/org/apache/spark/sql/hive/benchmark/MiscBenchmark.scala b/spark/spark-2.3/src/test/scala/org/apache/spark/sql/hive/benchmark/MiscBenchmark.scala new file mode 100644 index 0000000..0a9e4a6 --- /dev/null +++ b/spark/spark-2.3/src/test/scala/org/apache/spark/sql/hive/benchmark/MiscBenchmark.scala @@ -0,0 +1,268 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.hive.benchmark + +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.encoders.RowEncoder +import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.benchmark.BenchmarkBaseAccessor +import org.apache.spark.sql.expressions.Window +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.hive.HivemallOps._ +import org.apache.spark.sql.hive.internal.HivemallOpsImpl._ +import org.apache.spark.sql.types._ +import org.apache.spark.test.TestUtils +import org.apache.spark.util.Benchmark + +class TestFuncWrapper(df: DataFrame) { + + def hive_each_top_k(k: Column, group: Column, value: Column, args: Column*) + : DataFrame = withTypedPlan { + planHiveGenericUDTF( + df.repartition(group).sortWithinPartitions(group), + "hivemall.tools.EachTopKUDTF", + "each_top_k", + Seq(k, group, value) ++ args, + Seq("rank", "key") ++ args.map { _.expr match { + case ua: UnresolvedAttribute => ua.name + }} + ) + } + + /** + * A convenient function to wrap a logical plan and produce a DataFrame. + */ + @inline private[this] def withTypedPlan(logicalPlan: => LogicalPlan): DataFrame = { + val queryExecution = df.sparkSession.sessionState.executePlan(logicalPlan) + val outputSchema = queryExecution.sparkPlan.schema + new Dataset[Row](df.sparkSession, queryExecution, RowEncoder(outputSchema)) + } +} + +object TestFuncWrapper { + + /** + * Implicitly inject the [[TestFuncWrapper]] into [[DataFrame]]. + */ + implicit def dataFrameToTestFuncWrapper(df: DataFrame): TestFuncWrapper = + new TestFuncWrapper(df) + + def sigmoid(exprs: Column*): Column = withExpr { + planHiveGenericUDF( + "hivemall.tools.math.SigmoidGenericUDF", + "sigmoid", + exprs + ) + } + + /** + * A convenient function to wrap an expression and produce a Column. + */ + @inline private def withExpr(expr: Expression): Column = Column(expr) +} + +class MiscBenchmark extends BenchmarkBaseAccessor { + + val numIters = 10 + + private def addBenchmarkCase(name: String, df: DataFrame)(implicit benchmark: Benchmark): Unit = { + benchmark.addCase(name, numIters) { + _ => df.queryExecution.executedPlan.execute().foreach(x => {}) + } + } + + TestUtils.benchmark("closure/exprs/spark-udf/hive-udf") { + /** + * Java HotSpot(TM) 64-Bit Server VM 1.8.0_31-b13 on Mac OS X 10.10.2 + * Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + * + * sigmoid functions: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + * -------------------------------------------------------------------------------- + * exprs 7708 / 8173 3.4 294.0 1.0X + * closure 7722 / 8342 3.4 294.6 1.0X + * spark-udf 7963 / 8350 3.3 303.8 1.0X + * hive-udf 13977 / 14050 1.9 533.2 0.6X + */ + import sparkSession.sqlContext.implicits._ + val N = 1L << 18 + val testDf = sparkSession.range(N).selectExpr("rand() AS value").cache + + // First, cache data + testDf.count + + implicit val benchmark = new Benchmark("sigmoid", N) + def sigmoidExprs(expr: Column): Column = { + val one: () => Literal = () => Literal.create(1.0, DoubleType) + Column(one()) / (Column(one()) + exp(-expr)) + } + addBenchmarkCase( + "exprs", + testDf.select(sigmoidExprs($"value")) + ) + implicit val encoder = RowEncoder(StructType(StructField("value", DoubleType) :: Nil)) + addBenchmarkCase( + "closure", + testDf.map { d => + Row(1.0 / (1.0 + Math.exp(-d.getDouble(0)))) + } + ) + val sigmoidUdf = udf { (d: Double) => 1.0 / (1.0 + Math.exp(-d)) } + addBenchmarkCase( + "spark-udf", + testDf.select(sigmoidUdf($"value")) + ) + addBenchmarkCase( + "hive-udf", + testDf.select(TestFuncWrapper.sigmoid($"value")) + ) + benchmark.run() + } + + TestUtils.benchmark("top-k query") { + /** + * Java HotSpot(TM) 64-Bit Server VM 1.8.0_31-b13 on Mac OS X 10.10.2 + * Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + * + * top-k (k=100): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + * ------------------------------------------------------------------------------- + * rank 62748 / 62862 0.4 2393.6 1.0X + * each_top_k (hive-udf) 41421 / 41736 0.6 1580.1 1.5X + * each_top_k (exprs) 15793 / 16394 1.7 602.5 4.0X + */ + import sparkSession.sqlContext.implicits._ + import TestFuncWrapper._ + val topK = 100 + val N = 1L << 20 + val numGroup = 3 + val testDf = sparkSession.range(N).selectExpr( + s"id % $numGroup AS key", "rand() AS x", "CAST(id AS STRING) AS value" + ).cache + + // First, cache data + testDf.count + + implicit val benchmark = new Benchmark(s"top-k (k=$topK)", N) + addBenchmarkCase( + "rank", + testDf.withColumn("rank", rank().over(Window.partitionBy($"key").orderBy($"x".desc))) + .where($"rank" <= topK) + ) + addBenchmarkCase( + "each_top_k (hive-udf)", + testDf.hive_each_top_k(lit(topK), $"key", $"x", $"key", $"value") + ) + addBenchmarkCase( + "each_top_k (exprs)", + testDf.each_top_k(lit(topK), $"x".as("score"), $"key".as("group")) + ) + benchmark.run() + } + + TestUtils.benchmark("top-k join query") { + /** + * Java HotSpot(TM) 64-Bit Server VM 1.8.0_31-b13 on Mac OS X 10.10.2 + * Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + * + * top-k join (k=3): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + * ------------------------------------------------------------------------------- + * join + rank 65959 / 71324 0.0 503223.9 1.0X + * join + each_top_k 66093 / 78864 0.0 504247.3 1.0X + * top_k_join 5013 / 5431 0.0 38249.3 13.2X + */ + import sparkSession.sqlContext.implicits._ + val topK = 3 + val N = 1L << 10 + val M = 1L << 10 + val numGroup = 3 + val inputDf = sparkSession.range(N).selectExpr( + s"CAST(rand() * $numGroup AS INT) AS group", "id AS userId", "rand() AS x", "rand() AS y" + ).cache + val masterDf = sparkSession.range(M).selectExpr( + s"id % $numGroup AS group", "id AS posId", "rand() AS x", "rand() AS y" + ).cache + + // First, cache data + inputDf.count + masterDf.count + + implicit val benchmark = new Benchmark(s"top-k join (k=$topK)", N) + // Define a score column + val distance = sqrt( + pow(inputDf("x") - masterDf("x"), lit(2.0)) + + pow(inputDf("y") - masterDf("y"), lit(2.0)) + ).as("score") + addBenchmarkCase( + "join + rank", + inputDf.join(masterDf, inputDf("group") === masterDf("group")) + .select(inputDf("group"), $"userId", $"posId", distance) + .withColumn( + "rank", rank().over(Window.partitionBy($"group", $"userId").orderBy($"score".desc))) + .where($"rank" <= topK) + ) + addBenchmarkCase( + "join + each_top_k", + inputDf.join(masterDf, inputDf("group") === masterDf("group")) + .each_top_k(lit(topK), distance, inputDf("group").as("group")) + ) + addBenchmarkCase( + "top_k_join", + inputDf.top_k_join(lit(topK), masterDf, inputDf("group") === masterDf("group"), distance) + ) + benchmark.run() + } + + TestUtils.benchmark("codegen top-k join") { + /** + * Java HotSpot(TM) 64-Bit Server VM 1.8.0_31-b13 on Mac OS X 10.10.2 + * Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + * + * top_k_join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + * ----------------------------------------------------------------------------------- + * top_k_join wholestage off 3 / 5 2751.9 0.4 1.0X + * top_k_join wholestage on 1 / 1 6494.4 0.2 2.4X + */ + val topK = 3 + val N = 1L << 23 + val M = 1L << 22 + val numGroup = 3 + val inputDf = sparkSession.range(N).selectExpr( + s"CAST(rand() * $numGroup AS INT) AS group", "id AS userId", "rand() AS x", "rand() AS y" + ).cache + val masterDf = sparkSession.range(M).selectExpr( + s"id % $numGroup AS group", "id AS posId", "rand() AS x", "rand() AS y" + ).cache + + // First, cache data + inputDf.count + masterDf.count + + // Define a score column + val distance = sqrt( + pow(inputDf("x") - masterDf("x"), lit(2.0)) + + pow(inputDf("y") - masterDf("y"), lit(2.0)) + ) + runBenchmark("top_k_join", N) { + inputDf.top_k_join(lit(topK), masterDf, inputDf("group") === masterDf("group"), + distance.as("score")) + } + } +} http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/bd143146/spark/spark-2.3/src/test/scala/org/apache/spark/sql/hive/test/HivemallFeatureQueryTest.scala ---------------------------------------------------------------------- diff --git a/spark/spark-2.3/src/test/scala/org/apache/spark/sql/hive/test/HivemallFeatureQueryTest.scala b/spark/spark-2.3/src/test/scala/org/apache/spark/sql/hive/test/HivemallFeatureQueryTest.scala new file mode 100644 index 0000000..bc656d1 --- /dev/null +++ b/spark/spark-2.3/src/test/scala/org/apache/spark/sql/hive/test/HivemallFeatureQueryTest.scala @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.hive.test + +import scala.collection.mutable.Seq +import scala.reflect.runtime.universe.TypeTag + +import hivemall.tools.RegressionDatagen + +import org.apache.spark.sql.{Column, QueryTest} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection} +import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.test.SQLTestUtils + +/** + * Base class for tests with Hivemall features. + */ +abstract class HivemallFeatureQueryTest extends QueryTest with SQLTestUtils with TestHiveSingleton { + + import hiveContext.implicits._ + + protected val DummyInputData = Seq((0, 0)).toDF("c0", "c1") + + protected val IntList2Data = + Seq( + (8 :: 5 :: Nil, 6 :: 4 :: Nil), + (3 :: 1 :: Nil, 3 :: 2 :: Nil), + (2 :: Nil, 3 :: Nil) + ).toDF("target", "predict") + + protected val Float2Data = + Seq( + (0.8f, 0.3f), (0.3f, 0.9f), (0.2f, 0.4f) + ).toDF("target", "predict") + + protected val TinyTrainData = + Seq( + (0.0, "1:0.8" :: "2:0.2" :: Nil), + (1.0, "2:0.7" :: Nil), + (0.0, "1:0.9" :: Nil) + ).toDF("label", "features") + + protected val TinyTestData = + Seq( + (0.0, "1:0.6" :: "2:0.1" :: Nil), + (1.0, "2:0.9" :: Nil), + (0.0, "1:0.2" :: Nil), + (0.0, "2:0.1" :: Nil), + (0.0, "0:0.6" :: "2:0.4" :: Nil) + ).toDF("label", "features") + + protected val LargeRegrTrainData = RegressionDatagen.exec( + hiveContext, + n_partitions = 2, + min_examples = 100000, + seed = 3, + prob_one = 0.8f + ).cache + + protected val LargeRegrTestData = RegressionDatagen.exec( + hiveContext, + n_partitions = 2, + min_examples = 100, + seed = 3, + prob_one = 0.5f + ).cache + + protected val LargeClassifierTrainData = RegressionDatagen.exec( + hiveContext, + n_partitions = 2, + min_examples = 100000, + seed = 5, + prob_one = 0.8f, + cl = true + ).cache + + protected val LargeClassifierTestData = RegressionDatagen.exec( + hiveContext, + n_partitions = 2, + min_examples = 100, + seed = 5, + prob_one = 0.5f, + cl = true + ).cache +} http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/bd143146/spark/spark-2.3/src/test/scala/org/apache/spark/sql/test/VectorQueryTest.scala ---------------------------------------------------------------------- diff --git a/spark/spark-2.3/src/test/scala/org/apache/spark/sql/test/VectorQueryTest.scala b/spark/spark-2.3/src/test/scala/org/apache/spark/sql/test/VectorQueryTest.scala new file mode 100644 index 0000000..4e2a0c1 --- /dev/null +++ b/spark/spark-2.3/src/test/scala/org/apache/spark/sql/test/VectorQueryTest.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.test + +import java.io.File +import java.nio.charset.StandardCharsets + +import com.google.common.io.Files + +import org.apache.spark.sql.{DataFrame, QueryTest} +import org.apache.spark.sql.hive.HivemallOps._ +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.util.Utils + +/** + * Base class for tests with SparkSQL VectorUDT data. + */ +abstract class VectorQueryTest extends QueryTest with SQLTestUtils with TestHiveSingleton { + + private var trainDir: File = _ + private var testDir: File = _ + + // A `libsvm` schema is (Double, ml.linalg.Vector) + protected var mllibTrainDf: DataFrame = _ + protected var mllibTestDf: DataFrame = _ + + override def beforeAll(): Unit = { + super.beforeAll() + val trainLines = + """ + |1 1:1.0 3:2.0 5:3.0 + |0 2:4.0 4:5.0 6:6.0 + |1 1:1.1 4:1.0 5:2.3 7:1.0 + |1 1:1.0 4:1.5 5:2.1 7:1.2 + """.stripMargin + trainDir = Utils.createTempDir() + Files.write(trainLines, new File(trainDir, "train-00000"), StandardCharsets.UTF_8) + val testLines = + """ + |1 1:1.3 3:2.1 5:2.8 + |0 2:3.9 4:5.3 6:8.0 + """.stripMargin + testDir = Utils.createTempDir() + Files.write(testLines, new File(testDir, "test-00000"), StandardCharsets.UTF_8) + + mllibTrainDf = spark.read.format("libsvm").load(trainDir.getAbsolutePath) + // Must be cached because rowid() is deterministic + mllibTestDf = spark.read.format("libsvm").load(testDir.getAbsolutePath) + .withColumn("rowid", rowid()).cache + } + + override def afterAll(): Unit = { + try { + Utils.deleteRecursively(trainDir) + Utils.deleteRecursively(testDir) + } finally { + super.afterAll() + } + } + + protected def withTempModelDir(f: String => Unit): Unit = { + var tempDir: File = null + try { + tempDir = Utils.createTempDir() + f(tempDir.getAbsolutePath + "/xgboost_models") + } catch { + case e: Throwable => fail(s"Unexpected exception detected: ${e}") + } finally { + Utils.deleteRecursively(tempDir) + } + } +} http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/bd143146/spark/spark-2.3/src/test/scala/org/apache/spark/streaming/HivemallOpsWithFeatureSuite.scala ---------------------------------------------------------------------- diff --git a/spark/spark-2.3/src/test/scala/org/apache/spark/streaming/HivemallOpsWithFeatureSuite.scala b/spark/spark-2.3/src/test/scala/org/apache/spark/streaming/HivemallOpsWithFeatureSuite.scala new file mode 100644 index 0000000..0e1372d --- /dev/null +++ b/spark/spark-2.3/src/test/scala/org/apache/spark/streaming/HivemallOpsWithFeatureSuite.scala @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.streaming + +import scala.reflect.ClassTag + +import org.apache.spark.ml.feature.HivemallLabeledPoint +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.hive.HivemallOps._ +import org.apache.spark.sql.hive.test.HivemallFeatureQueryTest +import org.apache.spark.streaming.HivemallStreamingOps._ +import org.apache.spark.streaming.dstream.InputDStream +import org.apache.spark.streaming.scheduler.StreamInputInfo + +/** + * This is an input stream just for tests. + */ +private[this] class TestInputStream[T: ClassTag]( + ssc: StreamingContext, + input: Seq[Seq[T]], + numPartitions: Int) extends InputDStream[T](ssc) { + + override def start() {} + + override def stop() {} + + override def compute(validTime: Time): Option[RDD[T]] = { + logInfo("Computing RDD for time " + validTime) + val index = ((validTime - zeroTime) / slideDuration - 1).toInt + val selectedInput = if (index < input.size) input(index) else Seq[T]() + + // lets us test cases where RDDs are not created + if (selectedInput == null) { + return None + } + + // Report the input data's information to InputInfoTracker for testing + val inputInfo = StreamInputInfo(id, selectedInput.length.toLong) + ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo) + + val rdd = ssc.sc.makeRDD(selectedInput, numPartitions) + logInfo("Created RDD " + rdd.id + " with " + selectedInput) + Some(rdd) + } +} + +final class HivemallOpsWithFeatureSuite extends HivemallFeatureQueryTest { + + // This implicit value used in `HivemallStreamingOps` + implicit val sqlCtx = hiveContext + + /** + * Run a block of code with the given StreamingContext. + * This method do not stop a given SparkContext because other tests share the context. + */ + private def withStreamingContext[R](ssc: StreamingContext)(block: StreamingContext => R): Unit = { + try { + block(ssc) + ssc.start() + ssc.awaitTerminationOrTimeout(10 * 1000) // 10s wait + } finally { + try { + ssc.stop(stopSparkContext = false) + } catch { + case e: Exception => logError("Error stopping StreamingContext", e) + } + } + } + + // scalastyle:off line.size.limit + + /** + * This test below fails sometimes (too flaky), so we temporarily ignore it. + * The stacktrace of this failure is: + * + * HivemallOpsWithFeatureSuite: + * Exception in thread "broadcast-exchange-60" java.lang.OutOfMemoryError: Java heap space + * at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57) + * at java.nio.ByteBuffer.allocate(ByteBuffer.java:331) + * at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$4.apply(TorrentBroadcast.scala:231) + * at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$4.apply(TorrentBroadcast.scala:231) + * at org.apache.spark.util.io.ChunkedByteBufferOutputStream.allocateNewChunkIfNeeded(ChunkedByteBufferOutputStream.scala:78) + * at org.apache.spark.util.io.ChunkedByteBufferOutputStream.write(ChunkedByteBufferOutputStream.scala:65) + * at net.jpountz.lz4.LZ4BlockOutputStream.flushBufferedData(LZ4BlockOutputStream.java:205) + * at net.jpountz.lz4.LZ4BlockOutputStream.finish(LZ4BlockOutputStream.java:235) + * at net.jpountz.lz4.LZ4BlockOutputStream.close(LZ4BlockOutputStream.java:175) + * at java.io.ObjectOutputStream$BlockDataOutputStream.close(ObjectOutputStream.java:1827) + * at java.io.ObjectOutputStream.close(ObjectOutputStream.java:741) + * at org.apache.spark.serializer.JavaSerializationStream.close(JavaSerializer.scala:57) + * at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$blockifyObject$1.apply$mcV$sp(TorrentBroadcast.scala:238) + * at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1296) + * at org.apache.spark.broadcast.TorrentBroadcast$.blockifyObject(TorrentBroadcast.scala:237) + * at org.apache.spark.broadcast.TorrentBroadcast.writeBlocks(TorrentBroadcast.scala:107) + * at org.apache.spark.broadcast.TorrentBroadcast.<init>(TorrentBroadcast.scala:86) + * at org.apache.spark.broadcast.TorrentBroadcastFactory.newBroadcast(TorrentBroadcastFactory.scala:34) + * ... + */ + + // scalastyle:on line.size.limit + + ignore("streaming") { + import sqlCtx.implicits._ + + // We assume we build a model in advance + val testModel = Seq( + ("0", 0.3f), ("1", 0.1f), ("2", 0.6f), ("3", 0.2f) + ).toDF("feature", "weight") + + withStreamingContext(new StreamingContext(sqlCtx.sparkContext, Milliseconds(100))) { ssc => + val inputData = Seq( + Seq(HivemallLabeledPoint(features = "1:0.6" :: "2:0.1" :: Nil)), + Seq(HivemallLabeledPoint(features = "2:0.9" :: Nil)), + Seq(HivemallLabeledPoint(features = "1:0.2" :: Nil)), + Seq(HivemallLabeledPoint(features = "2:0.1" :: Nil)), + Seq(HivemallLabeledPoint(features = "0:0.6" :: "2:0.4" :: Nil)) + ) + + val inputStream = new TestInputStream[HivemallLabeledPoint](ssc, inputData, 1) + + // Apply predictions on input streams + val prediction = inputStream.predict { streamDf => + val df = streamDf.select(rowid(), $"features").explode_array($"features") + val testDf = df.select( + // TODO: `$"feature"` throws AnalysisException, why? + $"rowid", extract_feature(df("feature")), extract_weight(df("feature")) + ) + testDf.join(testModel, testDf("feature") === testModel("feature"), "LEFT_OUTER") + .select($"rowid", ($"weight" * $"value").as("value")) + .groupBy("rowid").sum("value") + .toDF("rowid", "value") + .select($"rowid", sigmoid($"value")) + } + + // Dummy output stream + prediction.foreachRDD(_ => {}) + } + } +} http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/bd143146/spark/spark-2.3/src/test/scala/org/apache/spark/test/TestUtils.scala ---------------------------------------------------------------------- diff --git a/spark/spark-2.3/src/test/scala/org/apache/spark/test/TestUtils.scala b/spark/spark-2.3/src/test/scala/org/apache/spark/test/TestUtils.scala new file mode 100644 index 0000000..fa7b6e5 --- /dev/null +++ b/spark/spark-2.3/src/test/scala/org/apache/spark/test/TestUtils.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.test + +import scala.reflect.runtime.{universe => ru} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.DataFrame + +object TestUtils extends Logging { + + // Do benchmark if INFO-log enabled + def benchmark(benchName: String)(testFunc: => Unit): Unit = { + if (log.isDebugEnabled) { + testFunc + } + } + + def expectResult(res: Boolean, errMsg: String): Unit = if (res) { + logWarning(errMsg) + } + + def invokeFunc(cls: Any, func: String, args: Any*): DataFrame = try { + // Invoke a function with the given name via reflection + val im = scala.reflect.runtime.currentMirror.reflect(cls) + val mSym = im.symbol.typeSignature.member(ru.newTermName(func)).asMethod + im.reflectMethod(mSym).apply(args: _*) + .asInstanceOf[DataFrame] + } catch { + case e: Exception => + assert(false, s"Invoking ${func} failed because: ${e.getMessage}") + null // Not executed + } +} + +// TODO: Any same function in o.a.spark.*? +class TestFPWrapper(d: Double) { + + // Check an equality between Double/Float values + def ~==(d: Double): Boolean = Math.abs(this.d - d) < 0.001 +} + +object TestFPWrapper { + + @inline implicit def toTestFPWrapper(d: Double): TestFPWrapper = { + new TestFPWrapper(d) + } +}
